Author: cutting Date: Fri Jun 15 15:02:45 2007 New Revision: 547811 URL: http://svn.apache.org/viewvc?view=rev&rev=547811 Log: HADOOP-1448. Add some examples to contrib/data_join. Contributed by Senthil.
Added: lucene/hadoop/trunk/src/contrib/data_join/src/examples/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinMapper.java lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinReducer.java lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleTaggedMapOutput.java Modified: lucene/hadoop/trunk/CHANGES.txt lucene/hadoop/trunk/src/contrib/build-contrib.xml lucene/hadoop/trunk/src/contrib/data_join/build.xml Modified: lucene/hadoop/trunk/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/CHANGES.txt?view=diff&rev=547811&r1=547810&r2=547811 ============================================================================== --- lucene/hadoop/trunk/CHANGES.txt (original) +++ lucene/hadoop/trunk/CHANGES.txt Fri Jun 15 15:02:45 2007 @@ -150,6 +150,9 @@ 47. HADOOP-1320. Rewrite RandomWriter example to bypass reduce. (Arun C Murthy via cutting) + 48. HADOOP-1449. Add some examples to contrib/data_join. + (Senthil Subramanian via cutting) + Release 0.13.0 - 2007-06-08 Modified: lucene/hadoop/trunk/src/contrib/build-contrib.xml URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/build-contrib.xml?view=diff&rev=547811&r1=547810&r2=547811 ============================================================================== --- lucene/hadoop/trunk/src/contrib/build-contrib.xml (original) +++ lucene/hadoop/trunk/src/contrib/build-contrib.xml Fri Jun 15 15:02:45 2007 @@ -16,7 +16,9 @@ <property name="src.dir" location="${root}/src/java"/> <property name="src.test" location="${root}/src/test"/> + <property name="src.examples" location="${root}/src/examples"/> + <available file="${src.examples}" type="dir" property="examples.available"/> <available file="${src.test}" type="dir" property="test.available"/> <property name="conf.dir" location="${hadoop.root}/conf"/> @@ -26,6 +28,7 @@ <property name="build.dir" location="${hadoop.root}/build/contrib/${name}"/> <property name="build.classes" location="${build.dir}/classes"/> <property name="build.test" location="${build.dir}/test"/> + <property name="build.examples" location="${build.dir}/examples"/> <property name="hadoop.log.dir" location="${build.dir}/test/logs"/> <!-- all jars together --> <property name="deploy.dir" location="${hadoop.root}/build/"/> @@ -71,6 +74,7 @@ <mkdir dir="${build.dir}"/> <mkdir dir="${build.classes}"/> <mkdir dir="${build.test}"/> + <mkdir dir="${build.examples}"/> <mkdir dir="${hadoop.log.dir}"/> <antcall target="init-contrib"/> </target> @@ -92,6 +96,23 @@ </javac> </target> + + <!-- ======================================================= --> + <!-- Compile a Hadoop contrib's example files (if available) --> + <!-- ======================================================= --> + <target name="compile-examples" depends="compile" if="examples.available"> + <echo message="contrib: ${name}"/> + <javac + encoding="${build.encoding}" + srcdir="${src.examples}" + includes="**/*.java" + destdir="${build.examples}" + debug="${javac.debug}"> + <classpath refid="classpath"/> + </javac> + </target> + + <!-- ================================================================== --> <!-- Compile test code --> <!-- ================================================================== --> @@ -118,8 +139,22 @@ basedir="${build.classes}" /> </target> + + + <!-- ====================================================== --> + <!-- Make a Hadoop contrib's examples jar --> + <!-- ====================================================== --> + <target name="jar-examples" depends="compile-examples"> + <echo message="contrib: ${name}"/> + <jar jarfile="${build.dir}/hadoop-${name}-examples.jar"> + <fileset dir="${build.classes}"> + </fileset> + <fileset dir="${build.examples}"> + </fileset> + </jar> + </target> - <target name="deploy" depends="jar"> + <target name="deploy" depends="jar, jar-examples"> <echo message="contrib: ${name}"/> <mkdir dir="${deploy.dir}"/> <copy file="${build.dir}/hadoop-${name}.jar" todir="${deploy.dir}"/> Modified: lucene/hadoop/trunk/src/contrib/data_join/build.xml URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/data_join/build.xml?view=diff&rev=547811&r1=547810&r2=547811 ============================================================================== --- lucene/hadoop/trunk/src/contrib/data_join/build.xml (original) +++ lucene/hadoop/trunk/src/contrib/data_join/build.xml Fri Jun 15 15:02:45 2007 @@ -20,4 +20,9 @@ </jar> </target> + <target name="jar-examples" depends="jar"> + <antcall target="hadoopbuildcontrib.jar-examples"> + </antcall> + </target> + </project> Added: lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt?view=auto&rev=547811 ============================================================================== --- lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt (added) +++ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/README.txt Fri Jun 15 15:02:45 2007 @@ -0,0 +1,50 @@ +************************************************* +*** Input Files (Note: tab-separated columns) *** +************************************************* +[:~]$ cat datajoin/input/A +A.a11 A.a12 +A.a21 A.a22 +B.a21 A.a32 +A.a31 A.a32 +B.a31 A.a32 + +[:~]$ cat datajoin/input/B +A.a11 B.a12 +A.a11 B.a13 +B.a11 B.a12 +B.a21 B.a22 +A.a31 B.a32 +B.a31 B.a32 + + +***************************** +*** Invoke SampleDataJoin *** +***************************** +[:~]$ $HADOOP_HOME/bin/hadoop jar hadoop-datajoin-examples.jar org.apache.hadoop.contrib.utils.join.DataJoinJob datajoin/input datajoin/output Text 1 org.apache.hadoop.contrib.utils.join.SampleDataJoinMapper org.apache.hadoop.contrib.utils.join.SampleDataJoinReducer org.apache.hadoop.contrib.utils.join.SampleTaggedMapOutput Text +Using TextInputFormat: Text +Using TextOutputFormat: Text +07/06/01 19:58:23 INFO mapred.InputFormatBase: Total input paths to process : 2 +Job job_kkzk08 is submitted +Job job_kkzk08 is still running. +07/06/01 19:58:24 INFO mapred.LocalJobRunner: collectedCount 5 +totalCount 5 + +07/06/01 19:58:24 INFO mapred.LocalJobRunner: collectedCount 6 +totalCount 6 + +07/06/01 19:58:24 INFO datajoin.job: key: A.a11 this.largestNumOfValues: 3 +07/06/01 19:58:24 INFO mapred.LocalJobRunner: actuallyCollectedCount 5 +collectedCount 7 +groupCount 6 + > reduce + + +******************* +*** Output File *** +******************* +[:~]$ cat datajoin/output/part-00000 +A.a11 A.a12 B.a12 +A.a11 A.a12 B.a13 +A.a31 A.a32 B.a32 +B.a21 A.a32 B.a22 +B.a31 A.a32 B.a32 Added: lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinMapper.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinMapper.java?view=auto&rev=547811 ============================================================================== --- lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinMapper.java (added) +++ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinMapper.java Fri Jun 15 15:02:45 2007 @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.contrib.utils.join; + +import org.apache.hadoop.io.Text; +import org.apache.hadoop.io.Writable; + +import org.apache.hadoop.contrib.utils.join.DataJoinMapperBase; +import org.apache.hadoop.contrib.utils.join.TaggedMapOutput; +import org.apache.hadoop.contrib.utils.join.SampleTaggedMapOutput; + +/** + * This is a subclass of DataJoinMapperBase that is used to + * demonstrate the functionality of INNER JOIN between 2 data + * sources (TAB separated text files) based on the first column. + */ +public class SampleDataJoinMapper extends DataJoinMapperBase { + + + protected Text generateInputTag(String inputFile) { + // tag the row with input file name (data source) + return new Text(inputFile); + } + + protected Text generateGroupKey(TaggedMapOutput aRecord) { + // first column in the input tab separated files becomes the key (to perform the JOIN) + String line = ((Text) aRecord.getData()).toString(); + String groupKey = ""; + String[] tokens = line.split("\\t", 2); + groupKey = tokens[0]; + return new Text(groupKey); + } + + protected TaggedMapOutput generateTaggedMapOutput(Writable value) { + TaggedMapOutput retv = new SampleTaggedMapOutput((Text) value); + retv.setTag(new Text(this.inputTag)); + return retv; + } +} Added: lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinReducer.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinReducer.java?view=auto&rev=547811 ============================================================================== --- lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinReducer.java (added) +++ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleDataJoinReducer.java Fri Jun 15 15:02:45 2007 @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.contrib.utils.join; + +import org.apache.hadoop.io.Text; + +import org.apache.hadoop.contrib.utils.join.DataJoinReducerBase; +import org.apache.hadoop.contrib.utils.join.TaggedMapOutput; + +/** + * This is a subclass of DataJoinReducerBase that is used to + * demonstrate the functionality of INNER JOIN between 2 data + * sources (TAB separated text files) based on the first column. + */ +public class SampleDataJoinReducer extends DataJoinReducerBase { + + /** + * + * @param tags + * a list of source tags + * @param values + * a value per source + * @return combined value derived from values of the sources + */ + protected TaggedMapOutput combine(Object[] tags, Object[] values) { + // eliminate rows which didnot match in one of the two tables (for INNER JOIN) + if (tags.length < 2) + return null; + String joinedStr = ""; + for (int i=0; i<tags.length; i++) { + if (i > 0) + joinedStr += "\t"; + // strip first column as it is the key on which we joined + String line = ((Text) (((TaggedMapOutput) values[i]).getData())).toString(); + String[] tokens = line.split("\\t", 2); + joinedStr += tokens[1]; + } + TaggedMapOutput retv = new SampleTaggedMapOutput(new Text(joinedStr)); + retv.setTag((Text) tags[0]); + return retv; + } +} Added: lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleTaggedMapOutput.java URL: http://svn.apache.org/viewvc/lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleTaggedMapOutput.java?view=auto&rev=547811 ============================================================================== --- lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleTaggedMapOutput.java (added) +++ lucene/hadoop/trunk/src/contrib/data_join/src/examples/org/apache/hadoop/contrib/utils/join/SampleTaggedMapOutput.java Fri Jun 15 15:02:45 2007 @@ -0,0 +1,60 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.contrib.utils.join; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +import org.apache.hadoop.io.Writable; +import org.apache.hadoop.io.Text; + +import org.apache.hadoop.contrib.utils.join.TaggedMapOutput; + +/** + * This is a subclass of TaggedMapOutput that is used to + * demonstrate the functionality of INNER JOIN between 2 data + * sources (TAB separated text files) based on the first column. + */ +public class SampleTaggedMapOutput extends TaggedMapOutput { + + private Text data; + + public SampleTaggedMapOutput() { + this.data = new Text(""); + } + + public SampleTaggedMapOutput(Text data) { + this.data = data; + } + + public Writable getData() { + return data; + } + + public void write(DataOutput out) throws IOException { + this.tag.write(out); + this.data.write(out); + } + + public void readFields(DataInput in) throws IOException { + this.tag.readFields(in); + this.data.readFields(in); + } +}