001    /**
002     * Licensed to the Apache Software Foundation (ASF) under one
003     * or more contributor license agreements.  See the NOTICE file
004     * distributed with this work for additional information
005     * regarding copyright ownership.  The ASF licenses this file
006     * to you under the Apache License, Version 2.0 (the
007     * "License"); you may not use this file except in compliance
008     * with the License.  You may obtain a copy of the License at
009     *
010     *     http://www.apache.org/licenses/LICENSE-2.0
011     *
012     * Unless required by applicable law or agreed to in writing, software
013     * distributed under the License is distributed on an "AS IS" BASIS,
014     * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015     * See the License for the specific language governing permissions and
016     * limitations under the License.
017     */
018    
019    package org.apache.hadoop.mapred.lib;
020    
021    import org.apache.hadoop.util.ReflectionUtils;
022    import org.apache.hadoop.classification.InterfaceAudience;
023    import org.apache.hadoop.classification.InterfaceStability;
024    import org.apache.hadoop.mapred.MapRunnable;
025    import org.apache.hadoop.mapred.JobConf;
026    import org.apache.hadoop.mapred.Mapper;
027    import org.apache.hadoop.mapred.RecordReader;
028    import org.apache.hadoop.mapred.OutputCollector;
029    import org.apache.hadoop.mapred.Reporter;
030    import org.apache.hadoop.mapred.SkipBadRecords;
031    import org.apache.hadoop.mapreduce.lib.map.MultithreadedMapper;
032    import org.apache.commons.logging.Log;
033    import org.apache.commons.logging.LogFactory;
034    
035    import java.io.IOException;
036    import java.util.concurrent.*;
037    
038    /**
039     * Multithreaded implementation for @link org.apache.hadoop.mapred.MapRunnable.
040     * <p>
041     * It can be used instead of the default implementation,
042     * @link org.apache.hadoop.mapred.MapRunner, when the Map operation is not CPU
043     * bound in order to improve throughput.
044     * <p>
045     * Map implementations using this MapRunnable must be thread-safe.
046     * <p>
047     * The Map-Reduce job has to be configured to use this MapRunnable class (using
048     * the JobConf.setMapRunnerClass method) and
049     * the number of thread the thread-pool can use with the
050     * <code>mapred.map.multithreadedrunner.threads</code> property, its default
051     * value is 10 threads.
052     * <p>
053     * @deprecated Use {@link MultithreadedMapper} instead.
054     */
055    @Deprecated
056    @InterfaceAudience.Public
057    @InterfaceStability.Stable
058    public class MultithreadedMapRunner<K1, V1, K2, V2>
059        implements MapRunnable<K1, V1, K2, V2> {
060    
061      private static final Log LOG =
062        LogFactory.getLog(MultithreadedMapRunner.class.getName());
063    
064      private JobConf job;
065      private Mapper<K1, V1, K2, V2> mapper;
066      private ExecutorService executorService;
067      private volatile IOException ioException;
068      private volatile RuntimeException runtimeException;
069      private boolean incrProcCount;
070    
071      @SuppressWarnings("unchecked")
072      public void configure(JobConf jobConf) {
073        int numberOfThreads =
074          jobConf.getInt(MultithreadedMapper.NUM_THREADS, 10);
075        if (LOG.isDebugEnabled()) {
076          LOG.debug("Configuring jobConf " + jobConf.getJobName() +
077                    " to use " + numberOfThreads + " threads");
078        }
079    
080        this.job = jobConf;
081        //increment processed counter only if skipping feature is enabled
082        this.incrProcCount = SkipBadRecords.getMapperMaxSkipRecords(job)>0 && 
083          SkipBadRecords.getAutoIncrMapperProcCount(job);
084        this.mapper = ReflectionUtils.newInstance(jobConf.getMapperClass(),
085            jobConf);
086    
087        // Creating a threadpool of the configured size to execute the Mapper
088        // map method in parallel.
089        executorService = new ThreadPoolExecutor(numberOfThreads, numberOfThreads, 
090                                                 0L, TimeUnit.MILLISECONDS,
091                                                 new BlockingArrayQueue
092                                                   (numberOfThreads));
093      }
094    
095      /**
096       * A blocking array queue that replaces offer and add, which throws on a full
097       * queue, to a put, which waits on a full queue.
098       */
099      private static class BlockingArrayQueue extends ArrayBlockingQueue<Runnable> {
100     
101        private static final long serialVersionUID = 1L;
102        public BlockingArrayQueue(int capacity) {
103          super(capacity);
104        }
105        public boolean offer(Runnable r) {
106          return add(r);
107        }
108        public boolean add(Runnable r) {
109          try {
110            put(r);
111          } catch (InterruptedException ie) {
112            Thread.currentThread().interrupt();
113          }
114          return true;
115        }
116      }
117    
118      private void checkForExceptionsFromProcessingThreads()
119          throws IOException, RuntimeException {
120        // Checking if a Mapper.map within a Runnable has generated an
121        // IOException. If so we rethrow it to force an abort of the Map
122        // operation thus keeping the semantics of the default
123        // implementation.
124        if (ioException != null) {
125          throw ioException;
126        }
127    
128        // Checking if a Mapper.map within a Runnable has generated a
129        // RuntimeException. If so we rethrow it to force an abort of the Map
130        // operation thus keeping the semantics of the default
131        // implementation.
132        if (runtimeException != null) {
133          throw runtimeException;
134        }
135      }
136    
137      public void run(RecordReader<K1, V1> input, OutputCollector<K2, V2> output,
138                      Reporter reporter)
139        throws IOException {
140        try {
141          // allocate key & value instances these objects will not be reused
142          // because execution of Mapper.map is not serialized.
143          K1 key = input.createKey();
144          V1 value = input.createValue();
145    
146          while (input.next(key, value)) {
147    
148            executorService.execute(new MapperInvokeRunable(key, value, output,
149                                    reporter));
150    
151            checkForExceptionsFromProcessingThreads();
152    
153            // Allocate new key & value instances as mapper is running in parallel
154            key = input.createKey();
155            value = input.createValue();
156          }
157    
158          if (LOG.isDebugEnabled()) {
159            LOG.debug("Finished dispatching all Mappper.map calls, job "
160                      + job.getJobName());
161          }
162    
163          // Graceful shutdown of the Threadpool, it will let all scheduled
164          // Runnables to end.
165          executorService.shutdown();
166    
167          try {
168    
169            // Now waiting for all Runnables to end.
170            while (!executorService.awaitTermination(100, TimeUnit.MILLISECONDS)) {
171              if (LOG.isDebugEnabled()) {
172                LOG.debug("Awaiting all running Mappper.map calls to finish, job "
173                          + job.getJobName());
174              }
175    
176              // NOTE: while Mapper.map dispatching has concluded there are still
177              // map calls in progress and exceptions would be thrown.
178              checkForExceptionsFromProcessingThreads();
179    
180            }
181    
182            // NOTE: it could be that a map call has had an exception after the
183            // call for awaitTermination() returing true. And edge case but it
184            // could happen.
185            checkForExceptionsFromProcessingThreads();
186    
187          } catch (IOException ioEx) {
188            // Forcing a shutdown of all thread of the threadpool and rethrowing
189            // the IOException
190            executorService.shutdownNow();
191            throw ioEx;
192          } catch (InterruptedException iEx) {
193            throw new RuntimeException(iEx);
194          }
195    
196        } finally {
197          mapper.close();
198        }
199      }
200    
201    
202      /**
203       * Runnable to execute a single Mapper.map call from a forked thread.
204       */
205      private class MapperInvokeRunable implements Runnable {
206        private K1 key;
207        private V1 value;
208        private OutputCollector<K2, V2> output;
209        private Reporter reporter;
210    
211        /**
212         * Collecting all required parameters to execute a Mapper.map call.
213         * <p>
214         *
215         * @param key
216         * @param value
217         * @param output
218         * @param reporter
219         */
220        public MapperInvokeRunable(K1 key, V1 value,
221                                   OutputCollector<K2, V2> output,
222                                   Reporter reporter) {
223          this.key = key;
224          this.value = value;
225          this.output = output;
226          this.reporter = reporter;
227        }
228    
229        /**
230         * Executes a Mapper.map call with the given Mapper and parameters.
231         * <p>
232         * This method is called from the thread-pool thread.
233         *
234         */
235        public void run() {
236          try {
237            // map pair to output
238            MultithreadedMapRunner.this.mapper.map(key, value, output, reporter);
239            if(incrProcCount) {
240              reporter.incrCounter(SkipBadRecords.COUNTER_GROUP, 
241                  SkipBadRecords.COUNTER_MAP_PROCESSED_RECORDS, 1);
242            }
243          } catch (IOException ex) {
244            // If there is an IOException during the call it is set in an instance
245            // variable of the MultithreadedMapRunner from where it will be
246            // rethrown.
247            synchronized (MultithreadedMapRunner.this) {
248              if (MultithreadedMapRunner.this.ioException == null) {
249                MultithreadedMapRunner.this.ioException = ex;
250              }
251            }
252          } catch (RuntimeException ex) {
253            // If there is a RuntimeException during the call it is set in an
254            // instance variable of the MultithreadedMapRunner from where it will be
255            // rethrown.
256            synchronized (MultithreadedMapRunner.this) {
257              if (MultithreadedMapRunner.this.runtimeException == null) {
258                MultithreadedMapRunner.this.runtimeException = ex;
259              }
260            }
261          }
262        }
263      }
264    
265    }