source: proiecte/HadoopJUnit/hadoop-0.20.1/src/test/org/apache/hadoop/mapred/lib/TestKeyFieldBasedComparator.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: 5.6 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.mapred.lib;
20
21import java.io.*;
22
23import org.apache.hadoop.fs.FileSystem;
24import org.apache.hadoop.fs.FileUtil;
25import org.apache.hadoop.fs.Path;
26import org.apache.hadoop.io.LongWritable;
27import org.apache.hadoop.io.Text;
28import org.apache.hadoop.mapred.FileInputFormat;
29import org.apache.hadoop.mapred.FileOutputFormat;
30import org.apache.hadoop.mapred.HadoopTestCase;
31import org.apache.hadoop.mapred.JobClient;
32import org.apache.hadoop.mapred.JobConf;
33import org.apache.hadoop.mapred.OutputLogFilter;
34import org.apache.hadoop.mapred.RunningJob;
35import org.apache.hadoop.mapred.TextInputFormat;
36import org.apache.hadoop.mapred.TextOutputFormat;
37
38
39public class TestKeyFieldBasedComparator extends HadoopTestCase {
40  JobConf conf;
41  JobConf localConf;
42 
43  String line1 = "123 -123 005120 123.9 0.01 0.18 010 10.0 4444.1 011 011 234";
44  String line2 = "134 -12 005100 123.10 -1.01 0.19 02 10.1 4444";
45
46  public TestKeyFieldBasedComparator() throws IOException {
47    super(HadoopTestCase.LOCAL_MR, HadoopTestCase.LOCAL_FS, 1, 1);
48    conf = createJobConf();
49    localConf = createJobConf();
50    localConf.set("map.output.key.field.separator", " ");
51  }
52  public void configure(String keySpec, int expect) throws Exception {
53    Path testdir = new Path("build/test/test.mapred.spill");
54    Path inDir = new Path(testdir, "in");
55    Path outDir = new Path(testdir, "out");
56    FileSystem fs = getFileSystem();
57    fs.delete(testdir, true);
58    conf.setInputFormat(TextInputFormat.class);
59    FileInputFormat.setInputPaths(conf, inDir);
60    FileOutputFormat.setOutputPath(conf, outDir);
61    conf.setOutputKeyClass(Text.class);
62    conf.setOutputValueClass(LongWritable.class);
63
64    conf.setNumMapTasks(1);
65    conf.setNumReduceTasks(2);
66
67    conf.setOutputFormat(TextOutputFormat.class);
68    conf.setOutputKeyComparatorClass(KeyFieldBasedComparator.class);
69    conf.setKeyFieldComparatorOptions(keySpec);
70    conf.setKeyFieldPartitionerOptions("-k1.1,1.1");
71    conf.set("map.output.key.field.separator", " ");
72    conf.setMapperClass(InverseMapper.class);
73    conf.setReducerClass(IdentityReducer.class);
74    if (!fs.mkdirs(testdir)) {
75      throw new IOException("Mkdirs failed to create " + testdir.toString());
76    }
77    if (!fs.mkdirs(inDir)) {
78      throw new IOException("Mkdirs failed to create " + inDir.toString());
79    }
80    // set up input data in 2 files
81    Path inFile = new Path(inDir, "part0");
82    FileOutputStream fos = new FileOutputStream(inFile.toString());
83    fos.write((line1 + "\n").getBytes());
84    fos.write((line2 + "\n").getBytes());
85    fos.close();
86    JobClient jc = new JobClient(conf);
87    RunningJob r_job = jc.submitJob(conf);
88    while (!r_job.isComplete()) {
89      Thread.sleep(1000);
90    }
91   
92    if (!r_job.isSuccessful()) {
93      fail("Oops! The job broke due to an unexpected error");
94    }
95    Path[] outputFiles = FileUtil.stat2Paths(
96        getFileSystem().listStatus(outDir,
97        new OutputLogFilter()));
98    if (outputFiles.length > 0) {
99      InputStream is = getFileSystem().open(outputFiles[0]);
100      BufferedReader reader = new BufferedReader(new InputStreamReader(is));
101      String line = reader.readLine();
102      //make sure we get what we expect as the first line, and also
103      //that we have two lines (both the lines must end up in the same
104      //reducer since the partitioner takes the same key spec for all
105      //lines
106      if (expect == 1) {
107        assertTrue(line.startsWith(line1));
108      } else if (expect == 2) {
109        assertTrue(line.startsWith(line2));
110      }
111      line = reader.readLine();
112      if (expect == 1) {
113        assertTrue(line.startsWith(line2));
114      } else if (expect == 2) {
115        assertTrue(line.startsWith(line1));
116      }
117      reader.close();
118    }
119  }
120  public void testBasicUnixComparator() throws Exception {
121    configure("-k1,1n", 1);
122    configure("-k2,2n", 1);
123    configure("-k2.2,2n", 2);
124    configure("-k3.4,3n", 2);
125    configure("-k3.2,3.3n -k4,4n", 2);
126    configure("-k3.2,3.3n -k4,4nr", 1);
127    configure("-k2.4,2.4n", 2);
128    configure("-k7,7", 1);
129    configure("-k7,7n", 2);
130    configure("-k8,8n", 1);
131    configure("-k9,9", 2);
132    configure("-k11,11",2);
133    configure("-k10,10",2);
134   
135    localTestWithoutMRJob("-k9,9", 1);
136
137    localTestWithoutMRJob("-k9n", 1);
138  }
139 
140  byte[] line1_bytes = line1.getBytes();
141  byte[] line2_bytes = line2.getBytes();
142
143  public void localTestWithoutMRJob(String keySpec, int expect) throws Exception {
144    KeyFieldBasedComparator<Void, Void> keyFieldCmp = new KeyFieldBasedComparator<Void, Void>();
145    localConf.setKeyFieldComparatorOptions(keySpec);
146    keyFieldCmp.configure(localConf);
147    int result = keyFieldCmp.compare(line1_bytes, 0, line1_bytes.length,
148        line2_bytes, 0, line2_bytes.length);
149    if ((expect >= 0 && result < 0) || (expect < 0 && result >= 0))
150      fail();
151  }
152}
Note: See TracBrowser for help on using the repository browser.