├── .gitignore ├── README ├── pom.xml └── src ├── main └── java │ ├── com │ └── tdunning │ │ └── plume │ │ ├── CombinerFn.java │ │ ├── DoFn.java │ │ ├── EmitFn.java │ │ ├── Ordering.java │ │ ├── PCollection.java │ │ ├── PTable.java │ │ ├── Pair.java │ │ ├── Plume.java │ │ ├── Tuple2.java │ │ ├── Tuple3.java │ │ ├── avro │ │ ├── AvroFile.java │ │ └── AvroTypes.java │ │ ├── local │ │ ├── eager │ │ │ ├── LocalCollection.java │ │ │ ├── LocalPlume.java │ │ │ └── LocalTable.java │ │ ├── lazy │ │ │ ├── ExecutionStep.java │ │ │ ├── LazyCollection.java │ │ │ ├── LazyPlume.java │ │ │ ├── LazyTable.java │ │ │ ├── LocalExecutor.java │ │ │ ├── MSCR.java │ │ │ ├── MSCRCombiner.java │ │ │ ├── MSCRMapper.java │ │ │ ├── MSCRReducer.java │ │ │ ├── MapRedExecutor.java │ │ │ ├── Optimizer.java │ │ │ ├── OptimizerTools.java │ │ │ ├── PlumeWorkflow.java │ │ │ └── op │ │ │ │ ├── CombineValues.java │ │ │ │ ├── DeferredOp.java │ │ │ │ ├── Flatten.java │ │ │ │ ├── GroupByKey.java │ │ │ │ ├── MultipleParallelDo.java │ │ │ │ ├── OneToOneOp.java │ │ │ │ └── ParallelDo.java │ │ └── mapReduce │ │ │ ├── Collector.java │ │ │ ├── MapReduce.java │ │ │ ├── MapReduceBuilder.java │ │ │ ├── Mapper.java │ │ │ ├── Reducer.java │ │ │ └── package.html │ │ ├── package.html │ │ └── types │ │ ├── BooleanType.java │ │ ├── BytesType.java │ │ ├── DoubleType.java │ │ ├── FloatType.java │ │ ├── IntegerType.java │ │ ├── LongType.java │ │ ├── PCollectionType.java │ │ ├── PTableType.java │ │ ├── PType.java │ │ ├── PairType.java │ │ ├── RecordType.java │ │ └── StringType.java │ └── org │ └── apache │ └── hadoop │ └── mapreduce │ └── lib │ └── input │ └── FileInputSplitWrapper.java └── test ├── java └── com │ └── tdunning │ └── plume │ ├── FlattenTest.java │ ├── WordCountTest.java │ ├── avro │ └── WriteAvroFile.java │ └── local │ ├── LogParseTest.java │ ├── lazy │ ├── BaseTestClass.java │ ├── BasicOptimizerTest.java │ ├── LocalExecutorTest.java │ ├── MapRedBypassTest.java │ ├── MapRedFlattenTest.java │ ├── MapRedMultipleGroupsTest.java │ ├── MapRedOnlyFlattensTest.java │ ├── MapRedSequenceFileTest.java │ ├── MapRedSingleFlattenChannelTest.java │ ├── MapRedTwoSequentialGBKTest.java │ ├── MapRedWordCountTest.java │ ├── TestOptimizer.java │ └── TestOptimizerTools.java │ └── mapReduce │ └── MapReduceTest.java └── resources ├── event2users.txt ├── eventslog.txt ├── log.txt ├── simple-text.avro └── simple-text.txt /.gitignore: -------------------------------------------------------------------------------- 1 | .idea/ 2 | *.iml 3 | target/ 4 | -------------------------------------------------------------------------------- /README: -------------------------------------------------------------------------------- 1 | Plume is a (so far) serial, eager approximate clone of FlumeJava. The intent is to experiment with the design of the API both to understand the design decisions the Google team made and to see if there are good alternatives. 2 | 3 | The ultimate goal is to provide something comparable to FlumeJava on top of Hadoop, but with a much more flexible execution model so that it is easy and efficient to code small problems using Plume as well as large ones. My theory is that small problems often grow into large ones and it is really nice to not have to re-implement everything as scaling happens. 4 | -------------------------------------------------------------------------------- /pom.xml: -------------------------------------------------------------------------------- 1 | 2 | 5 | 4.0.0 6 | 7 | com.tdunning 8 | plume 9 | 1.0 10 | 11 | 12 | 16 | 17 | Apache Snapshots 18 | http://repository.apache.org/snapshots/ 19 | 20 | true 21 | 22 | 23 | 24 | JBoss 25 | http://repository.jboss.org/nexus/content/groups/public/ 26 | 27 | true 28 | 29 | 30 | 31 | Apache Snapshots-group 32 | https://repository.apache.org/content/repositories/snapshots-group/ 33 | 34 | true 35 | 36 | 37 | true 38 | 39 | 40 | 41 | 42 | 43 | 44 | junit 45 | junit 46 | 4.13.1 47 | 48 | 49 | com.google.guava 50 | guava 51 | r03 52 | 53 | 54 | org.apache.avro 55 | avro 56 | 1.4.0-SNAPSHOT 57 | 58 | 59 | org.apache.hadoop 60 | hadoop-common 61 | 0.21.0-SNAPSHOT 62 | 63 | 64 | org.apache.hadoop 65 | hadoop-mapred 66 | 0.21.0-SNAPSHOT 67 | 68 | 69 | org.apache.hadoop 70 | hadoop-hdfs 71 | 0.21.0-SNAPSHOT 72 | 73 | 74 | org.slf4j 75 | slf4j-log4j12 76 | 1.6.1 77 | 78 | 79 | 80 | 81 | 82 | 83 | org.apache.maven.plugins 84 | maven-compiler-plugin 85 | 86 | 1.6 87 | 1.6 88 | 89 | 90 | 91 | 92 | 93 | 94 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/CombinerFn.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * Describes the interface for an associative aggregation function that can be applied one or more 22 | * times. Since this function is associative, it can be used as a combiner in addition to being 23 | * part of reducer. 24 | */ 25 | public abstract class CombinerFn { 26 | public abstract T combine(Iterable stuff); 27 | } 28 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/DoFn.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * Describes the key functional object that processes an input record and 22 | * outputs zero or more objects by means of an emitter object. 23 | */ 24 | public abstract class DoFn { 25 | public abstract void process(In v, EmitFn emitter); 26 | 27 | @Override 28 | public String toString() { 29 | String f = super.toString(); 30 | return "DoFn " + f.substring(f.indexOf("@") + 1, f.length()); 31 | } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/EmitFn.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * Describes the interface for an object used to emit results from a DoFn. 22 | */ 23 | public abstract class EmitFn { 24 | public abstract void emit(Out v); 25 | } 26 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/Ordering.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * Placeholder interface for ordering collection results. We should investigate what 22 | * guava provides with respect to ordering predicates before deciding whether to just 23 | * use java Comparator's. 24 | */ 25 | public abstract class Ordering { 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/PCollection.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | import com.tdunning.plume.types.PCollectionType; 21 | import com.tdunning.plume.types.PTableType; 22 | 23 | /** 24 | * Parallel collection. 25 | */ 26 | public interface PCollection extends Iterable { 27 | public PCollection map(DoFn fn, PCollectionType t); 28 | public PTable map(DoFn> fn, PTableType t); 29 | 30 | // derived operations 31 | 32 | public PTable count(); 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/PTable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | import com.tdunning.plume.types.PCollectionType; 21 | import com.tdunning.plume.types.PTableType; 22 | 23 | /** 24 | * Parallel table that supports map and reduce operations. 25 | */ 26 | public interface PTable extends PCollection>, Iterable> { 27 | 28 | /** 29 | * Performs a function on each element of a parallel table returning a collection of values. 30 | * 31 | * @param fn The function to perform. 32 | * @return A parallel collection whose content is the result of applying fn to each element of 33 | * this. 34 | */ 35 | public PCollection map(DoFn, R> fn, PCollectionType type); 36 | 37 | /** 38 | * Performs an operation on each element of a collection returning a transformed table. 39 | * 40 | * @param fn The function to perform on key/value pairs. 41 | * @return A parallel table containing the transformed data. 42 | */ 43 | public PTable map(DoFn, Pair> fn, PTableType type); 44 | 45 | /** 46 | * Groups the elements of a table by key returning a new table with the same keys, but all values 47 | * for the same key grouped together. 48 | * 49 | * @return The grouped table. 50 | */ 51 | public PTable> groupByKey(); 52 | 53 | /** 54 | * Groups the elements of a table by key returning a new table with the same keys, but all values 55 | * for the same key grouped together and in the order specified by the ordering. 56 | * 57 | * @param order Determines the ordering of the values for each key 58 | * @return 59 | */ 60 | public PTable> groupByKey(Ordering order); 61 | 62 | // TODO how can we state that V is Iterable for this one method? 63 | 64 | /** 65 | * Applies (possibly recursively) an associative function to elements of lists contained in a 66 | * table. 67 | * 68 | * @param fn The combination function transformation. 69 | * @return A table containing the combined values. 70 | */ 71 | public PTable combine(CombinerFn fn); 72 | 73 | // derived operations 74 | 75 | public PTable, Iterable>> join(PTable other); 76 | } 77 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/Pair.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * A pair of values, generally used as a key and value as the input or output of 22 | * a DoFn in the case where the input or output respectively are a PTable. 23 | * 24 | * IF cons'ing lots of Pairs becomes a problem, we may need to make this class 25 | * final to allow better compiler optimizations. 26 | */ 27 | public class Pair { 28 | private K key; 29 | private V value; 30 | 31 | public Pair(K key, V value) { 32 | this.key = key; 33 | this.value = value; 34 | } 35 | 36 | public static Pair create(V1 x1, V2 x2) { 37 | return new Pair(x1, x2); 38 | } 39 | 40 | public K getKey() { 41 | return key; 42 | } 43 | 44 | public V getValue() { 45 | return value; 46 | } 47 | 48 | @Override 49 | public String toString() { 50 | return "Pair{" + 51 | "key=" + key + 52 | ", value=" + value + 53 | '}'; 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/Plume.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | import java.io.IOException; 21 | 22 | import com.tdunning.plume.types.*; 23 | 24 | /** 25 | * A plume provides the runtime support for writing data-parallel programs. Each Plume implementation 26 | * defines a mode of execution. For instance, the local.eager.LocalPlume provides instant execution 27 | * without execution planning or any parallel implementation. 28 | */ 29 | public abstract class Plume { 30 | // general collection operations 31 | public abstract PCollection readTextFile(String name) throws IOException; 32 | public abstract PCollection readResourceFile(String name) throws IOException; 33 | public abstract PCollection readAvroFile(String name, PType type); 34 | public abstract PCollection fromJava(Iterable source); 35 | public abstract PCollection flatten(PCollection... args); 36 | 37 | public abstract void writeAvroFile(String name, PCollection data, PType type) throws IOException; 38 | 39 | public static StringType strings() { return new StringType(); } 40 | public static IntegerType integers() { return new IntegerType(); } 41 | public static LongType longs() { return new LongType(); } 42 | public static FloatType floats() { return new FloatType(); } 43 | public static DoubleType doubles() { return new DoubleType(); } 44 | public static BytesType bytes() { return new BytesType(); } 45 | public static BooleanType booleans() { return new BooleanType(); } 46 | 47 | public static PTableType tableOf(PType keyType, PType valueType) { 48 | return new PTableType(keyType, valueType); 49 | } 50 | 51 | public static PCollectionType collectionOf(PType elementType) { 52 | return new PCollectionType(elementType); 53 | } 54 | 55 | public static RecordType recordsOf(Class recordClass) { 56 | return new RecordType(recordClass); 57 | } 58 | 59 | } 60 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/Tuple2.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * Container for two objects used during joins. It is an open question whether a Pair should just 22 | * be a Tuple2 (or vice versa). 23 | */ 24 | public class Tuple2 { 25 | private V0 v0; 26 | private V1 v1; 27 | 28 | private Tuple2(V0 v0, V1 v1) { 29 | this.v0 = v0; 30 | this.v1 = v1; 31 | } 32 | 33 | public static Tuple2 create(V0 v0, V1 v1) { 34 | return new Tuple2(v0, v1); 35 | } 36 | 37 | public V0 get0() { 38 | return v0; 39 | } 40 | 41 | public V1 get1() { 42 | return v1; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/Tuple3.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | /** 21 | * A triple of three objects. 22 | */ 23 | public class Tuple3 { 24 | public Tuple3(V1 v1, V2 v2, V3 v3) { 25 | //To change body of created methods use File | Settings | File Templates. 26 | } 27 | 28 | public static Tuple3 create(V1 v1, V2 v2, V3 v3) { 29 | return new Tuple3(v1, v2, v3); 30 | } 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/avro/AvroFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.avro; 19 | 20 | import java.io.*; 21 | import java.util.Iterator; 22 | import java.util.Map; 23 | 24 | import com.tdunning.plume.types.PCollectionType; 25 | import com.tdunning.plume.types.PTableType; 26 | import org.apache.avro.Schema; 27 | import org.apache.avro.specific.SpecificDatumReader; 28 | import org.apache.avro.file.DataFileStream; 29 | 30 | import com.google.common.collect.Maps; 31 | 32 | import com.tdunning.plume.*; 33 | import com.tdunning.plume.types.PType; 34 | import com.tdunning.plume.local.eager.*; 35 | 36 | /** A PCollection for an Avro file. */ 37 | public class AvroFile implements PCollection { 38 | private File file; 39 | private Schema schema; 40 | 41 | public AvroFile(String name, PType type) { 42 | this.file = new File(name); 43 | this.schema = AvroTypes.getSchema(type); 44 | } 45 | 46 | @Override 47 | public PCollection map(DoFn fn, PCollectionType type) { 48 | final LocalCollection r = new LocalCollection(); 49 | 50 | for (T t : this) { 51 | fn.process(t, new EmitFn() { 52 | @Override 53 | public void emit(R y) { 54 | r.getData().add(y); 55 | } 56 | }); 57 | } 58 | return r; 59 | } 60 | 61 | @Override 62 | public PTable map(DoFn> fn, PTableType type) { 63 | final LocalTable r = new LocalTable(); 64 | for (final T t : this) { 65 | fn.process(t, new EmitFn>() { 66 | @Override 67 | public void emit(Pair value) { 68 | r.getData().add(value); 69 | } 70 | }); 71 | } 72 | return r; 73 | } 74 | 75 | @Override 76 | public PTable count() { 77 | Map x = Maps.newHashMap(); 78 | for (T t : this) { 79 | Integer v = x.get(t); 80 | if (v == null) { 81 | x.put(t, 1); 82 | } else { 83 | x.put(t, v + 1); 84 | } 85 | } 86 | LocalTable r = new LocalTable(); 87 | for (T t : x.keySet()) { 88 | r.getData().add(new Pair(t, x.get(t))); 89 | } 90 | return r; 91 | } 92 | 93 | @Override 94 | public Iterator iterator() { 95 | try { 96 | final DataFileStream data = 97 | new DataFileStream 98 | (new BufferedInputStream(new FileInputStream(file)), 99 | new SpecificDatumReader(schema)); 100 | // wrapper that closes the file when iteration is complete 101 | return new Iterator() { 102 | public boolean hasNext() { 103 | boolean value = data.hasNext(); 104 | if (!value) { 105 | close(); 106 | } 107 | return value; 108 | } 109 | public T next() { return data.next(); } 110 | public void remove() { throw new UnsupportedOperationException(); } 111 | protected void finalize() { close(); } 112 | private void close() { 113 | try { 114 | data.close(); 115 | } catch (IOException e){ 116 | throw new RuntimeException(e); 117 | } 118 | } 119 | }; 120 | } catch (IOException e) { 121 | throw new RuntimeException(e); 122 | } 123 | } 124 | 125 | } 126 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/avro/AvroTypes.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.avro; 19 | 20 | import com.tdunning.plume.types.*; 21 | import com.tdunning.plume.types.PType.Kind; 22 | 23 | import org.apache.avro.Schema; 24 | import org.apache.avro.Schema.Type; 25 | import org.apache.avro.mapred.Pair; 26 | 27 | /** 28 | * Translate between Plume types to Avro types. 29 | */ 30 | public class AvroTypes { 31 | // no public ctor 32 | private AvroTypes() { 33 | } 34 | 35 | /** 36 | * Convert Plume types to Avro types. 37 | */ 38 | public static Schema getSchema(PType type) { 39 | switch (type.kind()) { 40 | case BOOLEAN: 41 | return Schema.create(Type.BOOLEAN); 42 | case BYTES: 43 | return Schema.create(Type.BYTES); 44 | case DOUBLE: 45 | return Schema.create(Type.DOUBLE); 46 | case FLOAT: 47 | return Schema.create(Type.FLOAT); 48 | case INTEGER: 49 | return Schema.create(Type.INT); 50 | case LONG: 51 | return Schema.create(Type.LONG); 52 | case PAIR: 53 | PairType pairType = (PairType) type; 54 | return Pair.getPairSchema( 55 | getSchema(pairType.keyType()), 56 | getSchema(pairType.valueType())); 57 | case COLLECTION: 58 | PType elementType = ((PCollectionType) type).elementType(); 59 | return Schema.createArray(getSchema(elementType)); 60 | case TABLE: 61 | PType keyType = ((PTableType) type).keyType(); 62 | PType valueType = ((PTableType) type).valueType(); 63 | // PTable is an Avro map 64 | if (keyType.kind() == Kind.STRING) { 65 | return Schema.createMap(getSchema(new PairType(keyType, valueType))); 66 | } 67 | return Schema.createArray(getSchema(keyType)); 68 | case RECORD: 69 | return ((RecordType) type).schema(); 70 | case STRING: 71 | return Schema.create(Type.STRING); 72 | default: 73 | throw new RuntimeException("Unknown type: " + type); 74 | } 75 | } 76 | 77 | /** 78 | * Convert Avro types to Plume types. 79 | */ 80 | public static PType getPType(Schema schema) { 81 | // TODO FIXME 82 | throw new RuntimeException("Not yet implemented."); 83 | } 84 | 85 | } 86 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/eager/LocalCollection.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.eager; 19 | 20 | import com.google.common.collect.Lists; 21 | import com.google.common.collect.Maps; 22 | import com.tdunning.plume.*; 23 | import com.tdunning.plume.types.PCollectionType; 24 | import com.tdunning.plume.types.PTableType; 25 | 26 | import java.util.Iterator; 27 | import java.util.List; 28 | import java.util.Map; 29 | 30 | /** 31 | * Completely local version of a PCollection. 32 | */ 33 | public class LocalCollection implements PCollection { 34 | private List data = Lists.newArrayList(); 35 | 36 | @Override 37 | public PCollection map(DoFn fn, PCollectionType type) { 38 | final LocalCollection r = new LocalCollection(); 39 | for (T t : data) { 40 | fn.process(t, new EmitFn() { 41 | @Override 42 | public void emit(R y) { 43 | r.data.add(y); 44 | } 45 | }); 46 | } 47 | return r; 48 | } 49 | 50 | @Override 51 | public PTable map(DoFn> fn, PTableType type) { 52 | final LocalTable r = new LocalTable(); 53 | for (final T t : data) { 54 | fn.process(t, new EmitFn>() { 55 | @Override 56 | public void emit(Pair value) { 57 | r.getData().add(value); 58 | } 59 | }); 60 | } 61 | return r; 62 | } 63 | 64 | @Override 65 | public PTable count() { 66 | Map x = Maps.newHashMap(); 67 | for (T t : data) { 68 | Integer v = x.get(t); 69 | if (v == null) { 70 | x.put(t, 1); 71 | } else { 72 | x.put(t, v + 1); 73 | } 74 | } 75 | LocalTable r = new LocalTable(); 76 | for (T t : x.keySet()) { 77 | r.getData().add(new Pair(t, x.get(t))); 78 | } 79 | return r; 80 | } 81 | 82 | public static LocalCollection wrap(Iterable data) { 83 | return new LocalCollection().addAll(data); 84 | } 85 | 86 | public LocalCollection addAll(Iterable data) { 87 | for (T t : data) { 88 | this.data.add(t); 89 | } 90 | return this; 91 | } 92 | 93 | public List getData() { 94 | return data; 95 | } 96 | 97 | /** 98 | * Returns an iterator over a set of elements of type T. 99 | * 100 | * @return an Iterator. 101 | */ 102 | public Iterator iterator() { 103 | return data.iterator(); 104 | } 105 | } 106 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/eager/LocalPlume.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.eager; 19 | 20 | import com.google.common.base.Charsets; 21 | import com.google.common.io.Files; 22 | import com.google.common.io.Resources; 23 | import com.tdunning.plume.*; 24 | import com.tdunning.plume.types.*; 25 | import com.tdunning.plume.avro.*; 26 | 27 | import org.apache.avro.Schema; 28 | import org.apache.avro.file.DataFileReader; 29 | import org.apache.avro.file.DataFileWriter; 30 | import org.apache.avro.specific.SpecificDatumReader; 31 | import org.apache.avro.specific.SpecificDatumWriter; 32 | 33 | import java.io.File; 34 | import java.io.IOException; 35 | 36 | /** 37 | * Local plume runtime. All files are local, all tasks are run in threads. Currently threads == thread 38 | */ 39 | public class LocalPlume extends Plume { 40 | @Override 41 | public LocalCollection readTextFile(String name) throws IOException { 42 | return LocalCollection.wrap(Files.readLines(new File(name), Charsets.UTF_8)); 43 | } 44 | 45 | @Override 46 | public PCollection readResourceFile(String name) throws IOException { 47 | return LocalCollection.wrap(Resources.readLines(Resources.getResource(name), Charsets.UTF_8)); 48 | } 49 | 50 | @Override 51 | public PCollection readAvroFile(String name, PType type) { 52 | return new AvroFile(name, type); 53 | } 54 | 55 | @Override 56 | public LocalCollection fromJava(Iterable data) { 57 | return new LocalCollection().addAll(data); 58 | } 59 | 60 | @Override 61 | public PCollection flatten(PCollection... args) { 62 | LocalCollection r = new LocalCollection(); 63 | for (PCollection arg : args) { 64 | if (arg instanceof LocalCollection) { 65 | r.addAll(((LocalCollection) arg).getData()); 66 | } else { 67 | throw new UnsupportedOperationException("Can't flatten a " + arg.getClass() + " onto local collections"); 68 | } 69 | } 70 | return r; 71 | } 72 | 73 | @Override 74 | public void writeAvroFile(String name, PCollection data, PType type) throws IOException { 75 | Schema schema = AvroTypes.getSchema(type); 76 | DataFileWriter factory = new DataFileWriter(new SpecificDatumWriter(schema)); 77 | DataFileWriter out = factory.create(schema, new File(name)); 78 | for (T t : data) { 79 | out.append(t); 80 | } 81 | out.close(); 82 | } 83 | } 84 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/eager/LocalTable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.eager; 19 | 20 | import com.google.common.collect.Lists; 21 | import com.google.common.collect.Maps; 22 | 23 | import com.tdunning.plume.types.PCollectionType; 24 | import com.tdunning.plume.types.PTableType; 25 | import com.tdunning.plume.CombinerFn; 26 | import com.tdunning.plume.DoFn; 27 | import com.tdunning.plume.EmitFn; 28 | import com.tdunning.plume.Ordering; 29 | import com.tdunning.plume.PCollection; 30 | import com.tdunning.plume.PTable; 31 | import com.tdunning.plume.Pair; 32 | import com.tdunning.plume.Tuple2; 33 | 34 | import java.util.Iterator; 35 | import java.util.List; 36 | import java.util.Map; 37 | 38 | /** 39 | * Completely local eager version of a PTable. 40 | */ 41 | public class LocalTable extends LocalCollection> implements PTable { 42 | 43 | private List> data = Lists.newArrayList(); 44 | 45 | /** 46 | * Performs a function on each element of a parallel table returning a collection of values. 47 | * 48 | * @param fn The function to perform. 49 | * @return A parallel collection whose content is the result of applying fn to each element of 50 | * this. 51 | */ 52 | @Override 53 | public PCollection map(DoFn, R> fn, PCollectionType type) { 54 | final LocalCollection r = new LocalCollection(); 55 | for (Pair v : data) { 56 | fn.process(v, new EmitFn() { 57 | @Override 58 | public void emit(R y) { 59 | r.getData().add(y); 60 | } 61 | }); 62 | } 63 | return r; 64 | } 65 | 66 | 67 | /** 68 | * Performs an operation on each element of a collection returning a transformed table. 69 | * 70 | * @param fn The function to perform on key/value pairs. 71 | * @return A parallel table containing the transformed data. 72 | */ 73 | @Override 74 | public PTable map(DoFn, Pair> fn, PTableType type) { 75 | final LocalTable r = new LocalTable(); 76 | for (Pair v : data) { 77 | fn.process(v, new EmitFn>() { 78 | @Override 79 | public void emit(Pair value) { 80 | r.getData().add(value); 81 | } 82 | 83 | }); 84 | } 85 | return r; 86 | } 87 | 88 | /** 89 | * Groups the elements of a table by key returning a new table with the same keys, but all values 90 | * for the same key grouped together. 91 | * 92 | * @return The grouped table. 93 | */ 94 | @Override 95 | public PTable> groupByKey() { 96 | // can't use a guava multimap here because identical key,value pairs would be suppressed. 97 | Map> r = Maps.newHashMap(); 98 | for (Pair v : data) { 99 | List values = r.get(v.getKey()); 100 | if (values == null) { 101 | values = Lists.newArrayList(); 102 | r.put(v.getKey(), values); 103 | } 104 | values.add(v.getValue()); 105 | } 106 | return LocalTable.wrap(r); 107 | } 108 | 109 | private static PTable> wrap(Map> data) { 110 | LocalTable> r = new LocalTable>(); 111 | List>> list = r.getData(); 112 | for (K k : data.keySet()) { 113 | list.add(Pair.>create(k, data.get(k))); 114 | } 115 | return r; 116 | } 117 | 118 | /** 119 | * Groups the elements of a table by key returning a new table with the same keys, but all values 120 | * for the same key grouped together and in the order specified by the ordering. 121 | * 122 | * @return A table of keys and groups. 123 | */ 124 | @Override 125 | public PTable> groupByKey(Ordering order) { 126 | throw new UnsupportedOperationException("Not implemented yet ... help by making this work"); 127 | } 128 | 129 | /** 130 | * Applies (possibly recursively) an associative function to elements of lists contained in a 131 | * table. 132 | * 133 | * @return A table containing the combined values. 134 | */ 135 | @Override 136 | public PTable combine(CombinerFn combiner) { 137 | final LocalTable r = new LocalTable(); 138 | for (final Pair x : data) { 139 | @SuppressWarnings({"unchecked"}) Iterable v = (Iterable) x.getValue(); 140 | r.getData().add(Pair.create(x.getKey(), combiner.combine(v))); 141 | } 142 | return r; 143 | } 144 | 145 | @Override 146 | public PTable, Iterable>> join(PTable other) { 147 | Map> m0 = Maps.newHashMap(); 148 | for (Pair kvPair : data) { 149 | List v = m0.get(kvPair.getKey()); 150 | if (v == null) { 151 | v = Lists.newArrayList(); 152 | m0.put(kvPair.getKey(), v); 153 | } 154 | v.add(kvPair.getValue()); 155 | } 156 | Map> m1 = Maps.newHashMap(); 157 | for (Pair kvPair : ((LocalTable) other).getData()) { 158 | List v = m1.get(kvPair.getKey()); 159 | if (v == null) { 160 | v = Lists.newArrayList(); 161 | m1.put(kvPair.getKey(), v); 162 | } 163 | v.add(kvPair.getValue()); 164 | } 165 | 166 | LocalTable, Iterable>> z = new LocalTable, Iterable>>(); 167 | for (K k : m0.keySet()) { 168 | Iterable v0 = m0.get(k); 169 | Iterable v1 = m1.get(k); 170 | if (v1 == null) { 171 | v1 = Lists.newArrayList(); 172 | } else { 173 | m1.remove(k); 174 | } 175 | z.getData().add(Pair.create(k, Tuple2.create(v0, v1))); 176 | } 177 | 178 | for (K k : m1.keySet()) { 179 | List v1 = m1.get(k); 180 | List v0 = m0.get(k); 181 | if (v0 == null) { 182 | v0 = Lists.newArrayList(); 183 | } 184 | z.getData().add(Pair.create(k, Tuple2., Iterable>create(v0, v1))); 185 | } 186 | return z; 187 | } 188 | 189 | 190 | public List> getData() { 191 | return data; 192 | } 193 | 194 | /** 195 | * Returns an iterator over a set of elements of type T. 196 | * 197 | * @return an Iterator. 198 | */ 199 | public Iterator> iterator() { 200 | return data.iterator(); 201 | } 202 | } 203 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/ExecutionStep.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.util.HashSet; 21 | import java.util.Set; 22 | 23 | /** 24 | * Defines an execution plan recursively from a first ExecutionStep. 25 | * All the MSCRs in an ExecutionStep can be executed concurrently. 26 | */ 27 | public class ExecutionStep { 28 | 29 | Set mscrSteps = new HashSet(); 30 | ExecutionStep nextStep; 31 | 32 | public Set getMscrSteps() { 33 | return mscrSteps; 34 | } 35 | 36 | public ExecutionStep getNextStep() { 37 | return nextStep; 38 | } 39 | 40 | @Override 41 | public String toString() { 42 | return mscrSteps+(nextStep != null ? " \n"+nextStep : ""); 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/LazyCollection.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.util.ArrayList; 21 | import java.util.Iterator; 22 | import java.util.List; 23 | 24 | import com.google.common.collect.Lists; 25 | import com.tdunning.plume.DoFn; 26 | import com.tdunning.plume.PCollection; 27 | import com.tdunning.plume.PTable; 28 | import com.tdunning.plume.Pair; 29 | import com.tdunning.plume.local.lazy.op.DeferredOp; 30 | import com.tdunning.plume.local.lazy.op.ParallelDo; 31 | import com.tdunning.plume.types.PCollectionType; 32 | import com.tdunning.plume.types.PTableType; 33 | 34 | /** 35 | * A LazyCollection that can be either materialized or unmaterialized. 36 | * Unmaterialized collections have a reference to the {@link DeferredOp} that creates them. 37 | */ 38 | public class LazyCollection implements PCollection { 39 | 40 | PCollectionType type; 41 | 42 | boolean materialized = false; 43 | private List data; 44 | private String file; // points to a file in local filesystem, if collection is materialized like that TODO to be better defined 45 | 46 | DeferredOp deferredOp; 47 | 48 | List downOps; 49 | 50 | String plumeId = ""; 51 | 52 | public String getPlumeId() { 53 | return plumeId; 54 | } 55 | 56 | public void setPlumeId(String plumeId) { 57 | this.plumeId = plumeId; 58 | } 59 | 60 | /** 61 | * Build a PCollection with materialized state 62 | * 63 | * @param data Concrete data from which to build the PCollection. 64 | */ 65 | public LazyCollection(Iterable data, PCollectionType type) { 66 | this.data = Lists.newArrayList(data); 67 | this.type = type; 68 | materialized = true; 69 | } 70 | 71 | public List getData() { 72 | return data; 73 | } 74 | 75 | /** 76 | * Unmaterialized PCollection constructor 77 | */ 78 | LazyCollection() { 79 | } 80 | 81 | protected void addDownOp(DeferredOp op) { 82 | if(downOps == null) { 83 | downOps = new ArrayList(); 84 | } 85 | downOps.add(op); 86 | } 87 | 88 | @Override 89 | public Iterator iterator() { 90 | if(materialized) { 91 | return data.iterator(); 92 | } else { 93 | throw new UnsupportedOperationException("Can't iterate over unmaterialized PCollection"); 94 | } 95 | } 96 | 97 | /** 98 | * Creates a new LazyCollection from a deferred operation 99 | * which maps a PCollection to another PCollection 100 | */ 101 | @Override 102 | public PCollection map(DoFn fn, PCollectionType type) { 103 | LazyCollection dest = new LazyCollection(); 104 | ParallelDo op = new ParallelDo(fn, this, dest); 105 | dest.deferredOp = op; 106 | dest.type = type; 107 | addDownOp(op); 108 | return dest; 109 | } 110 | 111 | /** 112 | * Creates a new LazyTable from a deferred operation 113 | * which maps a PCollection to a PTable 114 | */ 115 | public PTable map(DoFn> fn, PTableType type) { 116 | LazyTable dest = new LazyTable(); 117 | ParallelDo> op = new ParallelDo>(fn, this, dest); 118 | dest.deferredOp = op; 119 | dest.type = type; 120 | addDownOp(op); 121 | return dest; 122 | } 123 | 124 | public PCollectionType getType() { 125 | return type; 126 | } 127 | 128 | public DeferredOp getDeferredOp() { 129 | return deferredOp; 130 | } 131 | 132 | public void setDeferredOp(DeferredOp deferredOp) { 133 | this.deferredOp = deferredOp; 134 | } 135 | 136 | public boolean isMaterialized() { 137 | return materialized; 138 | } 139 | 140 | public List getDownOps() { 141 | return downOps; 142 | } 143 | 144 | /** 145 | * TODO 146 | */ 147 | @Override 148 | public PTable count() { 149 | throw new UnsupportedOperationException("Net yet implemented"); 150 | } 151 | 152 | public String getFile() { 153 | return file; 154 | } 155 | 156 | public void setFile(String file) { 157 | this.file = file; 158 | } 159 | 160 | @Override 161 | public String toString() { 162 | String n = super.toString(); 163 | n = n.substring(n.indexOf("@") + 1, n.length()); 164 | return (getPlumeId() != "" ? getPlumeId() : n); 165 | } 166 | } 167 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/LazyPlume.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.io.IOException; 21 | 22 | import com.google.common.base.Charsets; 23 | import com.google.common.collect.Lists; 24 | import com.google.common.io.Resources; 25 | import com.tdunning.plume.PCollection; 26 | import com.tdunning.plume.PTable; 27 | import com.tdunning.plume.Pair; 28 | import com.tdunning.plume.Plume; 29 | import com.tdunning.plume.avro.AvroFile; 30 | import com.tdunning.plume.local.lazy.op.Flatten; 31 | import com.tdunning.plume.types.PCollectionType; 32 | import com.tdunning.plume.types.PTableType; 33 | import com.tdunning.plume.types.PType; 34 | import com.tdunning.plume.types.StringType; 35 | 36 | /** 37 | * Runtime for Plume implementing deferred execution and optimization. 38 | */ 39 | public class LazyPlume extends Plume { 40 | 41 | /** 42 | * Just points to a file, doesn't read it 43 | * 44 | * @param 45 | * @param name 46 | * @return 47 | * @throws IOException 48 | */ 49 | public PCollection readFile(String name, PCollectionType type) throws IOException { 50 | LazyCollection coll = new LazyCollection(); 51 | coll.materialized = true; 52 | coll.type = type; 53 | coll.setFile(name); 54 | return coll; 55 | } 56 | 57 | public PCollection fromJava(Iterable source, PCollectionType type) { 58 | return new LazyCollection(source, type); 59 | } 60 | 61 | public PTable fromJava(Iterable> source, PTableType type) { 62 | return new LazyTable(source, type); 63 | } 64 | 65 | public PCollection flatten(PCollectionType type, PCollection... args) { 66 | LazyCollection dest = new LazyCollection(); 67 | Flatten flatten = new Flatten(Lists.newArrayList(args), dest); 68 | dest.deferredOp = flatten; 69 | dest.type = type; 70 | for(PCollection col: args) { 71 | ((LazyCollection)col).addDownOp(flatten); 72 | } 73 | return dest; 74 | } 75 | 76 | public PTable flatten(PTableType type, PCollection>... args) { 77 | LazyTable dest = new LazyTable(); 78 | Flatten> flatten = new Flatten>(Lists.newArrayList(args), dest); 79 | dest.deferredOp = flatten; 80 | dest.type = type; 81 | for(PCollection> col: args) { 82 | ((LazyCollection>)col).addDownOp(flatten); 83 | } 84 | return dest; 85 | } 86 | 87 | @Override 88 | public PCollection flatten(PCollection... args) { 89 | return flatten(((LazyCollection)args[0]).type, args); 90 | } 91 | 92 | @Override 93 | public void writeAvroFile(String name, PCollection data, PType type) { 94 | throw new RuntimeException("Not done"); 95 | } 96 | 97 | @Override 98 | public PCollection fromJava(Iterable source) { 99 | return fromJava(source, new PCollectionType(new StringType())); 100 | } 101 | 102 | @Override 103 | public PCollection readTextFile(String name) throws IOException { 104 | return readFile(name, new PCollectionType(new StringType())); 105 | } 106 | 107 | /** 108 | * I guess the convention is that resource files are small enough to be read in memory 109 | */ 110 | @Override 111 | public PCollection readResourceFile(String name) throws IOException { 112 | return fromJava(Resources.readLines(Resources.getResource(name), Charsets.UTF_8)); 113 | } 114 | 115 | @Override 116 | public PCollection readAvroFile(String name, PType type) { 117 | return new AvroFile(name, type); 118 | } 119 | } 120 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/LazyTable.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import com.tdunning.plume.CombinerFn; 21 | import com.tdunning.plume.DoFn; 22 | import com.tdunning.plume.Ordering; 23 | import com.tdunning.plume.PCollection; 24 | import com.tdunning.plume.PTable; 25 | import com.tdunning.plume.Pair; 26 | import com.tdunning.plume.Tuple2; 27 | import com.tdunning.plume.local.lazy.op.CombineValues; 28 | import com.tdunning.plume.local.lazy.op.DeferredOp; 29 | import com.tdunning.plume.local.lazy.op.GroupByKey; 30 | import com.tdunning.plume.local.lazy.op.ParallelDo; 31 | import com.tdunning.plume.types.PCollectionType; 32 | import com.tdunning.plume.types.PTableType; 33 | import com.tdunning.plume.types.PType; 34 | 35 | /** 36 | * A LazyTable that can be either materialized or unmaterialized. 37 | * Unmaterialized collections have a reference to the {@link DeferredOp} that creates them. 38 | */ 39 | public class LazyTable extends LazyCollection> implements PTable { 40 | 41 | public LazyTable() { 42 | } 43 | 44 | public LazyTable(Iterable> data, PTableType type) { 45 | super(data, type); 46 | } 47 | 48 | /** 49 | * Creates a new LazyCollection from a {@link ParallelDo} deferred operation 50 | * which maps a PTable to a PCollection 51 | */ 52 | @Override 53 | public PCollection map(DoFn, R> fn, PCollectionType type) { 54 | LazyCollection dest = new LazyCollection(); 55 | ParallelDo, R> op = new ParallelDo, R>(fn, this, dest); 56 | dest.deferredOp = op; 57 | dest.type = type; 58 | addDownOp(op); 59 | return dest; 60 | } 61 | 62 | /** 63 | * Creates a new LazyTable from a {@link ParallelDo} deferred operation 64 | * which maps a PTable to another PTable 65 | */ 66 | @Override 67 | public PTable map(DoFn, Pair> fn, PTableType type) { 68 | LazyTable dest = new LazyTable(); 69 | ParallelDo, Pair> op = new ParallelDo, Pair>(fn, this, dest); 70 | dest.deferredOp = op; 71 | dest.type = type; 72 | addDownOp(op); 73 | return dest; 74 | } 75 | 76 | /** 77 | * Creates a new PTable from a {@link ParallelDo} deferred operation 78 | */ 79 | @Override 80 | public PTable> groupByKey() { 81 | LazyTable> dest = new LazyTable>(); 82 | dest.deferredOp = new GroupByKey(this, dest); 83 | dest.type = type; // Not sure if this is essentially correct 84 | addDownOp(dest.deferredOp); 85 | return dest; 86 | } 87 | 88 | /** 89 | * Creates a new PTable from a {@link GroupByKey} deferred operation 90 | * TODO this looks wrong since it doesn't pay attention to the order 91 | */ 92 | @Override 93 | public PTable> groupByKey(Ordering order) { 94 | LazyTable> dest = new LazyTable>(); 95 | GroupByKey groupByKey = new GroupByKey(this, dest); 96 | dest.deferredOp = groupByKey; 97 | dest.type = type; // Not sure if this is essentially correct 98 | addDownOp(groupByKey); 99 | return dest; 100 | } 101 | 102 | /** 103 | * Creates a new PTable from a {@link CombineValues} deferred operation 104 | */ 105 | @Override 106 | public PTable combine(CombinerFn fn) { 107 | LazyTable dest = new LazyTable(); 108 | // TODO check how to do this better instead of unchecked casting 109 | CombineValues combine = new CombineValues(fn, (LazyTable>) this, dest); 110 | dest.deferredOp = combine; 111 | dest.type = type; 112 | addDownOp(combine); 113 | return dest; 114 | } 115 | 116 | /** 117 | * TODO 118 | */ 119 | @Override 120 | public PTable, Iterable>> join( 121 | PTable other) { 122 | throw new UnsupportedOperationException("Net yet implemented"); 123 | } 124 | } 125 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/LocalExecutor.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | import java.util.Map; 23 | 24 | import com.google.common.collect.Lists; 25 | import com.google.common.collect.Maps; 26 | import com.tdunning.plume.DoFn; 27 | import com.tdunning.plume.EmitFn; 28 | import com.tdunning.plume.PCollection; 29 | import com.tdunning.plume.Pair; 30 | import com.tdunning.plume.local.lazy.op.DeferredOp; 31 | import com.tdunning.plume.local.lazy.op.Flatten; 32 | import com.tdunning.plume.local.lazy.op.GroupByKey; 33 | import com.tdunning.plume.local.lazy.op.MultipleParallelDo; 34 | import com.tdunning.plume.local.lazy.op.ParallelDo; 35 | 36 | /** 37 | * Dummy local executor that goes down-top by using recursive formulas and stores all intermediate results in-memory. 38 | */ 39 | public class LocalExecutor { 40 | 41 | /** 42 | * Execute one-output flow 43 | * 44 | * @param 45 | * @param output 46 | * @return 47 | */ 48 | @SuppressWarnings({ "unchecked", "rawtypes" }) 49 | public Iterable execute(LazyCollection output) { 50 | if (output.isMaterialized()) { 51 | return output.getData(); // nothing else to execute 52 | } else { 53 | DeferredOp op = output.getDeferredOp(); 54 | final List result = Lists.newArrayList(); 55 | // Flatten op 56 | if(op instanceof Flatten) { 57 | Flatten flatten = (Flatten)op; 58 | for(PCollection col: flatten.getOrigins()) { 59 | Iterable res = execute((LazyCollection) col ); 60 | result.addAll(Lists.newArrayList(res)); 61 | } 62 | return result; // done with it 63 | } 64 | Iterable parent; 65 | EmitFn emitter = new EmitFn() { 66 | @Override 67 | public void emit(T v) { 68 | result.add(v); 69 | } 70 | }; 71 | // ParallelDo 72 | if (op instanceof ParallelDo) { 73 | ParallelDo pDo = (ParallelDo) op; 74 | parent = execute((LazyCollection)pDo.getOrigin()); 75 | for (Object obj : parent) { 76 | pDo.getFunction().process(obj, emitter); 77 | } 78 | // MultipleParallelDo -> parallel operations that read the same collection 79 | // In this version of executor, we will only compute the current collection, not its neighbors 80 | } else if(op instanceof MultipleParallelDo) { 81 | MultipleParallelDo mPDo = (MultipleParallelDo) op; 82 | parent = execute((LazyCollection)mPDo.getOrigin()); 83 | DoFn function = (DoFn)mPDo.getDests().get(output); // get the function that corresponds to this collection 84 | for (Object obj : parent) { 85 | function.process(obj, emitter); 86 | } 87 | // GroupByKey 88 | } else if(op instanceof GroupByKey) { 89 | GroupByKey gBK = (GroupByKey) op; 90 | parent = execute(gBK.getOrigin()); 91 | Map groupMap = Maps.newHashMap(); 92 | // Perform in-memory group by operation 93 | for (Object obj : parent) { 94 | Pair p = (Pair)obj; 95 | List list = groupMap.get(p.getKey()); 96 | if(list == null) { 97 | list = new ArrayList(); 98 | } 99 | list.add(p.getValue()); 100 | groupMap.put(p.getKey(), list); 101 | } 102 | for (Map.Entry entry: groupMap.entrySet()) { 103 | result.add((T)new Pair(entry.getKey(), entry.getValue())); 104 | } 105 | } 106 | return result; 107 | } 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/MSCR.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.util.HashMap; 21 | import java.util.HashSet; 22 | import java.util.Map; 23 | import java.util.Set; 24 | 25 | import com.tdunning.plume.PCollection; 26 | import com.tdunning.plume.Pair; 27 | import com.tdunning.plume.local.lazy.op.CombineValues; 28 | import com.tdunning.plume.local.lazy.op.Flatten; 29 | import com.tdunning.plume.local.lazy.op.GroupByKey; 30 | import com.tdunning.plume.local.lazy.op.ParallelDo; 31 | 32 | /** 33 | * The MSCR abstraction as in FlumeJava paper. MSCR stands for map-shuffle-combine-reduce, 34 | * but it is more general than the normal map-reduce operation such as in Hadoop because 35 | * multiple inputs can exist each with a separate change of map functions. Likewise, 36 | * there are grouping keys and multiple outputs, one for each grouping key. 37 | * 38 | * An MSCR can be converted to a conventional map-reduce by tagging inputs and creating 39 | * tagged union data structures. The purpose of the MSCR abstraction is that it provides 40 | * a very convenient target for optimizations since it is higher-level than a primitive 41 | * map-reduce (that makes the optimizer easier to write) but at the same time there are no 42 | * significant optimization opportunities lost by not looking below the level of the MSCR 43 | * operations. 44 | **/ 45 | public class MSCR { 46 | 47 | private int id; // this id identifies the MSCR in the execution tree 48 | 49 | public int getId() { 50 | return id; 51 | } 52 | 53 | private Set> inputs = new HashSet>(); 54 | private Map, OutputChannel> outputChannels = 55 | new HashMap, OutputChannel>(); 56 | 57 | private Map, Integer> numberedChannels = new HashMap, Integer>(); 58 | private Map> channelByNumber = new HashMap>(); 59 | 60 | private int nChannels = 0; 61 | 62 | public MSCR(int id) { 63 | this.id = id; 64 | } 65 | 66 | public static class OutputChannel { 67 | 68 | Flatten> flatten = null; 69 | GroupByKey shuffle = null; 70 | CombineValues combiner = null; 71 | ParallelDo, T> reducer = null; 72 | 73 | PCollection output = null; 74 | 75 | public OutputChannel(GroupByKey shuffle) { 76 | this.shuffle = shuffle; 77 | } 78 | 79 | // By-pass / Single flatten channel 80 | public OutputChannel(PCollection output) { 81 | this.output = output; 82 | } 83 | 84 | public Flatten> getFlatten() { 85 | return flatten; 86 | } 87 | 88 | public GroupByKey getShuffle() { 89 | return shuffle; 90 | } 91 | 92 | public CombineValues getCombiner() { 93 | return combiner; 94 | } 95 | 96 | public ParallelDo, T> getReducer() { 97 | return reducer; 98 | } 99 | 100 | @Override 101 | public String toString() { 102 | return "Output > " + output + ", FTn [" + flatten + "] SHfl [" + shuffle + "] COMb [" + combiner + "] RED [" + reducer + "]"; 103 | } 104 | } 105 | 106 | public Map, OutputChannel> getOutputChannels() { 107 | return outputChannels; 108 | } 109 | 110 | public Set> getInputs() { 111 | return inputs; 112 | } 113 | 114 | public void addInput(PCollection input) { 115 | inputs.add(input); 116 | } 117 | 118 | public boolean hasInput(PCollection input) { 119 | return inputs.contains(input); 120 | } 121 | 122 | public boolean hasOutputChannel(PCollection indexBy) { 123 | return outputChannels.containsKey(indexBy); 124 | } 125 | 126 | public void addOutputChannel(OutputChannel outputChannel) { 127 | nChannels++; 128 | // TODO explain this 129 | LazyCollection indexBy = (LazyCollection)outputChannel.output; 130 | indexBy.setPlumeId(id+"_"+nChannels); // 131 | if(outputChannel.shuffle != null) { 132 | indexBy = outputChannel.shuffle.getOrigin(); 133 | } 134 | outputChannels.put(indexBy, outputChannel); 135 | getNumberedChannels().put(indexBy, nChannels); 136 | getChannelByNumber().put(nChannels, indexBy); 137 | } 138 | 139 | public Map, Integer> getNumberedChannels() { 140 | return numberedChannels; 141 | } 142 | 143 | public Map> getChannelByNumber() { 144 | return channelByNumber; 145 | } 146 | 147 | @Override 148 | public String toString() { 149 | String str = "MSCR Id "+id+" #inputs "+inputs.size()+" #outputs "+outputChannels.size()+"\n" 150 | +" -Inputs- "; 151 | for(PCollection input: inputs) { 152 | str += input+" with down ops "+((LazyCollection)input).downOps+" | "; 153 | } 154 | str += "\n -Outputs- " + outputChannels; 155 | return str; 156 | } 157 | } 158 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/MSCRCombiner.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.io.IOException; 21 | import java.util.List; 22 | 23 | import org.apache.hadoop.io.WritableComparable; 24 | import org.apache.hadoop.mapreduce.Reducer; 25 | 26 | import com.google.common.collect.Lists; 27 | import com.tdunning.plume.PCollection; 28 | import com.tdunning.plume.local.lazy.MSCR.OutputChannel; 29 | import com.tdunning.plume.local.lazy.MapRedExecutor.PlumeObject; 30 | import com.tdunning.plume.local.lazy.op.GroupByKey; 31 | 32 | /** 33 | * Combiner that is used to executed MSCR in Map/reds 34 | */ 35 | public class MSCRCombiner extends Reducer { 36 | 37 | MSCR mscr; // Current MSCR being executed 38 | 39 | protected void setup(Reducer.Context context) 40 | throws IOException, InterruptedException { 41 | 42 | this.mscr = MapRedExecutor.readMSCR(context.getConfiguration()); 43 | } 44 | 45 | @SuppressWarnings("unchecked") 46 | protected void reduce(final PlumeObject arg0, java.lang.Iterable values, 47 | Reducer.Context context) 48 | throws IOException, InterruptedException { 49 | 50 | PCollection col = mscr.getChannelByNumber().get(arg0.sourceId); 51 | OutputChannel oC = mscr.getOutputChannels().get(col); 52 | if(oC.combiner != null) { 53 | // Apply combiner function for this channel 54 | List vals = Lists.newArrayList(); 55 | for(PlumeObject val: values) { 56 | vals.add(val.obj); 57 | } 58 | WritableComparable result = (WritableComparable) oC.combiner.getCombiner().combine(vals); 59 | context.write(arg0, new PlumeObject(result, arg0.sourceId)); 60 | } else { 61 | // direct writing - write all key, value pairs 62 | for(PlumeObject val: values) { 63 | context.write(arg0, val); 64 | } 65 | } 66 | } 67 | } 68 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/MSCRMapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.io.IOException; 21 | import java.util.Map; 22 | 23 | import org.apache.hadoop.io.NullWritable; 24 | import org.apache.hadoop.io.WritableComparable; 25 | import org.apache.hadoop.mapreduce.Mapper; 26 | import org.apache.hadoop.mapreduce.lib.input.FileInputSplitWrapper; 27 | import org.apache.hadoop.mapreduce.lib.input.FileSplit; 28 | import org.mortbay.log.Log; 29 | 30 | import com.tdunning.plume.DoFn; 31 | import com.tdunning.plume.EmitFn; 32 | import com.tdunning.plume.PCollection; 33 | import com.tdunning.plume.Pair; 34 | import com.tdunning.plume.local.lazy.MapRedExecutor.PlumeObject; 35 | import com.tdunning.plume.local.lazy.op.DeferredOp; 36 | import com.tdunning.plume.local.lazy.op.Flatten; 37 | import com.tdunning.plume.local.lazy.op.GroupByKey; 38 | import com.tdunning.plume.local.lazy.op.MultipleParallelDo; 39 | import com.tdunning.plume.types.PCollectionType; 40 | import com.tdunning.plume.types.PTableType; 41 | 42 | /** 43 | * Mapper that is used to execute MSCR in MapReds 44 | */ 45 | public class MSCRMapper extends Mapper { 46 | 47 | MSCR mscr; // Current MSCR being executed 48 | String tmpFolder; 49 | 50 | protected void setup(Mapper.Context context) 51 | throws IOException, InterruptedException { 52 | 53 | this.mscr = MapRedExecutor.readMSCR(context.getConfiguration()); 54 | this.tmpFolder = context.getConfiguration().get(MapRedExecutor.TEMP_OUTPUT_PATH); 55 | }; 56 | 57 | @SuppressWarnings("unchecked") 58 | protected void map(WritableComparable key, WritableComparable value, 59 | final Mapper.Context context) 60 | throws IOException, InterruptedException { 61 | 62 | LazyCollection l = null; 63 | 64 | FileSplit fS = FileInputSplitWrapper.getFileInputSplit(context); 65 | 66 | // Get LazyCollection for this input (according to FileSplit) 67 | for(PCollection input: mscr.getInputs()) { 68 | LazyCollection thisL = (LazyCollection)input; 69 | if(thisL.getFile() == null) { 70 | thisL.setFile(tmpFolder + "/" + thisL.getPlumeId()); // Convention for intermediate results 71 | } 72 | if(fS.getPath().toString().startsWith(thisL.getFile()) || 73 | fS.getPath().toString().startsWith("file:" + thisL.getFile())) { 74 | l = thisL; 75 | break; 76 | } 77 | } 78 | 79 | if(l == null) { 80 | throw new RuntimeException("Unable to match input split with any MSCR input"); 81 | } 82 | 83 | // If this collection is a table -> process Pair, otherwise process value 84 | PCollectionType type = l.getType(); 85 | Object toProcess = value; 86 | if(type instanceof PTableType) { 87 | toProcess = Pair.create(key, value); 88 | } 89 | 90 | for(DeferredOp op: l.getDownOps()) { 91 | if(op instanceof MultipleParallelDo) { 92 | MultipleParallelDo mPDo = ((MultipleParallelDo)op); 93 | for(Object entry: mPDo.getDests().entrySet()) { 94 | Map.Entry en = (Map.Entry)entry; 95 | LazyCollection lCol = (LazyCollection)en.getKey(); 96 | DeferredOp childOp = null; 97 | if(lCol.getDownOps() != null && lCol.getDownOps().size() > 0) { 98 | childOp = lCol.getDownOps().get(0); 99 | } 100 | final Integer channel; 101 | if(childOp != null && childOp instanceof Flatten) { 102 | channel = mscr.getNumberedChannels().get(((Flatten)childOp).getDest()); 103 | } else if(childOp != null && childOp instanceof GroupByKey) { 104 | channel = mscr.getNumberedChannels().get(((GroupByKey)childOp).getOrigin()); 105 | } else { 106 | channel = mscr.getNumberedChannels().get(en.getKey()); // bypass channel? 107 | } 108 | if(channel == null) { 109 | // This is not for this MSCR - just skip it 110 | return; 111 | } 112 | // Call parallelDo function 113 | en.getValue().process(toProcess, new EmitFn() { 114 | @Override 115 | public void emit(Object v) { 116 | try { 117 | if(v instanceof Pair) { 118 | Pair p = (Pair)v; 119 | context.write( 120 | new PlumeObject((WritableComparable)p.getKey(), channel), 121 | new PlumeObject((WritableComparable)p.getValue(), channel) 122 | ); 123 | } else { 124 | context.write( 125 | new PlumeObject((WritableComparable)v, channel), 126 | new PlumeObject((WritableComparable)v, channel) 127 | ); 128 | } 129 | } catch (Exception e) { 130 | e.printStackTrace(); // TODO How to report this 131 | } 132 | } 133 | }); 134 | } 135 | } else { 136 | if(op instanceof Flatten) { 137 | l = (LazyCollection)((Flatten)op).getDest(); 138 | } 139 | int channel = mscr.getNumberedChannels().get(l); 140 | if(toProcess instanceof Pair) { 141 | context.write(new PlumeObject(key, channel), new PlumeObject(value, channel)); 142 | } else { 143 | context.write(new PlumeObject(value, channel), new PlumeObject(value, channel)); 144 | } 145 | } 146 | } 147 | }; 148 | } -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/MSCRReducer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.io.IOException; 21 | import java.util.List; 22 | 23 | import org.apache.hadoop.io.NullWritable; 24 | import org.apache.hadoop.io.WritableComparable; 25 | import org.apache.hadoop.mapreduce.Reducer; 26 | import org.apache.hadoop.mapreduce.lib.output.MultipleOutputs; 27 | 28 | import com.google.common.collect.Lists; 29 | import com.tdunning.plume.DoFn; 30 | import com.tdunning.plume.EmitFn; 31 | import com.tdunning.plume.PCollection; 32 | import com.tdunning.plume.PTable; 33 | import com.tdunning.plume.Pair; 34 | import com.tdunning.plume.local.lazy.MSCR.OutputChannel; 35 | import com.tdunning.plume.local.lazy.MapRedExecutor.PlumeObject; 36 | import com.tdunning.plume.local.lazy.op.ParallelDo; 37 | 38 | /** 39 | * Reducer that is used to execute MSCR in MapReds 40 | */ 41 | public class MSCRReducer extends Reducer { 42 | 43 | MultipleOutputs mos; 44 | MSCR mscr; // Current MSCR being executed 45 | 46 | protected void setup(Reducer.Context context) 47 | throws IOException, InterruptedException { 48 | 49 | this.mos = new MultipleOutputs(context); 50 | this.mscr = MapRedExecutor.readMSCR(context.getConfiguration()); 51 | } 52 | 53 | protected void cleanup(Reducer.Context context) 54 | throws IOException ,InterruptedException { 55 | 56 | mos.close(); 57 | } 58 | 59 | @SuppressWarnings("unchecked") 60 | protected void reduce(final PlumeObject arg0, java.lang.Iterable values, 61 | Reducer.Context arg2) 62 | throws IOException, InterruptedException { 63 | 64 | PCollection col = mscr.getChannelByNumber().get(arg0.sourceId); 65 | OutputChannel oC = mscr.getOutputChannels().get(col); 66 | if(oC.reducer != null) { 67 | // apply reducer 68 | ParallelDo pDo = oC.reducer; 69 | DoFn reducer = pDo.getFunction(); // TODO how to check / report this 70 | List vals = Lists.newArrayList(); 71 | for(PlumeObject val: values) { 72 | vals.add(val.obj); 73 | } 74 | reducer.process(Pair.create(arg0.obj, vals), new EmitFn() { 75 | @Override 76 | public void emit(Object v) { 77 | try { 78 | if(v instanceof Pair) { 79 | Pair p = (Pair)v; 80 | mos.write(arg0.sourceId+"", p.getKey(), p.getValue()); 81 | } else { 82 | mos.write(arg0.sourceId+"", NullWritable.get(), (WritableComparable)v); 83 | } 84 | } catch (Exception e) { 85 | e.printStackTrace(); // TODO How to report this 86 | } 87 | } 88 | }); 89 | } else { 90 | // direct writing - write all key, value pairs 91 | for(PlumeObject val: values) { 92 | if(oC.output instanceof PTable) { 93 | mos.write(arg0.sourceId+"", arg0.obj, val.obj); 94 | } else { 95 | mos.write(arg0.sourceId+"", NullWritable.get(), val.obj); 96 | } 97 | } 98 | } 99 | } 100 | } -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/PlumeWorkflow.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.util.ArrayList; 21 | import java.util.List; 22 | 23 | import com.tdunning.plume.PCollection; 24 | 25 | /** 26 | * The purpose of this class is to encapsulate Plume workflows so that they can be instantiated 27 | * in Mappers and Reducers. 28 | */ 29 | public abstract class PlumeWorkflow { 30 | 31 | List inputs; 32 | List outputs; 33 | 34 | public abstract void build(); 35 | 36 | public PlumeWorkflow() { 37 | init(); 38 | } 39 | 40 | protected void init() { 41 | inputs = new ArrayList(); 42 | outputs = new ArrayList(); 43 | } 44 | 45 | protected void addOutput(PCollection collection) { 46 | outputs.add(collection); 47 | } 48 | 49 | protected void addInput(PCollection collection) { 50 | inputs.add(collection); 51 | } 52 | 53 | public List getInputs() { 54 | return inputs; 55 | } 56 | 57 | public List getOutputs() { 58 | return outputs; 59 | } 60 | } 61 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/CombineValues.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | import com.tdunning.plume.CombinerFn; 21 | import com.tdunning.plume.DoFn; 22 | import com.tdunning.plume.EmitFn; 23 | import com.tdunning.plume.PCollection; 24 | import com.tdunning.plume.Pair; 25 | 26 | public class CombineValues extends ParallelDo>, Pair> { 27 | 28 | CombinerFn combiner; 29 | 30 | public CombineValues(final CombinerFn combiner, 31 | PCollection>> origin, PCollection> dest) { 32 | super(new DoFn>, Pair>() { 33 | @Override 34 | public void process(Pair> v, EmitFn> emitter) { 35 | emitter.emit(Pair.create(v.getKey(), combiner.combine(v.getValue()))); 36 | } 37 | } , origin, dest); 38 | this.combiner = combiner; 39 | } 40 | 41 | public CombinerFn getCombiner() { 42 | return combiner; 43 | } 44 | } 45 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/DeferredOp.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | public abstract class DeferredOp { 21 | 22 | } 23 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/Flatten.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | import java.util.List; 21 | 22 | import com.tdunning.plume.PCollection; 23 | 24 | public class Flatten extends DeferredOp { 25 | 26 | List> origins; 27 | PCollection dest; 28 | 29 | public Flatten(List> origins, PCollection dest) { 30 | this.origins = origins; 31 | this.dest = dest; 32 | } 33 | 34 | public List> getOrigins() { 35 | return origins; 36 | } 37 | public PCollection getDest() { 38 | return dest; 39 | } 40 | 41 | @Override 42 | public String toString() { 43 | return "Origins " + origins + " Destination " + dest; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/GroupByKey.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | import com.tdunning.plume.Pair; 21 | import com.tdunning.plume.local.lazy.LazyTable; 22 | 23 | public class GroupByKey extends OneToOneOp, Pair>> { 24 | 25 | LazyTable origin; 26 | LazyTable> dest; 27 | 28 | public GroupByKey(LazyTable origin, LazyTable> dest) { 29 | this.origin = origin; 30 | this.dest = dest; 31 | } 32 | 33 | public LazyTable> getDest() { 34 | return dest; 35 | } 36 | 37 | public LazyTable getOrigin() { 38 | return origin; 39 | } 40 | 41 | @Override 42 | public String toString() { 43 | return "Group " + origin + " to " + dest; 44 | } 45 | } 46 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/MultipleParallelDo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | import java.util.HashMap; 21 | import java.util.Map; 22 | 23 | import com.tdunning.plume.DoFn; 24 | import com.tdunning.plume.PCollection; 25 | 26 | public class MultipleParallelDo extends DeferredOp { 27 | 28 | PCollection origin; 29 | Map, DoFn> dests; 30 | 31 | public MultipleParallelDo(PCollection origin) { 32 | this.origin = origin; 33 | } 34 | 35 | public void addDest(DoFn function, PCollection dest) { 36 | if(dests == null) { 37 | dests = new HashMap, DoFn>(); 38 | } 39 | dests.put(dest, function); 40 | } 41 | 42 | public PCollection getOrigin() { 43 | return origin; 44 | } 45 | 46 | public Map, DoFn> getDests() { 47 | return dests; 48 | } 49 | 50 | @Override 51 | public String toString() { 52 | return "MultipleParallelDo, origin " + origin + " Dests " + dests; 53 | } 54 | } 55 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/OneToOneOp.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | import com.tdunning.plume.PCollection; 21 | 22 | public abstract class OneToOneOp extends DeferredOp { 23 | 24 | public abstract PCollection getOrigin(); 25 | public abstract PCollection getDest(); 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/lazy/op/ParallelDo.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy.op; 19 | 20 | import com.tdunning.plume.DoFn; 21 | import com.tdunning.plume.PCollection; 22 | 23 | public class ParallelDo extends MultipleParallelDo { 24 | 25 | DoFn function; 26 | PCollection dest; 27 | 28 | public ParallelDo(DoFn function, PCollection origin, PCollection dest) { 29 | super(origin); 30 | this.function = function; 31 | this.dest = dest; 32 | addDest(function, dest); 33 | } 34 | 35 | public DoFn getFunction() { 36 | return function; 37 | } 38 | 39 | public PCollection getDest() { 40 | return dest; 41 | } 42 | 43 | @Override 44 | public String toString() { 45 | return "ParallelDo " + function + " to " + origin + " result " + dest; 46 | } 47 | } 48 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/mapReduce/Collector.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.mapReduce; 19 | 20 | /** 21 | * Created by IntelliJ IDEA. User: tdunning Date: Aug 2, 2010 Time: 6:04:12 PM To change this 22 | * template use File | Settings | File Templates. 23 | */ 24 | public interface Collector { 25 | public void collect(K key, V value); 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/mapReduce/MapReduce.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.mapReduce; 19 | 20 | import com.google.common.collect.Lists; 21 | import com.google.common.collect.Maps; 22 | import com.tdunning.plume.Pair; 23 | 24 | import java.util.List; 25 | import java.util.Map; 26 | 27 | /** 28 | * Created by IntelliJ IDEA. User: tdunning Date: Aug 2, 2010 Time: 6:13:40 PM To change this 29 | * template use File | Settings | File Templates. 30 | */ 31 | public class MapReduce { 32 | private Mapper mapper; 33 | private Reducer reducer; 34 | private Reducer combiner; 35 | 36 | public MapReduce(Mapper mapper, Reducer reducer, Reducer combiner) { 37 | this.mapper = mapper; 38 | this.reducer = reducer; 39 | this.combiner = combiner; 40 | } 41 | 42 | public Iterable> run(Iterable> input) { 43 | final Map> shuffle = Maps.newHashMap(); 44 | for (Pair pair : input) { 45 | mapper.map(pair.getKey(), pair.getValue(), new Collector() { 46 | @Override 47 | public void collect(RK key, RV value) { 48 | List tmp = shuffle.get(key); 49 | if (tmp == null) { 50 | tmp = Lists.newArrayList(); 51 | shuffle.put(key, tmp); 52 | } 53 | tmp.add(value); 54 | if (combiner != null && tmp.size() > 10) { 55 | final List internalBuffer = tmp; 56 | combiner.reduce(key, tmp, new Collector() { 57 | @Override 58 | public void collect(RK key, RV value) { 59 | internalBuffer.clear(); 60 | internalBuffer.add(value); 61 | } 62 | }); 63 | } 64 | } 65 | }); 66 | } 67 | 68 | final List> output = Lists.newArrayList(); 69 | for (RK key : shuffle.keySet()) { 70 | if (reducer == null) { 71 | for (RV x : shuffle.get(key)) { 72 | output.add(Pair.create(key, (OV) x)); 73 | } 74 | } else { 75 | reducer.reduce(key, shuffle.get(key), new Collector() { 76 | @Override 77 | public void collect(RK key, OV value) { 78 | output.add(Pair.create(key, value)); 79 | } 80 | }); 81 | } 82 | } 83 | return output; 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/mapReduce/MapReduceBuilder.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.mapReduce; 19 | 20 | /** 21 | * Created by IntelliJ IDEA. User: tdunning Date: Aug 2, 2010 Time: 6:08:32 PM To change this 22 | * template use File | Settings | File Templates. 23 | */ 24 | public class MapReduceBuilder { 25 | private Mapper mapper; 26 | private Reducer reducer; 27 | private Reducer combiner; 28 | 29 | public MapReduceBuilder() { 30 | } 31 | 32 | public MapReduceBuilder map(Mapper mapper) { 33 | this.mapper = mapper; 34 | return this; 35 | } 36 | 37 | public MapReduceBuilder reduce(Reducer reducer) { 38 | this.reducer = reducer; 39 | return this; 40 | } 41 | 42 | public MapReduceBuilder combine(Reducer combiner) { 43 | this.combiner = combiner; 44 | return this; 45 | } 46 | 47 | public MapReduce build() { 48 | return new MapReduce(mapper, reducer, combiner); 49 | } 50 | } 51 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/mapReduce/Mapper.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.mapReduce; 19 | 20 | /** 21 | * Created by IntelliJ IDEA. User: tdunning Date: Aug 2, 2010 Time: 6:02:19 PM To change this 22 | * template use File | Settings | File Templates. 23 | */ 24 | public interface Mapper { 25 | public void map(K key, V value, Collector out); 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/mapReduce/Reducer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.mapReduce; 19 | 20 | /** 21 | * Created by IntelliJ IDEA. User: tdunning Date: Aug 2, 2010 Time: 6:04:58 PM To change this 22 | * template use File | Settings | File Templates. 23 | */ 24 | public interface Reducer { 25 | public void reduce(K key, Iterable values, Collector collector); 26 | } 27 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/local/mapReduce/package.html: -------------------------------------------------------------------------------- 1 | 17 | 18 | 19 | Provides a very simple toy-sized map reduce implementation based on Java collections. 20 |

