avro-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From cutt...@apache.org
Subject svn commit: r1095206 - in /avro/trunk: CHANGES.txt lang/java/mapred/src/main/java/org/apache/avro/mapred/package.html lang/java/mapred/src/test/java/org/apache/avro/mapred/TestSequenceFileReader.java
Date Tue, 19 Apr 2011 20:44:51 GMT
Author: cutting
Date: Tue Apr 19 20:44:51 2011
New Revision: 1095206

URL: http://svn.apache.org/viewvc?rev=1095206&view=rev
Log:
AVRO-802. Java: Add documentation for non-Avro input, map-only jobs.

Modified:
    avro/trunk/CHANGES.txt
    avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/package.html
    avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/TestSequenceFileReader.java

Modified: avro/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/avro/trunk/CHANGES.txt?rev=1095206&r1=1095205&r2=1095206&view=diff
==============================================================================
--- avro/trunk/CHANGES.txt (original)
+++ avro/trunk/CHANGES.txt Tue Apr 19 20:44:51 2011
@@ -40,7 +40,11 @@ Avro 1.5.1 (unreleased)
 
     AVRO-794. Makefile.am is no longer required in C++. (thiru)
 
-    AVRO-795. C++ Datafile reader makes it hard to build adaptive clients. (thiru)
+    AVRO-795. C++ Datafile reader makes it hard to build adaptive
+    clients. (thiru)
+
+    AVRO-802. Java: Add documentation for non-Avro input, map-only
+    jobs. (cutting)
 
   BUG FIXES
 

Modified: avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/package.html
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/package.html?rev=1095206&r1=1095205&r2=1095206&view=diff
==============================================================================
--- avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/package.html (original)
+++ avro/trunk/lang/java/mapred/src/main/java/org/apache/avro/mapred/package.html Tue Apr
19 20:44:51 2011
@@ -96,5 +96,23 @@ Avro data, with map and reduce functions
  </ul>
 </p>
 
+<p>For jobs whose input is non-Avro data file and which use a
+  non-Avro {@link org.apache.hadoop.mapred.Mapper} and no reducer,
+  i.e., a <i>map-only</i> job:
+ <ul>
+   <li>Set your input file format with {@link
+   org.apache.hadoop.mapred.JobConf#setInputFormat}.</li>
+   <li>Implement {@link org.apache.hadoop.mapred.Mapper} and specify
+   your job's mapper with {@link
+   org.apache.hadoop.mapred.JobConf#setMapperClass}.  The output key
+   and value type should be {@link org.apache.avro.mapred.AvroWrapper} and
+   {@link org.apache.hadoop.io.NullWritable}.</li>
+   <li>Call {@link
+   org.apache.hadoop.mapred.JobConf#setNumReduceTasks(int)} with zero.
+   <li>Call {@link org.apache.avro.mapred.AvroJob#setOutputSchema} with your
+   job's output schema.</li>
+ </ul>
+</p>
+
 </body>
 </html>

Modified: avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/TestSequenceFileReader.java
URL: http://svn.apache.org/viewvc/avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/TestSequenceFileReader.java?rev=1095206&r1=1095205&r2=1095206&view=diff
==============================================================================
--- avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/TestSequenceFileReader.java
(original)
+++ avro/trunk/lang/java/mapred/src/test/java/org/apache/avro/mapred/TestSequenceFileReader.java
Tue Apr 19 20:44:51 2011
@@ -30,6 +30,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.NullWritable;
 import org.apache.hadoop.mapred.JobClient;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.FileInputFormat;
@@ -55,7 +56,7 @@ public class TestSequenceFileReader {
   private static final int COUNT =
     Integer.parseInt(System.getProperty("test.count", "10"));
   private static final File DIR
-    = new File(System.getProperty("test.dir", "/tmp"));
+    = new File(System.getProperty("test.dir", "."));
   private static final File FILE = new File(DIR, "test.seq");
 
   private static final Schema SCHEMA
@@ -162,6 +163,45 @@ public class TestSequenceFileReader {
                new SpecificDatumReader<Pair<Long,CharSequence>>()));
   }
 
+  private static class NonAvroOnlyMapper
+    extends MapReduceBase
+    implements Mapper<LongWritable,Text,AvroWrapper<Pair<Long,Utf8>>,NullWritable>
{
+    
+    public void map(LongWritable key, Text value, 
+                    OutputCollector<AvroWrapper<Pair<Long,Utf8>>,NullWritable>
out, 
+                    Reporter reporter) throws IOException {
+      out.collect(new AvroWrapper<Pair<Long,Utf8>>(new Pair<Long,Utf8>(key.get(),
new Utf8(value.toString()))),
+                  NullWritable.get());
+    }
+  }
+
+  @Test
+  public void testNonAvroMapOnly() throws Exception {
+    JobConf job = new JobConf();
+    Path output = new Path(System.getProperty("test.dir",".")+"/seq-out");
+
+    output.getFileSystem(job).delete(output);
+    
+
+    // configure input for non-Avro sequence file
+    job.setInputFormat(SequenceFileInputFormat.class);
+    FileInputFormat.setInputPaths(job, FILE.toURI().toString());
+
+    // use a hadoop mapper that emits Avro output
+    job.setMapperClass(NonAvroOnlyMapper.class);
+
+    // configure output for avro
+    job.setNumReduceTasks(0);                     // map-only
+    FileOutputFormat.setOutputPath(job, output);
+    AvroJob.setOutputSchema(job, SCHEMA);
+
+    JobClient.runJob(job);
+
+    checkFile(new DataFileReader<Pair<Long,CharSequence>>
+              (new File(output.toString()+"/part-00000.avro"),
+               new SpecificDatumReader<Pair<Long,CharSequence>>()));
+  }
+
   private static class NonAvroReducer
     extends MapReduceBase
     implements Reducer<AvroKey<Long>,AvroValue<Utf8>,LongWritable,Text>
{



Mime
View raw message