[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 | |
---|
| 19 | package org.apache.hadoop.mapreduce.lib.map; |
---|
| 20 | |
---|
| 21 | import org.apache.hadoop.util.ReflectionUtils; |
---|
| 22 | import org.apache.hadoop.conf.Configuration; |
---|
| 23 | import org.apache.hadoop.mapreduce.Counter; |
---|
| 24 | import org.apache.hadoop.mapreduce.InputSplit; |
---|
| 25 | import org.apache.hadoop.mapreduce.Job; |
---|
| 26 | import org.apache.hadoop.mapreduce.JobContext; |
---|
| 27 | import org.apache.hadoop.mapreduce.Mapper; |
---|
| 28 | import org.apache.hadoop.mapreduce.RecordReader; |
---|
| 29 | import org.apache.hadoop.mapreduce.RecordWriter; |
---|
| 30 | import org.apache.hadoop.mapreduce.StatusReporter; |
---|
| 31 | import org.apache.hadoop.mapreduce.TaskAttemptContext; |
---|
| 32 | import org.apache.commons.logging.Log; |
---|
| 33 | import org.apache.commons.logging.LogFactory; |
---|
| 34 | |
---|
| 35 | import java.io.IOException; |
---|
| 36 | import java.util.ArrayList; |
---|
| 37 | import java.util.List; |
---|
| 38 | |
---|
| 39 | /** |
---|
| 40 | * Multithreaded implementation for @link org.apache.hadoop.mapreduce.Mapper. |
---|
| 41 | * <p> |
---|
| 42 | * It can be used instead of the default implementation, |
---|
| 43 | * @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU |
---|
| 44 | * bound in order to improve throughput. |
---|
| 45 | * <p> |
---|
| 46 | * Mapper implementations using this MapRunnable must be thread-safe. |
---|
| 47 | * <p> |
---|
| 48 | * The Map-Reduce job has to be configured with the mapper to use via |
---|
| 49 | * {@link #setMapperClass(Configuration, Class)} and |
---|
| 50 | * the number of thread the thread-pool can use with the |
---|
| 51 | * {@link #getNumberOfThreads(Configuration) method. The default |
---|
| 52 | * value is 10 threads. |
---|
| 53 | * <p> |
---|
| 54 | */ |
---|
| 55 | public class MultithreadedMapper<K1, V1, K2, V2> |
---|
| 56 | extends Mapper<K1, V1, K2, V2> { |
---|
| 57 | |
---|
| 58 | private static final Log LOG = LogFactory.getLog(MultithreadedMapper.class); |
---|
| 59 | private Class<? extends Mapper<K1,V1,K2,V2>> mapClass; |
---|
| 60 | private Context outer; |
---|
| 61 | private List<MapRunner> runners; |
---|
| 62 | |
---|
| 63 | /** |
---|
| 64 | * The number of threads in the thread pool that will run the map function. |
---|
| 65 | * @param job the job |
---|
| 66 | * @return the number of threads |
---|
| 67 | */ |
---|
| 68 | public static int getNumberOfThreads(JobContext job) { |
---|
| 69 | return job.getConfiguration(). |
---|
| 70 | getInt("mapred.map.multithreadedrunner.threads", 10); |
---|
| 71 | } |
---|
| 72 | |
---|
| 73 | /** |
---|
| 74 | * Set the number of threads in the pool for running maps. |
---|
| 75 | * @param job the job to modify |
---|
| 76 | * @param threads the new number of threads |
---|
| 77 | */ |
---|
| 78 | public static void setNumberOfThreads(Job job, int threads) { |
---|
| 79 | job.getConfiguration().setInt("mapred.map.multithreadedrunner.threads", |
---|
| 80 | threads); |
---|
| 81 | } |
---|
| 82 | |
---|
| 83 | /** |
---|
| 84 | * Get the application's mapper class. |
---|
| 85 | * @param <K1> the map's input key type |
---|
| 86 | * @param <V1> the map's input value type |
---|
| 87 | * @param <K2> the map's output key type |
---|
| 88 | * @param <V2> the map's output value type |
---|
| 89 | * @param job the job |
---|
| 90 | * @return the mapper class to run |
---|
| 91 | */ |
---|
| 92 | @SuppressWarnings("unchecked") |
---|
| 93 | public static <K1,V1,K2,V2> |
---|
| 94 | Class<Mapper<K1,V1,K2,V2>> getMapperClass(JobContext job) { |
---|
| 95 | return (Class<Mapper<K1,V1,K2,V2>>) |
---|
| 96 | job.getConfiguration().getClass("mapred.map.multithreadedrunner.class", |
---|
| 97 | Mapper.class); |
---|
| 98 | } |
---|
| 99 | |
---|
| 100 | /** |
---|
| 101 | * Set the application's mapper class. |
---|
| 102 | * @param <K1> the map input key type |
---|
| 103 | * @param <V1> the map input value type |
---|
| 104 | * @param <K2> the map output key type |
---|
| 105 | * @param <V2> the map output value type |
---|
| 106 | * @param job the job to modify |
---|
| 107 | * @param cls the class to use as the mapper |
---|
| 108 | */ |
---|
| 109 | public static <K1,V1,K2,V2> |
---|
| 110 | void setMapperClass(Job job, |
---|
| 111 | Class<? extends Mapper<K1,V1,K2,V2>> cls) { |
---|
| 112 | if (MultithreadedMapper.class.isAssignableFrom(cls)) { |
---|
| 113 | throw new IllegalArgumentException("Can't have recursive " + |
---|
| 114 | "MultithreadedMapper instances."); |
---|
| 115 | } |
---|
| 116 | job.getConfiguration().setClass("mapred.map.multithreadedrunner.class", |
---|
| 117 | cls, Mapper.class); |
---|
| 118 | } |
---|
| 119 | |
---|
| 120 | /** |
---|
| 121 | * Run the application's maps using a thread pool. |
---|
| 122 | */ |
---|
| 123 | @Override |
---|
| 124 | public void run(Context context) throws IOException, InterruptedException { |
---|
| 125 | outer = context; |
---|
| 126 | int numberOfThreads = getNumberOfThreads(context); |
---|
| 127 | mapClass = getMapperClass(context); |
---|
| 128 | if (LOG.isDebugEnabled()) { |
---|
| 129 | LOG.debug("Configuring multithread runner to use " + numberOfThreads + |
---|
| 130 | " threads"); |
---|
| 131 | } |
---|
| 132 | |
---|
| 133 | runners = new ArrayList<MapRunner>(numberOfThreads); |
---|
| 134 | for(int i=0; i < numberOfThreads; ++i) { |
---|
| 135 | MapRunner thread = new MapRunner(context); |
---|
| 136 | thread.start(); |
---|
| 137 | runners.add(i, thread); |
---|
| 138 | } |
---|
| 139 | for(int i=0; i < numberOfThreads; ++i) { |
---|
| 140 | MapRunner thread = runners.get(i); |
---|
| 141 | thread.join(); |
---|
| 142 | Throwable th = thread.throwable; |
---|
| 143 | if (th != null) { |
---|
| 144 | if (th instanceof IOException) { |
---|
| 145 | throw (IOException) th; |
---|
| 146 | } else if (th instanceof InterruptedException) { |
---|
| 147 | throw (InterruptedException) th; |
---|
| 148 | } else { |
---|
| 149 | throw new RuntimeException(th); |
---|
| 150 | } |
---|
| 151 | } |
---|
| 152 | } |
---|
| 153 | } |
---|
| 154 | |
---|
| 155 | private class SubMapRecordReader extends RecordReader<K1,V1> { |
---|
| 156 | private K1 key; |
---|
| 157 | private V1 value; |
---|
| 158 | private Configuration conf; |
---|
| 159 | |
---|
| 160 | @Override |
---|
| 161 | public void close() throws IOException { |
---|
| 162 | } |
---|
| 163 | |
---|
| 164 | @Override |
---|
| 165 | public float getProgress() throws IOException, InterruptedException { |
---|
| 166 | return 0; |
---|
| 167 | } |
---|
| 168 | |
---|
| 169 | @Override |
---|
| 170 | public void initialize(InputSplit split, |
---|
| 171 | TaskAttemptContext context |
---|
| 172 | ) throws IOException, InterruptedException { |
---|
| 173 | conf = context.getConfiguration(); |
---|
| 174 | } |
---|
| 175 | |
---|
| 176 | |
---|
| 177 | @Override |
---|
| 178 | public boolean nextKeyValue() throws IOException, InterruptedException { |
---|
| 179 | synchronized (outer) { |
---|
| 180 | if (!outer.nextKeyValue()) { |
---|
| 181 | return false; |
---|
| 182 | } |
---|
| 183 | key = ReflectionUtils.copy(outer.getConfiguration(), |
---|
| 184 | outer.getCurrentKey(), key); |
---|
| 185 | value = ReflectionUtils.copy(conf, outer.getCurrentValue(), value); |
---|
| 186 | return true; |
---|
| 187 | } |
---|
| 188 | } |
---|
| 189 | |
---|
| 190 | public K1 getCurrentKey() { |
---|
| 191 | return key; |
---|
| 192 | } |
---|
| 193 | |
---|
| 194 | @Override |
---|
| 195 | public V1 getCurrentValue() { |
---|
| 196 | return value; |
---|
| 197 | } |
---|
| 198 | } |
---|
| 199 | |
---|
| 200 | private class SubMapRecordWriter extends RecordWriter<K2,V2> { |
---|
| 201 | |
---|
| 202 | @Override |
---|
| 203 | public void close(TaskAttemptContext context) throws IOException, |
---|
| 204 | InterruptedException { |
---|
| 205 | } |
---|
| 206 | |
---|
| 207 | @Override |
---|
| 208 | public void write(K2 key, V2 value) throws IOException, |
---|
| 209 | InterruptedException { |
---|
| 210 | synchronized (outer) { |
---|
| 211 | outer.write(key, value); |
---|
| 212 | } |
---|
| 213 | } |
---|
| 214 | } |
---|
| 215 | |
---|
| 216 | private class SubMapStatusReporter extends StatusReporter { |
---|
| 217 | |
---|
| 218 | @Override |
---|
| 219 | public Counter getCounter(Enum<?> name) { |
---|
| 220 | return outer.getCounter(name); |
---|
| 221 | } |
---|
| 222 | |
---|
| 223 | @Override |
---|
| 224 | public Counter getCounter(String group, String name) { |
---|
| 225 | return outer.getCounter(group, name); |
---|
| 226 | } |
---|
| 227 | |
---|
| 228 | @Override |
---|
| 229 | public void progress() { |
---|
| 230 | outer.progress(); |
---|
| 231 | } |
---|
| 232 | |
---|
| 233 | @Override |
---|
| 234 | public void setStatus(String status) { |
---|
| 235 | outer.setStatus(status); |
---|
| 236 | } |
---|
| 237 | |
---|
| 238 | } |
---|
| 239 | |
---|
| 240 | private class MapRunner extends Thread { |
---|
| 241 | private Mapper<K1,V1,K2,V2> mapper; |
---|
| 242 | private Context subcontext; |
---|
| 243 | private Throwable throwable; |
---|
| 244 | |
---|
| 245 | MapRunner(Context context) throws IOException, InterruptedException { |
---|
| 246 | mapper = ReflectionUtils.newInstance(mapClass, |
---|
| 247 | context.getConfiguration()); |
---|
| 248 | subcontext = new Context(outer.getConfiguration(), |
---|
| 249 | outer.getTaskAttemptID(), |
---|
| 250 | new SubMapRecordReader(), |
---|
| 251 | new SubMapRecordWriter(), |
---|
| 252 | context.getOutputCommitter(), |
---|
| 253 | new SubMapStatusReporter(), |
---|
| 254 | outer.getInputSplit()); |
---|
| 255 | } |
---|
| 256 | |
---|
| 257 | public Throwable getThrowable() { |
---|
| 258 | return throwable; |
---|
| 259 | } |
---|
| 260 | |
---|
| 261 | @Override |
---|
| 262 | public void run() { |
---|
| 263 | try { |
---|
| 264 | mapper.run(subcontext); |
---|
| 265 | } catch (Throwable ie) { |
---|
| 266 | throwable = ie; |
---|
| 267 | } |
---|
| 268 | } |
---|
| 269 | } |
---|
| 270 | |
---|
| 271 | } |
---|