[120] | 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; |
---|
| 19 | |
---|
| 20 | import java.io.IOException; |
---|
| 21 | |
---|
| 22 | import junit.framework.TestCase; |
---|
| 23 | |
---|
| 24 | import org.apache.hadoop.conf.Configuration; |
---|
| 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.io.BytesWritable; |
---|
| 29 | import org.apache.hadoop.mapred.SortValidator.RecordStatsChecker.NonSplitableSequenceFileInputFormat; |
---|
| 30 | import org.apache.hadoop.mapred.lib.IdentityMapper; |
---|
| 31 | import org.apache.hadoop.mapred.lib.IdentityReducer; |
---|
| 32 | |
---|
| 33 | public class TestRackAwareTaskPlacement extends TestCase { |
---|
| 34 | private static final String rack1[] = new String[] { |
---|
| 35 | "/r1" |
---|
| 36 | }; |
---|
| 37 | private static final String hosts1[] = new String[] { |
---|
| 38 | "host1.rack1.com" |
---|
| 39 | }; |
---|
| 40 | private static final String rack2[] = new String[] { |
---|
| 41 | "/r2", "/r2" |
---|
| 42 | }; |
---|
| 43 | private static final String hosts2[] = new String[] { |
---|
| 44 | "host1.rack2.com", "host2.rack2.com" |
---|
| 45 | }; |
---|
| 46 | private static final String hosts3[] = new String[] { |
---|
| 47 | "host3.rack1.com" |
---|
| 48 | }; |
---|
| 49 | private static final String hosts4[] = new String[] { |
---|
| 50 | "host1.rack2.com" |
---|
| 51 | }; |
---|
| 52 | final Path inDir = new Path("/racktesting"); |
---|
| 53 | final Path outputPath = new Path("/output"); |
---|
| 54 | |
---|
| 55 | /** |
---|
| 56 | * Launches a MR job and tests the job counters against the expected values. |
---|
| 57 | * @param testName The name for the job |
---|
| 58 | * @param mr The MR cluster |
---|
| 59 | * @param fileSys The FileSystem |
---|
| 60 | * @param in Input path |
---|
| 61 | * @param out Output path |
---|
| 62 | * @param numMaps Number of maps |
---|
| 63 | * @param otherLocalMaps Expected value of other local maps |
---|
| 64 | * @param datalocalMaps Expected value of data(node) local maps |
---|
| 65 | * @param racklocalMaps Expected value of rack local maps |
---|
| 66 | */ |
---|
| 67 | static void launchJobAndTestCounters(String jobName, MiniMRCluster mr, |
---|
| 68 | FileSystem fileSys, Path in, Path out, |
---|
| 69 | int numMaps, int otherLocalMaps, |
---|
| 70 | int dataLocalMaps, int rackLocalMaps) |
---|
| 71 | throws IOException { |
---|
| 72 | JobConf jobConf = mr.createJobConf(); |
---|
| 73 | if (fileSys.exists(out)) { |
---|
| 74 | fileSys.delete(out, true); |
---|
| 75 | } |
---|
| 76 | RunningJob job = launchJob(jobConf, in, out, numMaps, jobName); |
---|
| 77 | Counters counters = job.getCounters(); |
---|
| 78 | assertEquals("Number of local maps", |
---|
| 79 | counters.getCounter(JobInProgress.Counter.OTHER_LOCAL_MAPS), otherLocalMaps); |
---|
| 80 | assertEquals("Number of Data-local maps", |
---|
| 81 | counters.getCounter(JobInProgress.Counter.DATA_LOCAL_MAPS), |
---|
| 82 | dataLocalMaps); |
---|
| 83 | assertEquals("Number of Rack-local maps", |
---|
| 84 | counters.getCounter(JobInProgress.Counter.RACK_LOCAL_MAPS), |
---|
| 85 | rackLocalMaps); |
---|
| 86 | mr.waitUntilIdle(); |
---|
| 87 | mr.shutdown(); |
---|
| 88 | } |
---|
| 89 | |
---|
| 90 | public void testTaskPlacement() throws IOException { |
---|
| 91 | String namenode = null; |
---|
| 92 | MiniDFSCluster dfs = null; |
---|
| 93 | MiniMRCluster mr = null; |
---|
| 94 | FileSystem fileSys = null; |
---|
| 95 | String testName = "TestForRackAwareness"; |
---|
| 96 | try { |
---|
| 97 | final int taskTrackers = 1; |
---|
| 98 | |
---|
| 99 | /* Start 3 datanodes, one in rack r1, and two in r2. Create three |
---|
| 100 | * files (splits). |
---|
| 101 | * 1) file1, just after starting the datanode on r1, with |
---|
| 102 | * a repl factor of 1, and, |
---|
| 103 | * 2) file2 & file3 after starting the other two datanodes, with a repl |
---|
| 104 | * factor of 3. |
---|
| 105 | * At the end, file1 will be present on only datanode1, and, file2 and |
---|
| 106 | * file3, will be present on all datanodes. |
---|
| 107 | */ |
---|
| 108 | Configuration conf = new Configuration(); |
---|
| 109 | conf.setBoolean("dfs.replication.considerLoad", false); |
---|
| 110 | dfs = new MiniDFSCluster(conf, 1, true, rack1, hosts1); |
---|
| 111 | dfs.waitActive(); |
---|
| 112 | fileSys = dfs.getFileSystem(); |
---|
| 113 | if (!fileSys.mkdirs(inDir)) { |
---|
| 114 | throw new IOException("Mkdirs failed to create " + inDir.toString()); |
---|
| 115 | } |
---|
| 116 | UtilsForTests.writeFile(dfs.getNameNode(), conf, new Path(inDir + "/file1"), (short)1); |
---|
| 117 | dfs.startDataNodes(conf, 2, true, null, rack2, hosts2, null); |
---|
| 118 | dfs.waitActive(); |
---|
| 119 | |
---|
| 120 | UtilsForTests.writeFile(dfs.getNameNode(), conf, new Path(inDir + "/file2"), (short)3); |
---|
| 121 | UtilsForTests.writeFile(dfs.getNameNode(), conf, new Path(inDir + "/file3"), (short)3); |
---|
| 122 | |
---|
| 123 | namenode = (dfs.getFileSystem()).getUri().getHost() + ":" + |
---|
| 124 | (dfs.getFileSystem()).getUri().getPort(); |
---|
| 125 | /* Run a job with the (only)tasktracker on rack2. The rack location |
---|
| 126 | * of the tasktracker will determine how many data/rack local maps it |
---|
| 127 | * runs. The hostname of the tasktracker is set to same as one of the |
---|
| 128 | * datanodes. |
---|
| 129 | */ |
---|
| 130 | mr = new MiniMRCluster(taskTrackers, namenode, 1, rack2, hosts4); |
---|
| 131 | |
---|
| 132 | /* The job is configured with three maps since there are three |
---|
| 133 | * (non-splittable) files. On rack2, there are two files and both |
---|
| 134 | * have repl of three. The blocks for those files must therefore be |
---|
| 135 | * present on all the datanodes, in particular, the datanodes on rack2. |
---|
| 136 | * The third input file is pulled from rack1. |
---|
| 137 | */ |
---|
| 138 | launchJobAndTestCounters(testName, mr, fileSys, inDir, outputPath, 3, 0, |
---|
| 139 | 2, 0); |
---|
| 140 | mr.shutdown(); |
---|
| 141 | |
---|
| 142 | /* Run a job with the (only)tasktracker on rack1. |
---|
| 143 | */ |
---|
| 144 | mr = new MiniMRCluster(taskTrackers, namenode, 1, rack1, hosts3); |
---|
| 145 | |
---|
| 146 | /* The job is configured with three maps since there are three |
---|
| 147 | * (non-splittable) files. On rack1, because of the way in which repl |
---|
| 148 | * was setup while creating the files, we will have all the three files. |
---|
| 149 | * Thus, a tasktracker will find all inputs in this rack. |
---|
| 150 | */ |
---|
| 151 | launchJobAndTestCounters(testName, mr, fileSys, inDir, outputPath, 3, 0, |
---|
| 152 | 0, 3); |
---|
| 153 | mr.shutdown(); |
---|
| 154 | |
---|
| 155 | } finally { |
---|
| 156 | if (dfs != null) { |
---|
| 157 | dfs.shutdown(); |
---|
| 158 | } |
---|
| 159 | if (mr != null) { |
---|
| 160 | mr.shutdown(); |
---|
| 161 | } |
---|
| 162 | } |
---|
| 163 | } |
---|
| 164 | static RunningJob launchJob(JobConf jobConf, Path inDir, Path outputPath, |
---|
| 165 | int numMaps, String jobName) throws IOException { |
---|
| 166 | jobConf.setJobName(jobName); |
---|
| 167 | jobConf.setInputFormat(NonSplitableSequenceFileInputFormat.class); |
---|
| 168 | jobConf.setOutputFormat(SequenceFileOutputFormat.class); |
---|
| 169 | FileInputFormat.setInputPaths(jobConf, inDir); |
---|
| 170 | FileOutputFormat.setOutputPath(jobConf, outputPath); |
---|
| 171 | jobConf.setMapperClass(IdentityMapper.class); |
---|
| 172 | jobConf.setReducerClass(IdentityReducer.class); |
---|
| 173 | jobConf.setOutputKeyClass(BytesWritable.class); |
---|
| 174 | jobConf.setOutputValueClass(BytesWritable.class); |
---|
| 175 | jobConf.setNumMapTasks(numMaps); |
---|
| 176 | jobConf.setNumReduceTasks(0); |
---|
| 177 | jobConf.setJar("build/test/testjar/testjob.jar"); |
---|
| 178 | return JobClient.runJob(jobConf); |
---|
| 179 | } |
---|
| 180 | } |
---|