21 | The purpose is to provide a framework for building and debugging the lazy evaluation 22 | based plan optimizer. 23 |

24 | 25 | 26 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/package.html: -------------------------------------------------------------------------------- 1 | 17 | 18 | 19 | 20 |

Plume

21 | Plume is an independent implementation of a system similar to FlumeJava. It allows map-reduce 22 | programs to be written easily with little regard for whether they will ultimately be executed 23 | in a parallel or sequential fashion. 24 |

25 | In order to write a Plume program, you need a Plume object to work with. This Plume defines 26 | the execution model of your program and there will ultimately be varieties of Plumes that 27 | execute code in local or parallel fashion, with or without Hadoop, and in eager 28 | and lazy fashion. So far, there is part of a local, eager Plume. 29 | 30 | The FlumeJava paper provides an excellent introduction to the concepts. 31 | With that paper in mind this code should be fairly simple. The highest level structure is summarized by 32 | the following classes: 33 |

    34 |
  • @see plume.local.eager.LocalPlume
  • 35 |
36 | 37 | 38 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/BooleanType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class BooleanType extends PType { 22 | public BooleanType() { super(Kind.BOOLEAN); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/BytesType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class BytesType extends PType { 22 | public BytesType() { super(Kind.BYTES); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/DoubleType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class DoubleType extends PType { 22 | public DoubleType() { super(Kind.DOUBLE); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/FloatType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class FloatType extends PType { 22 | public FloatType() { super(Kind.FLOAT); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/IntegerType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class IntegerType extends PType { 22 | public IntegerType() { super(Kind.INTEGER); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/LongType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class LongType extends PType { 22 | public LongType() { super(Kind.LONG); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/PCollectionType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | import com.tdunning.plume.PCollection; 21 | 22 | /** PCollection type. */ 23 | public class PCollectionType extends PType> { 24 | private PType elementType; 25 | 26 | public PCollectionType(PType elementType) { 27 | super(Kind.COLLECTION); 28 | this.elementType = elementType; 29 | } 30 | 31 | public PType elementType() { return elementType; } 32 | } 33 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/PTableType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | import com.tdunning.plume.PTable; 21 | import com.tdunning.plume.PCollection; 22 | import com.tdunning.plume.Pair; 23 | 24 | /** 25 | * Class used as a hint to map functions that they should return a PTable instead of a PCollection. 26 | */ 27 | public class PTableType extends PCollectionType> { 28 | public PTableType(PType keyType, PType valueType) { 29 | super(new PairType(keyType, valueType)); 30 | } 31 | 32 | public PType keyType() { 33 | return ((PairType)elementType()).keyType(); 34 | } 35 | 36 | public PType valueType() { 37 | return ((PairType)elementType()).valueType(); 38 | } 39 | } 40 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/PType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** Base of all types. */ 21 | public abstract class PType { 22 | public enum Kind { BOOLEAN, BYTES, DOUBLE, FLOAT, INTEGER, LONG, 23 | PAIR, COLLECTION, TABLE, RECORD, STRING } 24 | 25 | private Kind kind; 26 | 27 | protected PType(Kind kind) { this.kind = kind; } 28 | 29 | public Kind kind() { return kind; } 30 | 31 | } 32 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/PairType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | import com.tdunning.plume.Pair; 21 | 22 | /** Pair type. */ 23 | public class PairType extends PType> { 24 | private PType keyType; 25 | private PType valueType; 26 | 27 | public PairType(PType keyType, PType valueType) { 28 | super(Kind.PAIR); 29 | 30 | this.keyType = keyType; 31 | this.valueType = valueType; 32 | } 33 | 34 | public PType keyType() { return keyType; } 35 | public PType valueType() { return valueType; } 36 | } 37 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/RecordType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | import org.apache.avro.Schema; 21 | import org.apache.avro.specific.SpecificData; 22 | 23 | /** Record type. For Avro specific classes. */ 24 | public class RecordType extends PType { 25 | private Schema schema; 26 | 27 | public RecordType(Class c) { 28 | super(Kind.RECORD); 29 | this.schema = SpecificData.get().getSchema(c); 30 | } 31 | 32 | public Schema schema() { return schema; } 33 | } 34 | -------------------------------------------------------------------------------- /src/main/java/com/tdunning/plume/types/StringType.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.types; 19 | 20 | /** String type. */ 21 | public class StringType extends PType { 22 | public StringType() { super(Kind.STRING); } 23 | } 24 | -------------------------------------------------------------------------------- /src/main/java/org/apache/hadoop/mapreduce/lib/input/FileInputSplitWrapper.java: -------------------------------------------------------------------------------- 1 | package org.apache.hadoop.mapreduce.lib.input; 2 | 3 | import org.apache.hadoop.mapreduce.Mapper.Context; 4 | 5 | /** 6 | * This class has been created to allow {@link MSCRMapper} to access a FileSplit when it is wrapped by a 7 | * TaggedInputSplit. Because the latter is a private class, the only way to access it is by implementing this 8 | * business logic in the org.apache.hadoop.mapreduce.lib.input class space. 9 | */ 10 | public class FileInputSplitWrapper { 11 | 12 | public static FileSplit getFileInputSplit(Context context) { 13 | // This class is private! 14 | TaggedInputSplit t = (TaggedInputSplit)context.getInputSplit(); 15 | FileSplit fS = (FileSplit)t.getInputSplit(); 16 | return fS; 17 | } 18 | } 19 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/FlattenTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume; 19 | 20 | import com.google.common.collect.Lists; 21 | import com.google.common.collect.Maps; 22 | import com.google.common.collect.Sets; 23 | import com.tdunning.plume.local.eager.LocalPlume; 24 | import org.junit.Assert; 25 | import org.junit.Test; 26 | 27 | import java.util.List; 28 | import java.util.Map; 29 | import java.util.Set; 30 | 31 | import static com.tdunning.plume.Plume.*; 32 | 33 | public class FlattenTest { 34 | @Test 35 | public void flatten() { 36 | Set x1 = Sets.newHashSet(); 37 | for (int i = 0; i < 10; i++) { 38 | x1.add(i); 39 | } 40 | 41 | List x2 = Lists.newArrayList(); 42 | for (int i = 5; i < 15; i++) { 43 | x2.add(i); 44 | } 45 | 46 | Plume p = new LocalPlume(); 47 | PCollection x3 = p.flatten(p.fromJava(x1), p.fromJava(x2)); 48 | PTable x4 = x3.count(); 49 | 50 | Map r = Maps.newHashMap(); 51 | for (Pair pair : x4) { 52 | r.put(pair.getKey(), pair.getValue()); 53 | } 54 | for (int i = 0; i < 5; i++) { 55 | Assert.assertEquals(new Integer(1), r.get(i)); 56 | } 57 | for (int i = 5; i < 10; i++) { 58 | Assert.assertEquals(new Integer(2), r.get(i)); 59 | } 60 | for (int i = 10; i < 15; i++) { 61 | Assert.assertEquals(new Integer(1), r.get(i)); 62 | } 63 | 64 | PTable x5 = x4.map(new DoFn, Pair>() { 65 | @Override 66 | public void process(Pair v, EmitFn> emitter) { 67 | emitter.emit(new Pair(v.getKey(), v.getValue().toString())); 68 | } 69 | }, tableOf(integers(), strings())); 70 | for (Pair pair : x5) { 71 | Assert.assertEquals(r.get(pair.getKey()).toString(), pair.getValue()); 72 | } 73 | 74 | PCollection x6 = x4.map(new DoFn, String>() { 75 | @Override 76 | public void process(Pair v, EmitFn emitter) { 77 | emitter.emit(v.getValue().toString()); 78 | } 79 | }, collectionOf(strings())); 80 | Map r2 = Maps.newHashMap(); 81 | for (Pair v : x6.count()) { 82 | r2.put(v.getKey(), v.getValue()); 83 | } 84 | } 85 | } 86 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/WordCountTest.java: -------------------------------------------------------------------------------- 1 | package com.tdunning.plume; 2 | 3 | import static com.tdunning.plume.Plume.collectionOf; 4 | import static com.tdunning.plume.Plume.integers; 5 | import static com.tdunning.plume.Plume.strings; 6 | import static com.tdunning.plume.Plume.tableOf; 7 | import static org.junit.Assert.assertEquals; 8 | 9 | import java.io.IOException; 10 | import java.util.Map; 11 | 12 | import org.junit.Test; 13 | 14 | import com.google.common.base.CharMatcher; 15 | import com.google.common.base.Splitter; 16 | import com.google.common.collect.Maps; 17 | import com.google.common.io.Resources; 18 | import com.tdunning.plume.local.eager.LocalPlume; 19 | import com.tdunning.plume.local.lazy.LazyPlume; 20 | 21 | public class WordCountTest { 22 | @Test 23 | public void wordCount() throws IOException { 24 | Plume p = new LocalPlume(); 25 | countWords(p.readResourceFile("simple-text.txt")); 26 | } 27 | 28 | @Test 29 | public void lazyWordCount() throws IOException { 30 | /* 31 | * I comment out this test temporarily because to lazily execute a wordcount there is a bit more to do - 32 | * (an Executor has to be called before reading the result) 33 | */ 34 | // Plume p = new LazyPlume(); 35 | // countWords(p.readResourceFile("simple-text.txt")); 36 | } 37 | 38 | @Test 39 | public void lazyAvroWordCount() throws IOException { 40 | Plume p = new LazyPlume(); 41 | String file = Resources.getResource("simple-text.avro").getPath(); 42 | countWords(p.readAvroFile(file, strings())); 43 | } 44 | 45 | @Test 46 | public void wordCountAvro() throws IOException { 47 | Plume p = new LocalPlume(); 48 | String file = Resources.getResource("simple-text.avro").getPath(); 49 | countWords(p.readAvroFile(file, strings())); 50 | } 51 | 52 | private void countWords(PCollection lines) { 53 | final Splitter onNonWordChar = Splitter.on(CharMatcher.BREAKING_WHITESPACE); 54 | 55 | PCollection words = lines.map(new DoFn() { 56 | @Override 57 | public void process(T x, EmitFn emitter) { 58 | for (String word : onNonWordChar.split(x.toString())) { 59 | emitter.emit(word); 60 | } 61 | } 62 | }, collectionOf(strings())); 63 | 64 | PTable wc = words 65 | .map(new DoFn>() { 66 | @Override 67 | public void process(String x, EmitFn> emitter) { 68 | emitter.emit(Pair.create(x, 1)); 69 | } 70 | }, tableOf(strings(), integers())) 71 | .groupByKey() 72 | .combine(new CombinerFn() { 73 | @Override 74 | public Integer combine(Iterable counts) { 75 | int sum = 0; 76 | for (Integer k : counts) { 77 | sum += k; 78 | } 79 | return sum; 80 | } 81 | }); 82 | 83 | Map m = Maps.newHashMap(); 84 | for (Pair pair : wc) { 85 | m.put(pair.getKey(), pair.getValue()); 86 | } 87 | assertEquals(3, m.get("is").intValue()); 88 | assertEquals(3, m.get("some").intValue()); 89 | assertEquals(3, m.get("simple").intValue()); 90 | assertEquals(1, m.get("examples").intValue()); 91 | assertEquals(2, m.get("text").intValue()); 92 | } 93 | } 94 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/avro/WriteAvroFile.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.avro; 19 | 20 | import com.google.common.collect.Lists; 21 | import org.apache.avro.Schema; 22 | import org.apache.avro.file.DataFileReader; 23 | import org.apache.avro.file.DataFileStream; 24 | import org.apache.avro.file.DataFileWriter; 25 | import org.apache.avro.specific.SpecificDatumReader; 26 | import org.apache.avro.specific.SpecificDatumWriter; 27 | import org.junit.Assert; 28 | import org.junit.Test; 29 | 30 | import java.io.BufferedInputStream; 31 | import java.io.File; 32 | import java.io.FileInputStream; 33 | import java.io.IOException; 34 | 35 | import static org.junit.Assert.assertEquals; 36 | 37 | /** 38 | * Created by IntelliJ IDEA. User: tdunning Date: Aug 4, 2010 Time: 6:32:18 PM To change this 39 | * template use File | Settings | File Templates. 40 | */ 41 | public class WriteAvroFile { 42 | @Test 43 | public void writeSomething() throws IOException { 44 | Schema s = Schema.create(Schema.Type.DOUBLE); 45 | DataFileWriter x = new DataFileWriter(new SpecificDatumWriter(s)); 46 | File f = new File("x"); 47 | DataFileWriter z = x.create(s, f); 48 | f.deleteOnExit(); 49 | for (int i = 0; i < 10; i++) { 50 | z.append(3.0*i); 51 | } 52 | z.close(); 53 | 54 | DataFileReader in = new DataFileReader(new File("x"), 55 | new SpecificDatumReader(s)); 56 | int k = 0; 57 | while (in.hasNext()) { 58 | assertEquals(3.0 * k++, in.next(), 0); 59 | } 60 | in.close(); 61 | 62 | final DataFileStream data = 63 | new DataFileStream 64 | (new BufferedInputStream(new FileInputStream("x")), 65 | new SpecificDatumReader(s)); 66 | k = 0; 67 | while (data.hasNext()) { 68 | assertEquals(3.0 * k++, data.next(), 0); 69 | } 70 | data.close(); 71 | } 72 | } 73 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/LogParseTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local; 19 | 20 | import com.google.common.base.CharMatcher; 21 | import com.google.common.base.Splitter; 22 | import com.tdunning.plume.DoFn; 23 | import com.tdunning.plume.EmitFn; 24 | import com.tdunning.plume.Ordering; 25 | import com.tdunning.plume.PCollection; 26 | import com.tdunning.plume.PTable; 27 | import com.tdunning.plume.Pair; 28 | import com.tdunning.plume.Plume; 29 | import com.tdunning.plume.local.eager.LocalPlume; 30 | import org.junit.Test; 31 | 32 | import java.io.IOException; 33 | import java.util.Iterator; 34 | 35 | import static com.tdunning.plume.Plume.strings; 36 | 37 | public class LogParseTest { 38 | @Test 39 | public void parseGroupSort() throws IOException { 40 | Plume p = new LocalPlume(); 41 | PCollection logs = p.readResourceFile("log.txt"); 42 | PTable events = logs.map(new DoFn>() { 43 | @Override 44 | public void process(String logLine, EmitFn> emitter) { 45 | Event e = new Event(logLine); 46 | emitter.emit(new Pair(e.getName(), e)); 47 | } 48 | }, Plume.tableOf(strings(), strings())); 49 | 50 | PTable> byName = events.groupByKey(new Ordering() { 51 | // what goes here?? 52 | }); 53 | } 54 | 55 | private static final class Event implements Comparable { 56 | private static final Splitter onWhiteSpace = Splitter.on(CharMatcher.BREAKING_WHITESPACE); 57 | private String time; 58 | private String name; 59 | private String msg; 60 | 61 | public Event(String logLine) { 62 | Iterator pieces = onWhiteSpace.split(logLine).iterator(); 63 | time = pieces.next(); 64 | name = pieces.next(); 65 | msg = pieces.next(); 66 | } 67 | 68 | public String getName() { 69 | return name; 70 | } 71 | 72 | @Override 73 | public int compareTo(Event o) { 74 | return this.time.compareTo(o.time); 75 | } 76 | } 77 | } 78 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/BaseTestClass.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import java.util.Collections; 21 | import java.util.List; 22 | 23 | import org.apache.hadoop.io.IntWritable; 24 | import org.apache.hadoop.io.Text; 25 | import org.junit.Before; 26 | 27 | import static org.junit.Assert.assertEquals; 28 | 29 | import com.google.common.collect.Lists; 30 | import com.tdunning.plume.CombinerFn; 31 | import com.tdunning.plume.DoFn; 32 | import com.tdunning.plume.EmitFn; 33 | import com.tdunning.plume.Pair; 34 | import com.tdunning.plume.types.IntegerType; 35 | import com.tdunning.plume.types.PTableType; 36 | 37 | /** 38 | * Contains some utility methods and variables for local.lazy testing 39 | */ 40 | public class BaseTestClass { 41 | 42 | final static DoFn plusOne = new DoFn() { 43 | @Override 44 | public void process(Integer v, EmitFn emitter) { 45 | emitter.emit(v + 1); 46 | } 47 | }; 48 | 49 | final static DoFn timesTwo = new DoFn() { 50 | @Override 51 | public void process(Integer v, EmitFn emitter) { 52 | emitter.emit(v * 2); 53 | } 54 | }; 55 | 56 | final static DoFn> plusTwoPlusThree = new DoFn>() { 57 | @Override 58 | public void process(Integer v, EmitFn> emitter) { 59 | emitter.emit(Pair.create(v, v * 2)); 60 | emitter.emit(Pair.create(v, v * 3)); 61 | } 62 | }; 63 | 64 | final static DoFn identity = new DoFn() { 65 | @Override 66 | public void process(Object v, EmitFn emitter) { 67 | emitter.emit(v); 68 | } 69 | }; 70 | 71 | final static CombinerFn dummyCombiner = new CombinerFn() { 72 | @Override 73 | public Integer combine(Iterable stuff) { 74 | return 1; 75 | } 76 | }; 77 | 78 | final static CombinerFn countCombiner = new CombinerFn() { 79 | @Override 80 | public IntWritable combine(Iterable stuff) { 81 | int c = 0; 82 | for(IntWritable i : stuff) { 83 | c += i.get(); 84 | } 85 | return new IntWritable(c); 86 | } 87 | }; 88 | 89 | final static DoFn countReduceToText = new DoFn() { 90 | @Override 91 | public void process(Object v, EmitFn emitter) { 92 | Pair p = (Pair)v; 93 | emitter.emit(Pair.create(p.getKey(), 94 | new Text(""+countCombiner.combine((Iterable)p.getValue())))); 95 | } 96 | }; 97 | 98 | static void executeAndAssert(LazyCollection output, Integer[] expectedResult) { 99 | // Get a local executor 100 | LocalExecutor executor = new LocalExecutor(); 101 | // Execute current plan 102 | Iterable result = executor.execute(output); 103 | List l = Lists.newArrayList(result); 104 | Collections.sort(l); 105 | for(int i = 0; i < expectedResult.length; i++) { 106 | assertEquals(l.get(i).intValue(), expectedResult[i].intValue()); 107 | } 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/BasicOptimizerTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | import static org.junit.Assert.assertFalse; 22 | import static org.junit.Assert.assertTrue; 23 | 24 | import java.util.Map; 25 | 26 | import org.junit.Test; 27 | 28 | import com.google.common.collect.Lists; 29 | import com.tdunning.plume.DoFn; 30 | import com.tdunning.plume.EmitFn; 31 | import com.tdunning.plume.PCollection; 32 | import com.tdunning.plume.local.lazy.op.DeferredOp; 33 | import com.tdunning.plume.local.lazy.op.Flatten; 34 | import com.tdunning.plume.local.lazy.op.MultipleParallelDo; 35 | import com.tdunning.plume.local.lazy.op.ParallelDo; 36 | 37 | /** 38 | * These basic tests can be used to assert that the {@link Optimizer} behaves correctly for all basic operations 39 | */ 40 | public class BasicOptimizerTest extends BaseTestClass { 41 | 42 | @SuppressWarnings("unchecked") 43 | @Test 44 | public void testParallelDoSiblingFusion() { 45 | // Get Plume runtime 46 | LazyPlume plume = new LazyPlume(); 47 | // Create simple data 48 | PCollection input = plume.fromJava(Lists.newArrayList(1, 2, 3)); 49 | PCollection output1 = input.map(plusOne, null); 50 | PCollection output2 = input.map(timesTwo, null); 51 | LazyCollection lInput = (LazyCollection)input; 52 | LazyCollection lOutput1 = (LazyCollection)output1; 53 | LazyCollection lOutput2 = (LazyCollection)output2; 54 | assertEquals(lInput.downOps.size(), 2); 55 | // Execute and assert the result before optimizing 56 | executeAndAssert(lOutput1, new Integer[] { 2, 3, 4 }); 57 | executeAndAssert(lOutput2, new Integer[] { 2, 4, 6 }); 58 | // Get an Optimizer 59 | Optimizer optimizer = new Optimizer(); 60 | optimizer.fuseSiblingParallelDos(output1); // one output is enough to fuse both because they share the parent 61 | // Check that input child ops has shrinked to 1 62 | assertEquals(lInput.downOps.size(), 1); 63 | DeferredOp op = lInput.downOps.get(0); 64 | // Check that there is only one op pointing to both outputs 65 | assertEquals(op, lOutput1.deferredOp); 66 | assertEquals(op, lOutput2.deferredOp); 67 | assertTrue(op instanceof MultipleParallelDo); 68 | MultipleParallelDo mPDo = (MultipleParallelDo)op; 69 | Map, DoFn> mapOfPDos = mPDo.getDests(); 70 | // Check that the map of functions in MultipleParallelDo is correct 71 | assertEquals(mapOfPDos.get(output1), plusOne); 72 | assertEquals(mapOfPDos.get(output2), timesTwo); 73 | // Execute and assert the result afer optimizing 74 | executeAndAssert(lOutput1, new Integer[] { 2, 3, 4 }); 75 | executeAndAssert(lOutput2, new Integer[] { 2, 4, 6 }); 76 | } 77 | 78 | /** 79 | * In this test we will apply one fuseParallelDo (x + 1) o (x * 2) => (x + 1)*2 80 | */ 81 | @Test 82 | @SuppressWarnings("unchecked") 83 | public void testParallelDoFusion() { 84 | // Get Plume runtime 85 | LazyPlume plume = new LazyPlume(); 86 | // Create simple data 87 | PCollection input = plume.fromJava(Lists.newArrayList(1, 2, 3)); 88 | PCollection output = input.map(plusOne, null).map(timesTwo, null); 89 | // Execute and assert the result before optimizing 90 | executeAndAssert((LazyCollection)output, new Integer[] { 4, 6, 8 }); 91 | LazyCollection lOutput = (LazyCollection)output; 92 | ParallelDo oldPDo = (ParallelDo)lOutput.getDeferredOp(); 93 | LazyCollection intermediateCol = (LazyCollection)oldPDo.getOrigin(); 94 | assertEquals(intermediateCol.getDownOps().size(), 1); 95 | assertEquals(intermediateCol.getDownOps().get(0), oldPDo); 96 | // Get an Optimizer 97 | Optimizer optimizer = new Optimizer(); 98 | optimizer.fuseParallelDos(output); 99 | // Check that optimizer did what it's supposed to do 100 | ParallelDo newPDo = (ParallelDo)lOutput.getDeferredOp(); 101 | assertFalse(newPDo == oldPDo); 102 | assertEquals(intermediateCol.getDownOps().size(), 0); 103 | // Check that composed function does (x+1)*2 104 | newPDo.getFunction().process(5, new EmitFn() { 105 | @Override 106 | public void emit(Integer v) { 107 | assertEquals(v.intValue(), 12); // (5+1)*2 108 | } 109 | }); 110 | // Check that now output's parent is input 111 | assertEquals(newPDo.getOrigin(), input); 112 | // Execute and assert the result after optimizing 113 | executeAndAssert(lOutput, new Integer[] { 4, 6, 8 }); 114 | } 115 | 116 | /** 117 | * This test has two inputs, one flatten and then one ParallelDo. 118 | * After sinking flattens, the tree should be as: two inputs, one ParallelDo after each input and one final Flatten. 119 | */ 120 | @Test 121 | public void testSinkFlattens() { 122 | // Get Plume runtime 123 | LazyPlume plume = new LazyPlume(); 124 | // Create simple data 125 | PCollection input1 = plume.fromJava(Lists.newArrayList(1, 2, 3)); 126 | PCollection input2 = plume.fromJava(Lists.newArrayList(4, 5, 6)); 127 | PCollection output = plume.flatten(input1, input2).map(plusOne, null); 128 | LazyCollection lOutput = (LazyCollection)output; 129 | assertTrue(lOutput.getDeferredOp() instanceof ParallelDo); 130 | // Execute and assert the result before optimizing 131 | executeAndAssert((LazyCollection)output, new Integer[] { 2, 3, 4, 5, 6, 7 }); 132 | // Get an Optimizer 133 | Optimizer optimizer = new Optimizer(); 134 | optimizer.sinkFlattens(output); 135 | // Execute and assert the result after optimizing 136 | executeAndAssert((LazyCollection)output, new Integer[] { 2, 3, 4, 5, 6, 7 }); 137 | // Check that optimizer did what it's supposed to do 138 | assertTrue(lOutput.getDeferredOp() instanceof Flatten); 139 | Flatten flatten = (Flatten)lOutput.getDeferredOp(); 140 | assertEquals(flatten.getOrigins().size(), 2); 141 | for(int i = 0; i < 2; i++) { 142 | LazyCollection origin = (LazyCollection) flatten.getOrigins().get(i); 143 | ParallelDo newPDo = (ParallelDo)origin.getDeferredOp(); 144 | assertEquals(newPDo.getFunction(), plusOne); 145 | assertTrue(newPDo.getOrigin() == input1 || newPDo.getOrigin() == input2); 146 | } 147 | } 148 | } 149 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/LocalExecutorTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.integers; 21 | import static com.tdunning.plume.Plume.tableOf; 22 | 23 | import static org.junit.Assert.assertEquals; 24 | 25 | import java.util.Collections; 26 | import java.util.Comparator; 27 | import java.util.List; 28 | 29 | import org.junit.Test; 30 | 31 | import com.google.common.collect.Lists; 32 | import com.tdunning.plume.CombinerFn; 33 | import com.tdunning.plume.DoFn; 34 | import com.tdunning.plume.EmitFn; 35 | import com.tdunning.plume.PCollection; 36 | import com.tdunning.plume.PTable; 37 | import com.tdunning.plume.Pair; 38 | 39 | /** 40 | * These basic tests assert that the {@link LocalExecutor} behaves correctly for all basic operations 41 | */ 42 | public class LocalExecutorTest extends BaseTestClass { 43 | 44 | /** 45 | * This test runs a chain of two "ParallelDo" operations: (x+1), (x*2) 46 | */ 47 | @Test 48 | public void testNestedMap() { 49 | // Get Plume runtime 50 | LazyPlume plume = new LazyPlume(); 51 | // Create simple data 52 | PCollection input = plume.fromJava(Lists.newArrayList(1, 2, 3)); 53 | PCollection output = input.map(plusOne, null).map(timesTwo, null); 54 | executeAndAssert((LazyCollection)output, new Integer[] { 4, 6, 8 }); 55 | } 56 | 57 | /** 58 | * Deferred execution of ((1,2,3)+(4,5,6)) => x+1 59 | */ 60 | @Test 61 | public void testMapAndFlatten() { 62 | List l1 = Lists.newArrayList(1, 2, 3); 63 | List l2 = Lists.newArrayList(4, 5, 6); 64 | // Get Plume runtime 65 | LazyPlume plume = new LazyPlume(); 66 | PCollection output = plume.flatten(plume.fromJava(l1), plume.fromJava(l2)).map(plusOne, null); 67 | executeAndAssert((LazyCollection)output, new Integer[] { 2, 3, 4, 5, 6, 7 }); 68 | } 69 | 70 | /** 71 | * Try one deferred flatten (1,2,3)+(4,5,6) 72 | */ 73 | @Test 74 | public void testSimpleFlatten() { 75 | // Get Plume runtime 76 | LazyPlume plume = new LazyPlume(); 77 | List l1 = Lists.newArrayList(1, 2, 3); 78 | List l2 = Lists.newArrayList(4, 5, 6); 79 | PCollection output = plume.flatten(plume.fromJava(l1), plume.fromJava(l2)); 80 | executeAndAssert((LazyCollection)output, new Integer[] { 1, 2, 3, 4, 5, 6 }); 81 | } 82 | 83 | /** 84 | * Try one nested deferred flatten (7,8,9)+((1,2,3)+(4,5,6)) 85 | */ 86 | @Test 87 | public void testNestedFlatten() { 88 | // Get Plume runtime 89 | LazyPlume plume = new LazyPlume(); 90 | List l1 = Lists.newArrayList(1, 2, 3); 91 | List l2 = Lists.newArrayList(4, 5, 6); 92 | List l3 = Lists.newArrayList(7, 8, 9); 93 | PCollection output = 94 | plume.flatten(plume.fromJava(l3), plume.flatten(plume.fromJava(l1), plume.fromJava(l2))); 95 | executeAndAssert((LazyCollection)output, new Integer[] { 1, 2, 3, 4, 5, 6, 7, 8, 9 }); 96 | } 97 | 98 | /** 99 | * Try one group by from table (1,2),(1,3),(2,4),(2,6),(3,6),(3,9) 100 | */ 101 | @Test 102 | public void testGroupByKey() { 103 | // Get Plume runtime 104 | LazyPlume plume = new LazyPlume(); 105 | List l1 = Lists.newArrayList(1, 2, 3); 106 | PTable> output = plume 107 | .fromJava(l1).map(plusTwoPlusThree, tableOf(integers(), integers())) 108 | .groupByKey(); 109 | LazyTable> lOutput = (LazyTable>)output; 110 | // Get an executor 111 | LocalExecutor executor = new LocalExecutor(); 112 | Iterable>> executedOutput = executor.execute(lOutput); 113 | List>> outputList = Lists.newArrayList(executedOutput); 114 | Collections.sort(outputList, new Comparator>() { 115 | @Override 116 | public int compare(Pair arg0, Pair arg1) { 117 | return arg0.getKey().compareTo(arg1.getKey()); 118 | } 119 | }); 120 | assertEquals(outputList.get(0).getKey().intValue(), 1); 121 | assertEquals(outputList.get(1).getKey().intValue(), 2); 122 | assertEquals(outputList.get(2).getKey().intValue(), 3); 123 | List lR1 = Lists.newArrayList(outputList.get(0).getValue()); 124 | List lR2 = Lists.newArrayList(outputList.get(1).getValue()); 125 | List lR3 = Lists.newArrayList(outputList.get(2).getValue()); 126 | Collections.sort(lR1); 127 | Collections.sort(lR2); 128 | Collections.sort(lR3); 129 | assertEquals(lR1.get(0).intValue(), 2); 130 | assertEquals(lR1.get(1).intValue(), 3); 131 | assertEquals(lR2.get(0).intValue(), 4); 132 | assertEquals(lR2.get(1).intValue(), 6); 133 | assertEquals(lR3.get(0).intValue(), 6); 134 | assertEquals(lR3.get(1).intValue(), 9); 135 | } 136 | 137 | /** 138 | * Group by and combine adding all values from table (1,2),(1,3),(2,4),(2,6),(3,6),(3,9) 139 | * Should raise result (1,(2+3)),(2,(4+6)),(3,(6+9)) = (1,5),(2,10),(3,15) 140 | */ 141 | @Test 142 | public void testCombine() { 143 | DoFn> fn = new DoFn>() { 144 | @Override 145 | public void process(Integer v, EmitFn> emitter) { 146 | emitter.emit(Pair.create(v, v * 2)); 147 | emitter.emit(Pair.create(v, v * 3)); 148 | } 149 | }; 150 | // Get Plume runtime 151 | LazyPlume plume = new LazyPlume(); 152 | List l1 = Lists.newArrayList(1, 2, 3); 153 | PTable output = plume 154 | .fromJava(l1).map(fn, tableOf(integers(), integers())) 155 | .groupByKey() 156 | .combine(new CombinerFn() { 157 | @Override 158 | public Integer combine(Iterable stuff) { 159 | Integer result = 0; 160 | for(Integer i: stuff) { 161 | result += i; 162 | } 163 | return result; 164 | } 165 | }); 166 | // Get an executor 167 | LocalExecutor executor = new LocalExecutor(); 168 | LazyTable lOutput = (LazyTable)output; 169 | Iterable> executedOutput = executor.execute(lOutput); 170 | List> outputList = Lists.newArrayList(executedOutput); 171 | Collections.sort(outputList, new Comparator>() { 172 | @Override 173 | public int compare(Pair arg0, Pair arg1) { 174 | return arg0.getKey().compareTo(arg1.getKey()); 175 | } 176 | }); 177 | assertEquals(outputList.get(0).getKey().intValue(), 1); 178 | assertEquals(outputList.get(0).getValue().intValue(), 5); 179 | assertEquals(outputList.get(1).getKey().intValue(), 2); 180 | assertEquals(outputList.get(1).getValue().intValue(), 10); 181 | assertEquals(outputList.get(2).getKey().intValue(), 3); 182 | assertEquals(outputList.get(2).getValue().intValue(), 15); 183 | } 184 | } 185 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedBypassTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.collectionOf; 21 | import static com.tdunning.plume.Plume.strings; 22 | import static com.tdunning.plume.Plume.tableOf; 23 | import static org.junit.Assert.assertEquals; 24 | 25 | import java.io.File; 26 | import java.io.IOException; 27 | import java.util.List; 28 | import java.util.Map; 29 | 30 | import org.apache.hadoop.conf.Configuration; 31 | import org.apache.hadoop.fs.FileSystem; 32 | import org.apache.hadoop.fs.Path; 33 | import org.apache.hadoop.io.Text; 34 | import org.junit.Test; 35 | 36 | import com.google.common.base.Charsets; 37 | import com.google.common.collect.Maps; 38 | import com.google.common.io.Files; 39 | import com.google.common.io.Resources; 40 | import com.tdunning.plume.DoFn; 41 | import com.tdunning.plume.EmitFn; 42 | import com.tdunning.plume.PCollection; 43 | import com.tdunning.plume.Pair; 44 | 45 | /** 46 | * This test asserts that bypass channels as described in FlumeJava paper work for Plume MSCRs. 47 | */ 48 | public class MapRedBypassTest { 49 | 50 | /** 51 | * In this example we open a file and apply two functions to it. One of them performs a group by key and the other one 52 | * just adds as output the result of the second function (bypass channel). 53 | */ 54 | public static class MapRedBypassWorkflow extends PlumeWorkflow { 55 | 56 | public MapRedBypassWorkflow() { 57 | } 58 | 59 | @SuppressWarnings("unchecked") 60 | @Override 61 | public void build() { 62 | init(); 63 | 64 | LazyPlume plume = new LazyPlume(); 65 | PCollection input; 66 | try { 67 | // Read input 68 | input = plume.readFile("/tmp/input-wordcount.txt", collectionOf(strings())); 69 | // Add it as workflow's input 70 | addInput(input); 71 | } catch (IOException e) { 72 | throw new RuntimeException(); 73 | } 74 | 75 | PCollection bypassTransform = input.map(new DoFn() { 76 | @Override 77 | public void process(Object v, EmitFn emitter) { 78 | Text t = (Text)v; 79 | emitter.emit(Pair.create(new Text(t + "-blah"), new Text(t + "-bloh"))); 80 | }}, tableOf(strings(), strings())); 81 | 82 | addOutput(bypassTransform); 83 | 84 | PCollection groupedTransform = input.map(new DoFn() { 85 | @Override 86 | public void process(Object v, EmitFn emitter) { 87 | Text t = (Text)v; 88 | emitter.emit(Pair.create(t, new Text("foo"))); 89 | }}, tableOf(strings(), strings())).groupByKey(); 90 | 91 | addOutput(groupedTransform); 92 | } 93 | } 94 | 95 | @Test 96 | public void test() throws Exception { 97 | String outputPath = "/tmp/output-plume-bypasstest"; 98 | String inputPath = "/tmp/input-wordcount.txt"; 99 | // Prepare input for test 100 | FileSystem system = FileSystem.getLocal(new Configuration()); 101 | system.copyFromLocalFile(new Path(Resources.getResource("simple-text.txt").getPath()), new Path(inputPath)); 102 | // Prepare output for test 103 | system.delete(new Path(outputPath), true); 104 | // Prepare workflow 105 | MapRedBypassWorkflow workFlow = new MapRedBypassWorkflow(); 106 | // Execute it 107 | MapRedExecutor executor = new MapRedExecutor(); 108 | executor.execute(workFlow, outputPath); 109 | 110 | List str = Files.readLines(new File(outputPath+"/1_1/1-r-00000"), Charsets.UTF_8); 111 | Map m = Maps.newHashMap(); 112 | for (String line: str) { 113 | m.put(line.split("\t")[0], line.split("\t")[1]); // not super-optimal, but less code 114 | } 115 | assertEquals(m.get("To test text processing with some simple-blah"), "To test text processing with some simple-bloh"); 116 | assertEquals(m.get("some simple text-blah"), "some simple text-bloh"); 117 | assertEquals(m.get("is is-blah"), "is is-bloh"); 118 | 119 | str = Files.readLines(new File(outputPath+"/1_2/2-r-00000"), Charsets.UTF_8); 120 | m = Maps.newHashMap(); 121 | for (String line: str) { 122 | m.put(line.split("\t")[0], line.split("\t")[1]); // not super-optimal, but less code 123 | } 124 | assertEquals(m.get("To test text processing with some simple"), "foo"); 125 | assertEquals(m.get("some simple text"), "foo"); 126 | assertEquals(m.get("is is"), "foo"); 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedFlattenTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.collectionOf; 21 | import static com.tdunning.plume.Plume.tableOf; 22 | import static com.tdunning.plume.Plume.strings; 23 | import static com.tdunning.plume.Plume.integers; 24 | 25 | import java.io.IOException; 26 | import java.text.SimpleDateFormat; 27 | import java.util.Date; 28 | 29 | import org.apache.hadoop.conf.Configuration; 30 | import org.apache.hadoop.fs.FileSystem; 31 | import org.apache.hadoop.fs.Path; 32 | import org.apache.hadoop.io.IntWritable; 33 | import org.apache.hadoop.io.NullWritable; 34 | import org.apache.hadoop.io.Text; 35 | import org.junit.Ignore; 36 | import org.junit.Test; 37 | 38 | import com.google.common.io.Resources; 39 | import com.tdunning.plume.DoFn; 40 | import com.tdunning.plume.EmitFn; 41 | import com.tdunning.plume.PCollection; 42 | import com.tdunning.plume.Pair; 43 | 44 | /** 45 | * This tests shows how two different MSCRS are executed sequentially: one from a single {@link Flatten} operation and another one 46 | * with three {@link GroupByKey} that is dependent on the first one. 47 | * 48 | */ 49 | public class MapRedFlattenTest extends BaseTestClass { 50 | 51 | static String inputPathEvent2 = "/tmp/input-event2.txt"; 52 | static String inputPathLogFile = "/tmp/input-logfile.txt"; 53 | 54 | public static class MapRedFlattenTestWorkflow extends PlumeWorkflow { 55 | 56 | @SuppressWarnings({ "unchecked", "rawtypes" }) 57 | @Override 58 | public void build() { 59 | init(); 60 | 61 | LazyPlume plume = new LazyPlume(); 62 | // Get input files 63 | PCollection inputEvent2, inputLogFile; 64 | try { 65 | inputEvent2 = plume.readFile(inputPathEvent2, collectionOf(strings())); 66 | inputLogFile = plume.readFile(inputPathLogFile, collectionOf(strings())); 67 | // Add as inputs 68 | addInput(inputEvent2); 69 | addInput(inputLogFile); 70 | } catch (IOException e) { 71 | throw new RuntimeException(); 72 | } 73 | 74 | /** 75 | * We use flatten to aggregate one log file we have with a list of users that used one new event. 76 | * The list of users is converted to the log format before flattening by adding a date and a event name. 77 | */ 78 | PCollection aggregateLog = plume.flatten( 79 | inputLogFile, 80 | inputEvent2.map(new DoFn() { 81 | @Override 82 | public void process(Text v, EmitFn emitter) { 83 | emitter.emit(new Text(new SimpleDateFormat("yyyy/MM/dd").format(new Date())+"\t"+"new_event"+"\t"+v.toString())); 84 | } 85 | }, collectionOf(strings()))); 86 | 87 | /** 88 | * We use the aggregate log to calculate a map of [date, user] -> #clicks 89 | */ 90 | PCollection dateUserClicks = aggregateLog.map(new DoFn() { 91 | @Override 92 | public void process(Text v, EmitFn emitter) { 93 | String[] splittedLine = v.toString().split("\t"); 94 | Text dateUser = new Text(splittedLine[0]+"\t"+splittedLine[2]); 95 | emitter.emit(Pair.create(dateUser, new IntWritable(1))); 96 | } 97 | }, tableOf(strings(), integers())) 98 | .groupByKey() 99 | .combine(countCombiner) 100 | .map(countReduceToText, tableOf(strings(), strings())); 101 | 102 | /** 103 | * We use the aggregate log to calculate a map of [date] -> #clicks 104 | */ 105 | PCollection dateClicks = aggregateLog.map(new DoFn() { 106 | @Override 107 | public void process(Text v, EmitFn emitter) { 108 | String[] splittedLine = v.toString().split("\t"); 109 | emitter.emit(Pair.create(new Text(splittedLine[0]), new IntWritable(1))); 110 | } 111 | }, tableOf(strings(), integers())) 112 | .groupByKey() 113 | .combine(countCombiner) 114 | .map(countReduceToText, tableOf(strings(), strings())); 115 | 116 | /** 117 | * We use the aggregate log to calculate a list of uniq users 118 | */ 119 | PCollection uniqUsers = aggregateLog.map(new DoFn() { 120 | @Override 121 | public void process(Text v, EmitFn emitter) { 122 | String[] splittedLine = v.toString().split("\t"); 123 | emitter.emit(Pair.create(new Text(splittedLine[2]), new Text(""))); 124 | } 125 | }, tableOf(strings(), strings())) 126 | .groupByKey() 127 | .map(new DoFn() { // Reduce - just emit the key 128 | @Override 129 | public void process(Pair v, EmitFn emitter) { 130 | emitter.emit((Text)v.getKey()); 131 | } 132 | }, collectionOf(strings())); 133 | 134 | addOutput(dateUserClicks); 135 | addOutput(dateClicks); 136 | addOutput(uniqUsers); 137 | } 138 | } 139 | 140 | @Test 141 | public void test() throws Exception { 142 | String outputPath = "/tmp/output-plume-flattentest"; 143 | // Prepare input for test 144 | FileSystem system = FileSystem.getLocal(new Configuration()); 145 | system.copyFromLocalFile(new Path(Resources.getResource("event2users.txt").getPath()), new Path(inputPathEvent2)); 146 | system.copyFromLocalFile(new Path(Resources.getResource("eventslog.txt").getPath()), new Path(inputPathLogFile)); 147 | // Prepare output for test 148 | system.delete(new Path(outputPath), true); 149 | // Prepare workflow 150 | MapRedFlattenTestWorkflow workFlow = new MapRedFlattenTestWorkflow(); 151 | // Execute it 152 | MapRedExecutor executor = new MapRedExecutor(); 153 | executor.execute(workFlow, outputPath); 154 | } 155 | } 156 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedMultipleGroupsTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.collectionOf; 21 | import static com.tdunning.plume.Plume.integers; 22 | import static com.tdunning.plume.Plume.strings; 23 | import static com.tdunning.plume.Plume.tableOf; 24 | import static org.junit.Assert.assertTrue; 25 | 26 | import java.io.File; 27 | import java.io.IOException; 28 | import java.util.Iterator; 29 | import java.util.StringTokenizer; 30 | 31 | import org.apache.hadoop.conf.Configuration; 32 | import org.apache.hadoop.fs.FileSystem; 33 | import org.apache.hadoop.fs.Path; 34 | import org.apache.hadoop.io.IntWritable; 35 | import org.apache.hadoop.io.Text; 36 | import org.apache.hadoop.io.WritableComparable; 37 | import org.junit.Test; 38 | 39 | import com.google.common.io.Resources; 40 | import com.tdunning.plume.DoFn; 41 | import com.tdunning.plume.EmitFn; 42 | import com.tdunning.plume.PCollection; 43 | import com.tdunning.plume.Pair; 44 | 45 | /** 46 | * This test shows how three different group by's can be performed using Plume API which output three different 47 | * things and end up being executed in a single Map/Red job. 48 | */ 49 | public class MapRedMultipleGroupsTest extends BaseTestClass { 50 | 51 | /** 52 | * The PlumeWorkflow class is extendend so that it can be instantiated via reflection at hadoop mappers/reducers 53 | */ 54 | public static class MultipleGroupsWorkflow extends PlumeWorkflow { 55 | 56 | @SuppressWarnings("unchecked") 57 | @Override 58 | public void build() { 59 | init(); 60 | 61 | // Get one input file 62 | LazyPlume plume = new LazyPlume(); 63 | PCollection input; 64 | try { 65 | input = plume.readFile("/tmp/input-wordcount.txt", collectionOf(strings())); 66 | // Add as input 67 | addInput(input); 68 | } catch (IOException e) { 69 | throw new RuntimeException(); 70 | } 71 | 72 | final IntWritable one = new IntWritable(1); 73 | 74 | // Define a map that counts and group by #chars of line 75 | PCollection po1 = input.map(new DoFn() { 76 | @Override 77 | public void process(Object v, EmitFn emitter) { 78 | StringTokenizer itr = new StringTokenizer(v.toString()); 79 | int length = 0; 80 | while (itr.hasMoreTokens()) { 81 | length += itr.nextToken().length(); 82 | } 83 | emitter.emit(Pair.create(new IntWritable(length), one)); 84 | } 85 | }, tableOf(integers(), integers())) 86 | .groupByKey() 87 | .map(countReduceToText, tableOf(integers(), strings())); 88 | 89 | // Define a map that counts and group by #tokens of line 90 | PCollection po2 = input.map(new DoFn() { 91 | @Override 92 | public void process(Object v, EmitFn emitter) { 93 | StringTokenizer itr = new StringTokenizer(v.toString()); 94 | int length = 0; 95 | while (itr.hasMoreTokens()) { 96 | length ++; 97 | itr.nextToken(); 98 | } 99 | emitter.emit(Pair.create(new IntWritable(length), one)); 100 | } 101 | }, tableOf(integers(), integers())) 102 | .groupByKey() 103 | .map(countReduceToText, tableOf(integers(), strings())); 104 | 105 | // Define a map that counts appearances of chars 106 | PCollection po3 = input.map(new DoFn() { 107 | @Override 108 | public void process(Object v, EmitFn emitter) { 109 | StringTokenizer itr = new StringTokenizer(v.toString()); 110 | while (itr.hasMoreTokens()) { 111 | String token = itr.nextToken(); 112 | for(int i = 0; i < token.length(); i++) { 113 | emitter.emit(Pair.create(new Text(token.charAt(i)+""), one)); 114 | } 115 | } 116 | } 117 | }, tableOf(strings(), integers())) 118 | .groupByKey() 119 | .map(countReduceToText, tableOf(strings(), strings())); 120 | 121 | // Add the output of the three group by's to this workflow's outputs 122 | addOutput(po1); 123 | addOutput(po2); 124 | addOutput(po3); 125 | } 126 | } 127 | 128 | @Test 129 | public void test() throws IOException, InterruptedException, ClassNotFoundException { 130 | String inputPath = "/tmp/input-wordcount.txt"; 131 | String outputPath = "/tmp/output-plume-complex"; 132 | // Prepare input for test 133 | FileSystem system = FileSystem.getLocal(new Configuration()); 134 | system.copyFromLocalFile(new Path(Resources.getResource("simple-text.txt").getPath()), new Path(inputPath)); 135 | // Prepare output for test 136 | system.delete(new Path(outputPath), true); 137 | // Prepare workflow 138 | MultipleGroupsWorkflow workFlow = new MultipleGroupsWorkflow(); 139 | // Execute it 140 | MapRedExecutor executor = new MapRedExecutor(); 141 | executor.execute(workFlow, outputPath); 142 | 143 | // Just assert that 3 output files were written and have content 144 | /** 145 | * TODO This test has to check the actual results of the 3 outputs 146 | */ 147 | for(int i = 1; i <= 3; i++) { 148 | File f = new File(outputPath+"/1_" + i + "/" + i +"-r-00000"); 149 | assertTrue(f.exists()); 150 | assertTrue(f.length() > 64); 151 | } 152 | } 153 | } 154 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedOnlyFlattensTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.collectionOf; 21 | import static com.tdunning.plume.Plume.strings; 22 | 23 | import java.io.IOException; 24 | 25 | import org.apache.hadoop.conf.Configuration; 26 | import org.apache.hadoop.fs.FileSystem; 27 | import org.apache.hadoop.fs.Path; 28 | import org.apache.hadoop.io.Text; 29 | import org.junit.Test; 30 | 31 | import com.google.common.io.Resources; 32 | import com.tdunning.plume.DoFn; 33 | import com.tdunning.plume.EmitFn; 34 | import com.tdunning.plume.PCollection; 35 | 36 | /** 37 | * This test checks that an MSCR is created out of a workplan that consists only of Flatten operations. There is more than one 38 | * {@link Flatten} and these have co-related inputs, so they have to be merged into a single MSCR. 39 | */ 40 | public class MapRedOnlyFlattensTest { 41 | 42 | static String inputPathEvent2 = "/tmp/input-event2.txt"; 43 | static String inputPathLogFile2 = "/tmp/input-logfile2.txt"; 44 | static String inputPathLogFile = "/tmp/input-logfile.txt"; 45 | 46 | public static class MapRedOnlyFlattensTestWorkflow extends PlumeWorkflow { 47 | 48 | @SuppressWarnings({ "unchecked", "rawtypes" }) 49 | @Override 50 | public void build() { 51 | init(); 52 | 53 | LazyPlume plume = new LazyPlume(); 54 | // Get input files 55 | PCollection inputEvent2, inputLogFile, inputLogFile2; 56 | try { 57 | inputEvent2 = plume.readFile(inputPathEvent2, collectionOf(strings())); 58 | inputLogFile2 = plume.readFile(inputPathLogFile2, collectionOf(strings())); 59 | inputLogFile = plume.readFile(inputPathLogFile, collectionOf(strings())); 60 | // Add as inputs 61 | addInput(inputEvent2); 62 | addInput(inputLogFile); 63 | } catch (IOException e) { 64 | throw new RuntimeException(); 65 | } 66 | 67 | /** 68 | * Emit the user of the log file - flatten it with users file 69 | */ 70 | PCollection output = plume.flatten(collectionOf(strings()), 71 | inputEvent2, 72 | inputLogFile.map(new DoFn() { 73 | @Override 74 | public void process(Text v, EmitFn emitter) { 75 | String[] splittedLine = v.toString().split("\t"); 76 | emitter.emit(new Text(splittedLine[2])); 77 | } 78 | }, collectionOf(strings()))); 79 | 80 | /** 81 | * Flatten two log files 82 | */ 83 | PCollection output2 = plume.flatten(collectionOf(strings()), 84 | inputLogFile2, 85 | inputLogFile); 86 | 87 | addOutput(output); 88 | addOutput(output2); 89 | } 90 | } 91 | 92 | @Test 93 | public void test() throws Exception { 94 | String outputPath = "/tmp/output-plume-onlyflattentest"; 95 | // Prepare input for test 96 | FileSystem system = FileSystem.getLocal(new Configuration()); 97 | system.copyFromLocalFile(new Path(Resources.getResource("event2users.txt").getPath()), new Path(inputPathEvent2)); 98 | system.copyFromLocalFile(new Path(Resources.getResource("eventslog.txt").getPath()), new Path(inputPathLogFile)); 99 | system.copyFromLocalFile(new Path(Resources.getResource("eventslog.txt").getPath()), new Path(inputPathLogFile2)); 100 | // Prepare output for test 101 | system.delete(new Path(outputPath), true); 102 | // Prepare workflow 103 | MapRedOnlyFlattensTestWorkflow workFlow = new MapRedOnlyFlattensTestWorkflow(); 104 | // Execute it 105 | MapRedExecutor executor = new MapRedExecutor(); 106 | executor.execute(workFlow, outputPath); 107 | } 108 | } 109 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedSequenceFileTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.integers; 21 | import static com.tdunning.plume.Plume.tableOf; 22 | import static org.junit.Assert.*; 23 | 24 | import java.io.File; 25 | import java.io.IOException; 26 | 27 | import org.apache.hadoop.conf.Configuration; 28 | import org.apache.hadoop.fs.FileSystem; 29 | import org.apache.hadoop.fs.Path; 30 | import org.apache.hadoop.io.IntWritable; 31 | import org.apache.hadoop.io.SequenceFile; 32 | import org.apache.hadoop.mapreduce.Job; 33 | import org.junit.Ignore; 34 | import org.junit.Test; 35 | 36 | import com.google.common.io.Files; 37 | import com.tdunning.plume.DoFn; 38 | import com.tdunning.plume.EmitFn; 39 | import com.tdunning.plume.PCollection; 40 | import com.tdunning.plume.Pair; 41 | 42 | /** 43 | * The purpose of this test is to assert that {@link MapRedExecutor} behaves correctly with SequenceFile as input/output 44 | */ 45 | public class MapRedSequenceFileTest { 46 | 47 | final static String inputPath = "/tmp/input-simpletest/file"; 48 | 49 | /** 50 | * This workflow will only load a table and add one to each number 51 | */ 52 | public static class OtherWorkflow extends PlumeWorkflow { 53 | 54 | public OtherWorkflow() { 55 | } 56 | 57 | @SuppressWarnings("unchecked") 58 | @Override 59 | public void build() { 60 | init(); 61 | LazyPlume plume = new LazyPlume(); 62 | PCollection input; 63 | try { 64 | // Get input file 65 | input = plume.readFile(inputPath, tableOf(integers(), integers())); 66 | // Add as input for this workflow 67 | addInput(input); 68 | } catch (IOException e) { 69 | throw new RuntimeException(e); 70 | } 71 | // Define its output 72 | PCollection output = input.map(new DoFn, Pair>() { 73 | @Override 74 | public void process(Pair v, 75 | EmitFn> emitter) { 76 | emitter.emit(Pair.create(new IntWritable(v.getKey().get() + 1), new IntWritable(v.getValue().get() + 1))); 77 | } 78 | }, tableOf(integers(), integers())).groupByKey(); 79 | 80 | // Add it as workflow's output 81 | addOutput(output); 82 | } 83 | } 84 | 85 | @Test 86 | public void test() throws Exception { 87 | /* 88 | * Create input which is SequenceFile with data 1,2\n3,4 89 | */ 90 | Configuration conf = new Configuration(); 91 | Path p = new Path(inputPath); 92 | FileSystem localFS = FileSystem.getLocal(conf); 93 | if(localFS.exists(p)) { 94 | localFS.delete(p, true); // wipe it if needed 95 | } 96 | SequenceFile.Writer writer = SequenceFile.createWriter(localFS, conf, p, IntWritable.class, IntWritable.class); 97 | writer.append(new IntWritable(1), new IntWritable(2)); 98 | writer.append(new IntWritable(3), new IntWritable(4)); 99 | writer.close(); 100 | String outputPath = "/tmp/output-plume-simpletest"; 101 | // Prepare input for test 102 | FileSystem system = FileSystem.getLocal(new Configuration()); 103 | // Prepare output for test 104 | system.delete(new Path(outputPath), true); 105 | // Prepare workflow 106 | OtherWorkflow workFlow = new OtherWorkflow(); 107 | // Execute it 108 | MapRedExecutor executor = new MapRedExecutor(); 109 | executor.execute(workFlow, outputPath); 110 | /* 111 | * Read output which is SequenceFile and assert that it has data 2,3\n4,5 112 | */ 113 | p = new Path(outputPath + "/1_1/1-r-00000"); 114 | SequenceFile.Reader reader = new SequenceFile.Reader(localFS, p, conf); 115 | IntWritable key = new IntWritable(1); 116 | IntWritable value = new IntWritable(1); 117 | reader.next(key, value); 118 | assertEquals(key.get(), 2); 119 | assertEquals(value.get(), 3); 120 | reader.next(key, value); 121 | assertEquals(key.get(), 4); 122 | assertEquals(value.get(), 5); 123 | reader.close(); 124 | } 125 | } 126 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedSingleFlattenChannelTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | import static com.tdunning.plume.Plume.collectionOf; 20 | import static com.tdunning.plume.Plume.strings; 21 | import static com.tdunning.plume.Plume.tableOf; 22 | 23 | import java.io.IOException; 24 | 25 | import org.apache.hadoop.conf.Configuration; 26 | import org.apache.hadoop.fs.FileSystem; 27 | import org.apache.hadoop.fs.Path; 28 | import org.apache.hadoop.io.Text; 29 | import org.junit.Test; 30 | 31 | import com.google.common.io.Resources; 32 | import com.tdunning.plume.DoFn; 33 | import com.tdunning.plume.EmitFn; 34 | import com.tdunning.plume.PCollection; 35 | import com.tdunning.plume.Pair; 36 | 37 | /** 38 | * 39 | */ 40 | public class MapRedSingleFlattenChannelTest { 41 | 42 | /** 43 | * In this example we open a file and apply two functions to it. One of them performs a group by key and the other one 44 | * just adds as output the result of the second function (bypass channel). 45 | */ 46 | public static class MapRedSingleFlattenChannelTestWorkflow extends PlumeWorkflow { 47 | 48 | public MapRedSingleFlattenChannelTestWorkflow() { 49 | } 50 | 51 | @SuppressWarnings("unchecked") 52 | @Override 53 | public void build() { 54 | init(); 55 | 56 | LazyPlume plume = new LazyPlume(); 57 | PCollection input; 58 | PCollection input2; 59 | try { 60 | // Read input 61 | input = plume.readFile("/tmp/input-wordcount.txt", collectionOf(strings())); 62 | input2 = plume.readFile("/tmp/input-moretext.txt", collectionOf(strings())); 63 | // Add it as workflow's input 64 | addInput(input); 65 | } catch (IOException e) { 66 | throw new RuntimeException(); 67 | } 68 | 69 | PCollection transform = input.map(new DoFn() { 70 | @Override 71 | public void process(Object v, EmitFn emitter) { 72 | Text t = (Text)v; 73 | emitter.emit(new Text(t.toString()+"-bar")); 74 | }}, collectionOf(strings())); 75 | 76 | addOutput(plume.flatten(input2, transform)); // flatten with another file 77 | 78 | PCollection groupedTransform = input.map(new DoFn() { 79 | @Override 80 | public void process(Object v, EmitFn emitter) { 81 | Text t = (Text)v; 82 | emitter.emit(Pair.create(t, new Text("foo"))); 83 | }}, tableOf(strings(), strings())).groupByKey(); 84 | 85 | addOutput(groupedTransform); 86 | } 87 | } 88 | 89 | @Test 90 | public void test() throws Exception { 91 | String outputPath = "/tmp/output-plume-singleflattenchanneltest"; 92 | String inputPath = "/tmp/input-wordcount.txt"; 93 | String inputPath2 = "/tmp/input-moretext.txt"; 94 | // Prepare input for test 95 | FileSystem system = FileSystem.getLocal(new Configuration()); 96 | system.copyFromLocalFile(new Path(Resources.getResource("simple-text.txt").getPath()), new Path(inputPath)); 97 | system.copyFromLocalFile(new Path(Resources.getResource("simple-text.txt").getPath()), new Path(inputPath2)); 98 | // Prepare output for test 99 | system.delete(new Path(outputPath), true); 100 | // Prepare workflow 101 | MapRedSingleFlattenChannelTestWorkflow workFlow = new MapRedSingleFlattenChannelTestWorkflow(); 102 | // Execute it 103 | MapRedExecutor executor = new MapRedExecutor(); 104 | executor.execute(workFlow, outputPath); 105 | /** 106 | * TODO add test validation 107 | */ 108 | } 109 | } 110 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedTwoSequentialGBKTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.collectionOf; 21 | import static com.tdunning.plume.Plume.strings; 22 | import static com.tdunning.plume.Plume.tableOf; 23 | import static org.junit.Assert.assertEquals; 24 | 25 | import java.io.File; 26 | import java.io.IOException; 27 | import java.util.List; 28 | import java.util.Map; 29 | 30 | import org.apache.hadoop.conf.Configuration; 31 | import org.apache.hadoop.fs.FileSystem; 32 | import org.apache.hadoop.fs.Path; 33 | import org.apache.hadoop.io.Text; 34 | import org.junit.Test; 35 | 36 | import com.google.common.base.Charsets; 37 | import com.google.common.collect.Maps; 38 | import com.google.common.io.Files; 39 | import com.google.common.io.Resources; 40 | import com.tdunning.plume.DoFn; 41 | import com.tdunning.plume.EmitFn; 42 | import com.tdunning.plume.PCollection; 43 | import com.tdunning.plume.Pair; 44 | 45 | /** 46 | * This test asserts that Plume creates two MSCR and therefore two MapReduce jobs when one Group By Key follows another 47 | */ 48 | public class MapRedTwoSequentialGBKTest { 49 | 50 | public static class TwoSequentialGBKWorkflow extends PlumeWorkflow { 51 | 52 | public TwoSequentialGBKWorkflow() { 53 | } 54 | 55 | @SuppressWarnings("unchecked") 56 | @Override 57 | public void build() { 58 | init(); 59 | 60 | LazyPlume plume = new LazyPlume(); 61 | PCollection input; 62 | try { 63 | // Read input 64 | input = plume.readFile("/tmp/input-wordcount.txt", collectionOf(strings())); 65 | // Add it as workflow's input 66 | addInput(input); 67 | } catch (IOException e) { 68 | throw new RuntimeException(); 69 | } 70 | 71 | PCollection output = input.map(new DoFn() { 72 | @Override 73 | public void process(Object v, EmitFn emitter) { 74 | Text t = (Text)v; 75 | // do some foo processing 76 | emitter.emit(Pair.create(t, new Text("foo"))); 77 | }}, tableOf(strings(), strings())) 78 | .groupByKey() 79 | .map(new DoFn() { 80 | public void process(Object v, EmitFn emitter) { 81 | Pair p = (Pair)v; 82 | // do some more foo processing 83 | emitter.emit(Pair.create(p.getKey(), new Text("bar"))); 84 | } 85 | }, tableOf(strings(), strings())) 86 | // second group by key 87 | .groupByKey() 88 | .map(new DoFn() { 89 | public void process(Object v, EmitFn emitter) { 90 | Pair p = (Pair)v; 91 | // do some more foo processing 92 | emitter.emit(Pair.create(p.getKey(), new Text("bar 2"))); 93 | } 94 | }, tableOf(strings(), strings())); 95 | 96 | addOutput(output); 97 | } 98 | } 99 | 100 | @Test 101 | public void test() throws Exception { 102 | String outputPath = "/tmp/output-plume-twosequentialgbktest"; 103 | String inputPath = "/tmp/input-wordcount.txt"; 104 | // Prepare input for test 105 | FileSystem system = FileSystem.getLocal(new Configuration()); 106 | system.copyFromLocalFile(new Path(Resources.getResource("simple-text.txt").getPath()), new Path(inputPath)); 107 | // Prepare output for test 108 | system.delete(new Path(outputPath), true); 109 | // Prepare workflow 110 | TwoSequentialGBKWorkflow workFlow = new TwoSequentialGBKWorkflow(); 111 | 112 | // Execute it 113 | MapRedExecutor executor = new MapRedExecutor(); 114 | executor.execute(workFlow, outputPath); 115 | 116 | String outputId = ((LazyCollection)workFlow.getOutputs().get(0)).getPlumeId(); 117 | List str = Files.readLines(new File(outputPath+"/"+outputId+"/1-r-00000"), Charsets.UTF_8); 118 | 119 | Map m = Maps.newHashMap(); 120 | for (String line: str) { 121 | m.put(line.split("\t")[0], line.split("\t")[1]); // not super-optimal, but less code 122 | } 123 | assertEquals("bar 2", m.get("To test text processing with some simple")); 124 | assertEquals("bar 2", m.get("examples is some simple")); 125 | assertEquals("bar 2", m.get("is is")); 126 | assertEquals("bar 2", m.get("some simple text")); 127 | } 128 | } 129 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/MapRedWordCountTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static com.tdunning.plume.Plume.integers; 21 | import static com.tdunning.plume.Plume.strings; 22 | import static com.tdunning.plume.Plume.tableOf; 23 | import static org.junit.Assert.assertEquals; 24 | 25 | import java.io.File; 26 | import java.io.IOException; 27 | import java.util.List; 28 | import java.util.Map; 29 | import java.util.StringTokenizer; 30 | 31 | import org.apache.hadoop.conf.Configuration; 32 | import org.apache.hadoop.fs.FileSystem; 33 | import org.apache.hadoop.fs.Path; 34 | import org.apache.hadoop.io.IntWritable; 35 | import org.apache.hadoop.io.Text; 36 | import org.junit.Test; 37 | 38 | import com.google.common.base.Charsets; 39 | import com.google.common.collect.Maps; 40 | import com.google.common.io.Files; 41 | import com.google.common.io.Resources; 42 | import com.tdunning.plume.CombinerFn; 43 | import com.tdunning.plume.DoFn; 44 | import com.tdunning.plume.EmitFn; 45 | import com.tdunning.plume.PCollection; 46 | import com.tdunning.plume.Pair; 47 | import com.tdunning.plume.types.PCollectionType; 48 | import com.tdunning.plume.types.StringType; 49 | 50 | import static com.tdunning.plume.Plume.*; 51 | 52 | /** 53 | * This test asserts that {@link MapRedExecutor} behaves well under the famous WordCount test 54 | */ 55 | public class MapRedWordCountTest extends BaseTestClass { 56 | 57 | /** 58 | * The WordCount Workflow 59 | */ 60 | @SuppressWarnings({ "unchecked", "rawtypes" }) 61 | public static class WordCountWorkflow extends PlumeWorkflow { 62 | 63 | public WordCountWorkflow() { 64 | } 65 | 66 | @Override 67 | public void build() { 68 | init(); 69 | 70 | LazyPlume plume = new LazyPlume(); 71 | PCollection input; 72 | try { 73 | // Read input 74 | input = plume.readFile("/tmp/input-wordcount.txt", collectionOf(strings())); 75 | // Add it as workflow's input 76 | addInput(input); 77 | } catch (IOException e) { 78 | throw new RuntimeException(); 79 | } 80 | 81 | // Define the wordcount map 82 | DoFn wordCountMap = new DoFn() { 83 | @Override 84 | public void process(Object v, EmitFn emitter) { 85 | StringTokenizer itr = new StringTokenizer(v.toString()); 86 | while (itr.hasMoreTokens()) { 87 | emitter.emit(Pair.create(new Text(itr.nextToken()), new IntWritable(1))); 88 | } 89 | } 90 | }; 91 | 92 | // Define the wordcount output 93 | PCollection output = input.map(wordCountMap, tableOf(strings(), integers())) 94 | .groupByKey() 95 | .combine(countCombiner) 96 | .map(countReduceToText, tableOf(strings(), strings())); 97 | 98 | // Add wordcount's output as workflow's output 99 | addOutput(output); 100 | } 101 | } 102 | 103 | /** 104 | * The wordcount example to test with local hadoop 105 | * 106 | * @throws IOException 107 | * @throws ClassNotFoundException 108 | * @throws InterruptedException 109 | */ 110 | @Test 111 | public void testWordCount() throws IOException, InterruptedException, ClassNotFoundException { 112 | String inputPath = "/tmp/input-wordcount.txt"; 113 | String outputPath = "/tmp/output-plume-wordcount"; 114 | // Prepare input for test 115 | FileSystem system = FileSystem.getLocal(new Configuration()); 116 | system.copyFromLocalFile(new Path(Resources.getResource("simple-text.txt").getPath()), new Path(inputPath)); 117 | // Prepare output for test 118 | system.delete(new Path(outputPath), true); 119 | // Prepare workflow 120 | WordCountWorkflow workFlow = new WordCountWorkflow(); 121 | // Execute it 122 | MapRedExecutor executor = new MapRedExecutor(); 123 | executor.execute(workFlow, outputPath); 124 | 125 | List str = Files.readLines(new File(outputPath+"/1_1/1-r-00000"), Charsets.UTF_8); 126 | 127 | Map m = Maps.newHashMap(); 128 | for (String line: str) { 129 | m.put(line.split("\t")[0], line.split("\t")[1]); // not super-optimal, but less code 130 | } 131 | assertEquals(3+"", m.get("is")); 132 | assertEquals(3+"", m.get("some")); 133 | assertEquals(3+"", m.get("simple")); 134 | assertEquals(1+"", m.get("examples")); 135 | assertEquals(2+"", m.get("text")); 136 | } 137 | } 138 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/TestOptimizer.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static org.junit.Assert.*; 21 | import static com.tdunning.plume.Plume.integers; 22 | import static com.tdunning.plume.Plume.tableOf; 23 | 24 | import java.util.ArrayList; 25 | import java.util.Iterator; 26 | import java.util.List; 27 | 28 | import org.junit.Test; 29 | 30 | import com.google.common.collect.Lists; 31 | import com.tdunning.plume.DoFn; 32 | import com.tdunning.plume.EmitFn; 33 | import com.tdunning.plume.PCollection; 34 | import com.tdunning.plume.PTable; 35 | import com.tdunning.plume.Pair; 36 | 37 | /** 38 | * This set of tests assert that the {@link Optimizer} works under well-known schemas like the ones in FlumeJava paper. 39 | */ 40 | public class TestOptimizer extends BaseTestClass { 41 | 42 | /** 43 | * Test figure 4 of FlumeJava paper 44 | */ 45 | @SuppressWarnings({ "rawtypes", "unchecked" }) 46 | @Test 47 | public void testFigure4() { 48 | // Get Plume runtime 49 | LazyPlume plume = new LazyPlume(); 50 | // Create simple data 51 | PCollection input1 = plume.fromJava(Lists.newArrayList(Pair.create(1, 1))); 52 | PCollection input2 = plume.fromJava(Lists.newArrayList(Pair.create(2, 2))); 53 | PCollection input3 = plume.fromJava(Lists.newArrayList(Pair.create(3, 3))); 54 | PCollection input4 = plume.fromJava(Lists.newArrayList(Pair.create(4, 4))); 55 | 56 | PCollection output1 = plume.flatten(tableOf(integers(), integers()), 57 | input1.map(identity, tableOf(integers(), integers())), 58 | input2.map(identity, tableOf(integers(), integers()))) 59 | .groupByKey(); 60 | 61 | PCollection output2 = plume.flatten(tableOf(integers(), integers()), 62 | input2.map(identity, tableOf(integers(), integers())), 63 | input3.map(identity, tableOf(integers(), integers())), 64 | input4.map(identity, tableOf(integers(), integers()))) 65 | .groupByKey() 66 | .combine(dummyCombiner) 67 | .map(identity, null); 68 | 69 | PCollection output3 = plume.flatten(tableOf(integers(), integers()), 70 | input4.map(identity, tableOf(integers(), integers()))) 71 | .groupByKey() 72 | .map(identity, null); 73 | 74 | Optimizer optimizer = new Optimizer(); 75 | ExecutionStep step = optimizer.optimize( 76 | Lists.newArrayList(input1, input2, input3, input4), 77 | Lists.newArrayList(output1, output2, output3) 78 | ); 79 | 80 | assertEquals(step.mscrSteps.size(), 1); 81 | assertEquals(step.nextStep, null); 82 | } 83 | 84 | @SuppressWarnings({ "rawtypes", "unchecked" }) 85 | public void testFigure5() { 86 | // Get Plume runtime 87 | LazyPlume plume = new LazyPlume(); 88 | // Create simple data 89 | PCollection input1 = plume.fromJava(Lists.newArrayList(Pair.create(1, 1))); 90 | PCollection input2 = plume.fromJava(Lists.newArrayList(Pair.create(2, 2))); 91 | PCollection input3 = plume.fromJava(Lists.newArrayList(Pair.create(3, 3))); 92 | PCollection input4 = plume.fromJava(Lists.newArrayList(Pair.create(4, 4))); 93 | 94 | PCollection partial1 = input1.map(identity, tableOf(integers(), integers())); 95 | PCollection partial2 = 96 | plume.flatten(tableOf(integers(), integers()), 97 | input2.map(identity, tableOf(integers(), integers())), 98 | input3.map(identity, tableOf(integers(), integers())) 99 | .map(identity, null) 100 | .map(identity, null)); 101 | 102 | PCollection partial3 = 103 | input4.map(identity, tableOf(integers(), integers())) 104 | .groupByKey() 105 | .combine(dummyCombiner) 106 | .map(identity, null); 107 | 108 | PCollection output = plume.flatten(tableOf(integers(), integers()), partial1, partial2, partial3) 109 | .groupByKey() 110 | .map(identity, null); 111 | 112 | Optimizer optimizer = new Optimizer(); 113 | ExecutionStep step = optimizer.optimize( 114 | Lists.newArrayList(input1, input2, input3, input4), 115 | Lists.newArrayList(output, partial1) 116 | ); 117 | 118 | assertEquals(step.mscrSteps.size(), 1); 119 | assertNotNull(step.nextStep); 120 | assertEquals(step.nextStep.mscrSteps.size(), 1); 121 | assertNull(step.nextStep.nextStep); 122 | } 123 | 124 | public void testEasyChain() { 125 | LazyPlume plume = new LazyPlume(); 126 | /* 127 | * Two lists, empty mapper, reducer that emits first number 128 | * 129 | * Input: (1,1),(1,2),(1,3) + (2,10),(2,20),(3,30) => (1, (1,2,3)),(2, (10,20)),(3,30) => (1,1),(2,10),(3,30) 130 | */ 131 | List> l1 = Lists.newArrayList(Pair.create(1,1),Pair.create(1,2),Pair.create(1,3)); 132 | List> l2 = Lists.newArrayList(Pair.create(2,10),Pair.create(2,20),Pair.create(3,30)); 133 | PTable i1 = plume.fromJava(l1, tableOf(integers(), integers())); 134 | PTable i2 = plume.fromJava(l2, tableOf(integers(), integers())); 135 | PTable o = plume.flatten(tableOf(integers(), integers()), i1, i2) 136 | .groupByKey() 137 | .map(new DoFn>, Pair>() { 138 | @Override 139 | public void process(Pair> v, 140 | EmitFn> emitter) { 141 | emitter.emit(Pair.create(v.getKey(), v.getValue().iterator().next())); 142 | } 143 | }, tableOf(integers(), integers())); 144 | 145 | LocalExecutor executor = new LocalExecutor(); 146 | Iterable> result = executor.execute((LazyTable)o); 147 | Iterator> it = result.iterator(); 148 | Pair next = it.next(); 149 | // assert the expected result without assuming pairs are ordered 150 | assertTrue(next.getKey() == 1 && next.getValue() == 1 || next.getKey() == 2 && next.getValue() == 10 || next.getKey() == 3 && next.getValue() == 30); 151 | next = it.next(); 152 | assertTrue(next.getKey() == 1 && next.getValue() == 1 || next.getKey() == 2 && next.getValue() == 10 || next.getKey() == 3 && next.getValue() == 30); 153 | next = it.next(); 154 | assertTrue(next.getKey() == 1 && next.getValue() == 1 || next.getKey() == 2 && next.getValue() == 10 || next.getKey() == 3 && next.getValue() == 30); 155 | 156 | Optimizer optimizer = new Optimizer(); 157 | List outputs = new ArrayList(); 158 | List inputs = new ArrayList(); 159 | inputs.add(i1); inputs.add(i2); 160 | outputs.add(o); 161 | ExecutionStep step = optimizer.optimize(inputs, outputs); 162 | assertTrue(step.getMscrSteps().size() == 1); 163 | 164 | MSCR toExecute = step.getMscrSteps().iterator().next(); 165 | assertEquals(toExecute.getInputs().size(), 2); 166 | assertEquals(toExecute.getOutputChannels().size(), 1); 167 | } 168 | } -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/lazy/TestOptimizerTools.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.lazy; 19 | 20 | import static org.junit.Assert.assertEquals; 21 | import static com.tdunning.plume.Plume.integers; 22 | import static com.tdunning.plume.Plume.tableOf; 23 | import static org.junit.Assert.assertTrue; 24 | 25 | import java.util.ArrayList; 26 | import java.util.Iterator; 27 | import java.util.List; 28 | import java.util.Set; 29 | 30 | import org.junit.Test; 31 | 32 | import com.google.common.collect.Lists; 33 | import com.tdunning.plume.DoFn; 34 | import com.tdunning.plume.EmitFn; 35 | import com.tdunning.plume.PCollection; 36 | import com.tdunning.plume.Pair; 37 | import com.tdunning.plume.local.lazy.op.DeferredOp; 38 | import com.tdunning.plume.local.lazy.op.GroupByKey; 39 | 40 | public class TestOptimizerTools extends BaseTestClass { 41 | 42 | @Test 43 | @SuppressWarnings({"unchecked"}) 44 | public void testGroupByKeys() { 45 | // Get Plume runtime 46 | LazyPlume plume = new LazyPlume(); 47 | // Create simple data 48 | PCollection input1 = plume.fromJava(Lists.newArrayList(1, 2, 3)); 49 | PCollection input2 = plume.fromJava(Lists.newArrayList(4, 5, 6)); 50 | PCollection output = 51 | plume.flatten( 52 | input1.map(plusTwoPlusThree, tableOf(integers(), integers())) 53 | .groupByKey(), 54 | input2.map(plusTwoPlusThree, tableOf(integers(), integers())) 55 | .groupByKey()) 56 | .map(new DoFn>, Integer>() { 57 | @Override 58 | public void process(Pair> v, 59 | EmitFn emitter) { 60 | emitter.emit(1); 61 | } 62 | }, null); 63 | List groupBys = OptimizerTools.getAll(output, GroupByKey.class); 64 | assertEquals(groupBys.size(), 2); 65 | } 66 | 67 | @SuppressWarnings("unchecked") 68 | @Test 69 | public void testMSCRBlocks() { 70 | // Get Plume runtime 71 | LazyPlume plume = new LazyPlume(); 72 | // Create simple data 73 | PCollection input1 = plume.fromJava(Lists.newArrayList(1, 2, 3)); 74 | PCollection input2 = plume.fromJava(Lists.newArrayList(4, 5, 6)); 75 | PCollection input3 = plume.fromJava(Lists.newArrayList(7, 8, 9)); 76 | // One inner group by key 77 | PCollection output = 78 | plume.flatten( 79 | tableOf(integers(), integers()), 80 | input1.map(plusTwoPlusThree, tableOf(integers(), integers())), 81 | input2.map(plusTwoPlusThree, tableOf(integers(), integers())), 82 | input3.map(plusTwoPlusThree, tableOf(integers(), integers())) 83 | .groupByKey() 84 | .combine(dummyCombiner) 85 | ) 86 | .groupByKey(); 87 | 88 | List outputs = new ArrayList(); 89 | outputs.add(output); 90 | Set mscrBlocks = OptimizerTools.getMSCRBlocks(outputs); 91 | assertEquals(mscrBlocks.size(), 2); 92 | Iterator iterator = mscrBlocks.iterator(); 93 | for(int i = 0; i < 2; i++) { 94 | MSCR mscr = iterator.next(); 95 | if(mscr.hasInput(input1)) { 96 | assertTrue(mscr.hasInput(input2)); 97 | assertEquals(mscr.getInputs().size(), 3); 98 | } else if(mscr.hasInput(input3)) { 99 | assertEquals(mscr.getInputs().size(), 1); 100 | } 101 | } 102 | } 103 | } 104 | -------------------------------------------------------------------------------- /src/test/java/com/tdunning/plume/local/mapReduce/MapReduceTest.java: -------------------------------------------------------------------------------- 1 | /* 2 | * Licensed to the Apache Software Foundation (ASF) under one or more 3 | * contributor license agreements. See the NOTICE file distributed with 4 | * this work for additional information regarding copyright ownership. 5 | * The ASF licenses this file to You under the Apache License, Version 2.0 6 | * (the "License"); you may not use this file except in compliance with 7 | * the License. You may obtain a copy of the License at 8 | * 9 | * http://www.apache.org/licenses/LICENSE-2.0 10 | * 11 | * Unless required by applicable law or agreed to in writing, software 12 | * distributed under the License is distributed on an "AS IS" BASIS, 13 | * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 | * See the License for the specific language governing permissions and 15 | * limitations under the License. 16 | */ 17 | 18 | package com.tdunning.plume.local.mapReduce; 19 | 20 | import com.google.common.base.Function; 21 | import com.google.common.collect.HashMultiset; 22 | import com.google.common.collect.Iterables; 23 | import com.google.common.collect.Lists; 24 | import com.google.common.collect.Maps; 25 | import com.google.common.collect.Multiset; 26 | import com.google.common.collect.Sets; 27 | import com.tdunning.plume.Pair; 28 | import org.junit.Test; 29 | 30 | import java.util.ArrayList; 31 | import java.util.List; 32 | import java.util.Map; 33 | import java.util.Random; 34 | import java.util.Scanner; 35 | import java.util.Set; 36 | 37 | import static org.junit.Assert.assertEquals; 38 | 39 | public class MapReduceTest { 40 | @Test 41 | public void mapOnly() { 42 | MapReduceBuilder x1 = new MapReduceBuilder(); 43 | MapReduceBuilder x2 = x1.map(new Mapper() { 44 | @Override 45 | public void map(Integer key, Integer value, Collector out) { 46 | out.collect(key, value + 1); 47 | } 48 | }); 49 | MapReduce mr = x2.build(); 50 | 51 | Random gen = new Random(); 52 | 53 | List> in = Lists.newArrayList( 54 | Pair.create(gen.nextInt(), 1), 55 | Pair.create(gen.nextInt(), 2), 56 | Pair.create(gen.nextInt(), 3), 57 | Pair.create(gen.nextInt(), 4) 58 | ); 59 | 60 | Iterable> out = mr.run(in); 61 | 62 | Set r = Sets.newTreeSet(Iterables.transform(out, new Function, Integer>() { 63 | @Override 64 | public Integer apply(Pair x) { 65 | return x.getValue(); 66 | } 67 | })); 68 | 69 | assertEquals(4, r.size()); 70 | r.removeAll(Lists.newArrayList(2, 3, 4, 5)); 71 | assertEquals(0, r.size()); 72 | } 73 | 74 | @Test 75 | public void mapAndReduce() { 76 | List> words = Lists.newArrayList(); 77 | Multiset ref = HashMultiset.create(); 78 | 79 | int k = 0; 80 | Random gen = new Random(); 81 | for (String letter : "abcdefghij".split("")) { 82 | // add 2^k of this letter 83 | for (int i = 0; i < (1 << k); i++) { 84 | words.add(Pair.create(gen.nextInt(), letter)); 85 | ref.add(letter); 86 | } 87 | k++; 88 | } 89 | 90 | MapReduce mr = new MapReduceBuilder() 91 | .map(new Mapper() { 92 | @Override 93 | public void map(Integer key, String value, Collector out) { 94 | out.collect(value, 1); 95 | } 96 | }) 97 | .reduce(new Reducer() { 98 | @Override 99 | public void reduce(String key, Iterable values, Collector out) { 100 | int sum = 0; 101 | for (Integer value : values) { 102 | sum += value; 103 | } 104 | out.collect(key, sum); 105 | } 106 | }) 107 | .build(); 108 | 109 | Iterable> out = mr.run(words); 110 | for (Pair pair : out) { 111 | assertEquals(ref.count(pair.getKey()), pair.getValue().intValue()); 112 | } 113 | } 114 | 115 | @Test 116 | public void mapReduceAndCombine() { 117 | List> words = Lists.newArrayList(); 118 | Multiset ref = HashMultiset.create(); 119 | 120 | int k = 0; 121 | Random gen = new Random(); 122 | for (String letter : "abcdefghij".split("")) { 123 | // add 2^k of this letter 124 | for (int i = 0; i < (1 << k); i++) { 125 | words.add(Pair.create(gen.nextInt(), letter)); 126 | ref.add(letter); 127 | } 128 | k++; 129 | } 130 | 131 | Reducer r = new Reducer() { 132 | @Override 133 | public void reduce(String key, Iterable values, Collector out) { 134 | int sum = 0; 135 | for (Integer value : values) { 136 | sum += value; 137 | } 138 | out.collect(key, sum); 139 | } 140 | }; 141 | MapReduce mr = new MapReduceBuilder() 142 | .map(new Mapper() { 143 | @Override 144 | public void map(Integer key, String value, Collector out) { 145 | out.collect(value, 1); 146 | } 147 | }) 148 | .reduce(r) 149 | .combine(r) 150 | .build(); 151 | 152 | Iterable> out = mr.run(words); 153 | for (Pair pair : out) { 154 | assertEquals(ref.count(pair.getKey()), pair.getValue().intValue()); 155 | } 156 | } 157 | } 158 | -------------------------------------------------------------------------------- /src/test/resources/event2users.txt: -------------------------------------------------------------------------------- 1 | david 2 | jon 3 | sean 4 | pere -------------------------------------------------------------------------------- /src/test/resources/eventslog.txt: -------------------------------------------------------------------------------- 1 | 2010/09/09 event_1 pere 2 | 2010/09/09 event_1 david 3 | 2010/09/10 event_2 david 4 | 2010/09/11 event_2 pere 5 | 2010/09/12 event_1 frasier 6 | -------------------------------------------------------------------------------- /src/test/resources/log.txt: -------------------------------------------------------------------------------- 1 | 2010-01-01 ted message 1 2 | 2010-02-10 rahul how do messages get time-sorted 3 | 2010-02-11 ted Use an Ordering in the group-by 4 | 2010-02-12 rahul but that isn't implemented yet 5 | 2010-02-13 ted View that as an opportunity to shine! 6 | -------------------------------------------------------------------------------- /src/test/resources/simple-text.avro: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/tdunning/Plume/f802774a53c98d49b5b72f11f38a658845242934/src/test/resources/simple-text.avro -------------------------------------------------------------------------------- /src/test/resources/simple-text.txt: -------------------------------------------------------------------------------- 1 | is is 2 | some simple text 3 | To test text processing with some simple 4 | examples is some simple --------------------------------------------------------------------------------