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 | package org.apache.hadoop.mapred.lib; |
---|
19 | |
---|
20 | import java.io.DataOutputStream; |
---|
21 | import java.io.IOException; |
---|
22 | |
---|
23 | import junit.framework.TestCase; |
---|
24 | |
---|
25 | import org.apache.hadoop.fs.FileSystem; |
---|
26 | import org.apache.hadoop.fs.Path; |
---|
27 | import org.apache.hadoop.hdfs.MiniDFSCluster; |
---|
28 | import org.apache.hadoop.mapred.InputSplit; |
---|
29 | import org.apache.hadoop.mapred.JobConf; |
---|
30 | import org.apache.hadoop.mapred.KeyValueTextInputFormat; |
---|
31 | import org.apache.hadoop.mapred.Mapper; |
---|
32 | import org.apache.hadoop.mapred.OutputCollector; |
---|
33 | import org.apache.hadoop.mapred.Reporter; |
---|
34 | import org.apache.hadoop.mapred.TextInputFormat; |
---|
35 | |
---|
36 | public class TestDelegatingInputFormat extends TestCase { |
---|
37 | |
---|
38 | public void testSplitting() throws Exception { |
---|
39 | JobConf conf = new JobConf(); |
---|
40 | conf.set("fs.hdfs.impl", |
---|
41 | "org.apache.hadoop.hdfs.ChecksumDistributedFileSystem"); |
---|
42 | MiniDFSCluster dfs = null; |
---|
43 | try { |
---|
44 | dfs = new MiniDFSCluster(conf, 4, true, new String[] { "/rack0", |
---|
45 | "/rack0", "/rack1", "/rack1" }, new String[] { "host0", "host1", |
---|
46 | "host2", "host3" }); |
---|
47 | FileSystem fs = dfs.getFileSystem(); |
---|
48 | |
---|
49 | Path path = getPath("/foo/bar", fs); |
---|
50 | Path path2 = getPath("/foo/baz", fs); |
---|
51 | Path path3 = getPath("/bar/bar", fs); |
---|
52 | Path path4 = getPath("/bar/baz", fs); |
---|
53 | |
---|
54 | final int numSplits = 100; |
---|
55 | |
---|
56 | MultipleInputs.addInputPath(conf, path, TextInputFormat.class, |
---|
57 | MapClass.class); |
---|
58 | MultipleInputs.addInputPath(conf, path2, TextInputFormat.class, |
---|
59 | MapClass2.class); |
---|
60 | MultipleInputs.addInputPath(conf, path3, KeyValueTextInputFormat.class, |
---|
61 | MapClass.class); |
---|
62 | MultipleInputs.addInputPath(conf, path4, TextInputFormat.class, |
---|
63 | MapClass2.class); |
---|
64 | DelegatingInputFormat inFormat = new DelegatingInputFormat(); |
---|
65 | InputSplit[] splits = inFormat.getSplits(conf, numSplits); |
---|
66 | |
---|
67 | int[] bins = new int[3]; |
---|
68 | for (InputSplit split : splits) { |
---|
69 | assertTrue(split instanceof TaggedInputSplit); |
---|
70 | final TaggedInputSplit tis = (TaggedInputSplit) split; |
---|
71 | int index = -1; |
---|
72 | |
---|
73 | if (tis.getInputFormatClass().equals(KeyValueTextInputFormat.class)) { |
---|
74 | // path3 |
---|
75 | index = 0; |
---|
76 | } else if (tis.getMapperClass().equals(MapClass.class)) { |
---|
77 | // path |
---|
78 | index = 1; |
---|
79 | } else { |
---|
80 | // path2 and path4 |
---|
81 | index = 2; |
---|
82 | } |
---|
83 | |
---|
84 | bins[index]++; |
---|
85 | } |
---|
86 | |
---|
87 | // Each bin is a unique combination of a Mapper and InputFormat, and |
---|
88 | // DelegatingInputFormat should split each bin into numSplits splits, |
---|
89 | // regardless of the number of paths that use that Mapper/InputFormat |
---|
90 | for (int count : bins) { |
---|
91 | assertEquals(numSplits, count); |
---|
92 | } |
---|
93 | |
---|
94 | assertTrue(true); |
---|
95 | } finally { |
---|
96 | if (dfs != null) { |
---|
97 | dfs.shutdown(); |
---|
98 | } |
---|
99 | } |
---|
100 | } |
---|
101 | |
---|
102 | static Path getPath(final String location, final FileSystem fs) |
---|
103 | throws IOException { |
---|
104 | Path path = new Path(location); |
---|
105 | |
---|
106 | // create a multi-block file on hdfs |
---|
107 | DataOutputStream out = fs.create(path, true, 4096, (short) 2, 512, null); |
---|
108 | for (int i = 0; i < 1000; ++i) { |
---|
109 | out.writeChars("Hello\n"); |
---|
110 | } |
---|
111 | out.close(); |
---|
112 | |
---|
113 | return path; |
---|
114 | } |
---|
115 | |
---|
116 | static class MapClass implements Mapper<String, String, String, String> { |
---|
117 | |
---|
118 | public void map(String key, String value, |
---|
119 | OutputCollector<String, String> output, Reporter reporter) |
---|
120 | throws IOException { |
---|
121 | } |
---|
122 | |
---|
123 | public void configure(JobConf job) { |
---|
124 | } |
---|
125 | |
---|
126 | public void close() throws IOException { |
---|
127 | } |
---|
128 | } |
---|
129 | |
---|
130 | static class MapClass2 extends MapClass { |
---|
131 | } |
---|
132 | |
---|
133 | } |
---|