source: proiecte/HadoopJUnit/hadoop-0.20.1/src/test/org/apache/hadoop/mapreduce/TestMapReduceLocal.java @ 120

Last change on this file since 120 was 120, checked in by (none), 14 years ago

Added the mail files for the Hadoop JUNit Project

  • Property svn:executable set to *
File size: 6.1 KB
Line 
1/**
2 * Licensed to the Apache Software Foundation (ASF) under one
3 * or more contributor license agreements.  See the NOTICE file
4 * distributed with this work for additional information
5 * regarding copyright ownership.  The ASF licenses this file
6 * to you under the Apache License, Version 2.0 (the
7 * "License"); you may not use this file except in compliance
8 * with the License.  You may obtain a copy of the License at
9 *
10 *     http://www.apache.org/licenses/LICENSE-2.0
11 *
12 * Unless required by applicable law or agreed to in writing, software
13 * distributed under the License is distributed on an "AS IS" BASIS,
14 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15 * See the License for the specific language governing permissions and
16 * limitations under the License.
17 */
18
19package org.apache.hadoop.mapreduce;
20
21import java.io.BufferedReader;
22import java.io.DataInputStream;
23import java.io.DataOutputStream;
24import java.io.IOException;
25import java.io.InputStreamReader;
26
27import junit.framework.TestCase;
28
29import org.apache.hadoop.conf.Configuration;
30import org.apache.hadoop.examples.WordCount;
31import org.apache.hadoop.examples.WordCount.IntSumReducer;
32import org.apache.hadoop.examples.WordCount.TokenizerMapper;
33import org.apache.hadoop.fs.FileSystem;
34import org.apache.hadoop.fs.Path;
35import org.apache.hadoop.io.IntWritable;
36import org.apache.hadoop.io.LongWritable;
37import org.apache.hadoop.io.Text;
38import org.apache.hadoop.mapred.MiniMRCluster;
39import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
40import org.apache.hadoop.mapreduce.lib.input.LineRecordReader;
41import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
42import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
43
44/**
45 * A JUnit test to test min map-reduce cluster with local file system.
46 */
47public class TestMapReduceLocal extends TestCase {
48  private static Path TEST_ROOT_DIR =
49    new Path(System.getProperty("test.build.data","/tmp"));
50  private static Configuration conf = new Configuration();
51  private static FileSystem localFs;
52  static {
53    try {
54      localFs = FileSystem.getLocal(conf);
55    } catch (IOException io) {
56      throw new RuntimeException("problem getting local fs", io);
57    }
58  }
59
60  public static Path writeFile(String name, String data) throws IOException {
61    Path file = new Path(TEST_ROOT_DIR + "/" + name);
62    localFs.delete(file, false);
63    DataOutputStream f = localFs.create(file);
64    f.write(data.getBytes());
65    f.close();
66    return file;
67  }
68
69  public static String readFile(String name) throws IOException {
70    DataInputStream f = localFs.open(new Path(TEST_ROOT_DIR + "/" + name));
71    BufferedReader b = new BufferedReader(new InputStreamReader(f));
72    StringBuilder result = new StringBuilder();
73    String line = b.readLine();
74    while (line != null) {
75     result.append(line);
76     result.append('\n');
77     line = b.readLine();
78    }
79    b.close();
80    return result.toString();
81  }
82
83  public void testWithLocal() throws Exception {
84    MiniMRCluster mr = null;
85    try {
86      mr = new MiniMRCluster(2, "file:///", 3);
87      Configuration conf = mr.createJobConf();
88      runWordCount(conf);
89    } finally {
90      if (mr != null) { mr.shutdown(); }
91    }
92  }
93
94  public static class TrackingTextInputFormat extends TextInputFormat {
95
96    public static class MonoProgressRecordReader extends LineRecordReader {
97      private float last = 0.0f;
98      private boolean progressCalled = false;
99      @Override
100      public float getProgress() {
101        progressCalled = true;
102        final float ret = super.getProgress();
103        assertTrue("getProgress decreased", ret >= last);
104        last = ret;
105        return ret;
106      }
107      @Override
108      public synchronized void close() throws IOException {
109        assertTrue("getProgress never called", progressCalled);
110        super.close();
111      }
112    }
113
114    @Override
115    public RecordReader<LongWritable, Text> createRecordReader(
116        InputSplit split, TaskAttemptContext context) {
117      return new MonoProgressRecordReader();
118    }
119  }
120
121  private void runWordCount(Configuration conf
122                            ) throws IOException,
123                                     InterruptedException,
124                                     ClassNotFoundException {
125    final String COUNTER_GROUP = "org.apache.hadoop.mapred.Task$Counter";
126    localFs.delete(new Path(TEST_ROOT_DIR + "/in"), true);
127    localFs.delete(new Path(TEST_ROOT_DIR + "/out"), true);   
128    writeFile("in/part1", "this is a test\nof word count test\ntest\n");
129    writeFile("in/part2", "more test");
130    Job job = new Job(conf, "word count");     
131    job.setJarByClass(WordCount.class);
132    job.setMapperClass(TokenizerMapper.class);
133    job.setCombinerClass(IntSumReducer.class);
134    job.setReducerClass(IntSumReducer.class);
135    job.setOutputKeyClass(Text.class);
136    job.setOutputValueClass(IntWritable.class);
137    job.setInputFormatClass(TrackingTextInputFormat.class);
138    FileInputFormat.addInputPath(job, new Path(TEST_ROOT_DIR + "/in"));
139    FileOutputFormat.setOutputPath(job, new Path(TEST_ROOT_DIR + "/out"));
140    assertTrue(job.waitForCompletion(false));
141    String out = readFile("out/part-r-00000");
142    System.out.println(out);
143    assertEquals("a\t1\ncount\t1\nis\t1\nmore\t1\nof\t1\ntest\t4\nthis\t1\nword\t1\n",
144                 out);
145    Counters ctrs = job.getCounters();
146    System.out.println("Counters: " + ctrs);
147    long combineIn = ctrs.findCounter(COUNTER_GROUP,
148                                      "COMBINE_INPUT_RECORDS").getValue();
149    long combineOut = ctrs.findCounter(COUNTER_GROUP, 
150                                       "COMBINE_OUTPUT_RECORDS").getValue();
151    long reduceIn = ctrs.findCounter(COUNTER_GROUP,
152                                     "REDUCE_INPUT_RECORDS").getValue();
153    long mapOut = ctrs.findCounter(COUNTER_GROUP, 
154                                   "MAP_OUTPUT_RECORDS").getValue();
155    assertEquals("map out = combine in", mapOut, combineIn);
156    assertEquals("combine out = reduce in", combineOut, reduceIn);
157    assertTrue("combine in > combine out", combineIn > combineOut);
158    String group = "Random Group";
159    CounterGroup ctrGrp = ctrs.getGroup(group);
160    assertEquals(0, ctrGrp.size());
161  }
162
163}
Note: See TracBrowser for help on using the repository browser.