Issue Details (XML | Word | Printable)

Key: HADOOP-153
Type: New Feature New Feature
Status: Closed Closed
Resolution: Fixed
Priority: Major Major
Assignee: Sharad Agarwal
Reporter: Doug Cutting
Votes: 1
Watchers: 12
Operations

If you were logged in you would be able to see more operations.
Hadoop Common

skip records that fail Task

Created: 21/Apr/06 12:40 AM   Updated: 08/Jul/09 04:51 PM
Return to search
Component/s: None
Affects Version/s: 0.2.0
Fix Version/s: 0.19.0

Time Tracking:
Not Specified

File Attachments:
  Size
Text File Licensed for inclusion in ASF works 153_1.patch 2008-07-17 01:09 PM Sharad Agarwal 37 kB
Text File Licensed for inclusion in ASF works 153_2.patch 2008-07-22 11:47 AM Sharad Agarwal 57 kB
Text File Licensed for inclusion in ASF works 153_3.patch 2008-07-25 10:56 AM Sharad Agarwal 63 kB
Text File Licensed for inclusion in ASF works 153_4.patch 2008-07-25 11:59 AM Sharad Agarwal 63 kB
Text File Licensed for inclusion in ASF works 153_5.patch 2008-07-29 06:12 AM Sharad Agarwal 63 kB
Text File Licensed for inclusion in ASF works 153_6.patch 2008-08-04 01:30 PM Sharad Agarwal 68 kB
Text File Licensed for inclusion in ASF works 153_7.patch 2008-08-06 10:28 AM Sharad Agarwal 68 kB
Text File Licensed for inclusion in ASF works 153_8.patch 2008-08-11 06:41 AM Sharad Agarwal 68 kB
Text File Licensed for inclusion in ASF works 153_9.patch 2008-08-11 11:04 AM Sharad Agarwal 68 kB
Text File Licensed for inclusion in ASF works skipRecords_wip1.patch 2008-06-24 12:05 PM Enis Soztutar 30 kB
Issue Links:
Dependants
 
Reference
 

Hadoop Flags: Reviewed
Release Note: Introduced record skipping where tasks fail on certain records. (org.apache.hadoop.mapred.SkipBadRecords)
Resolution Date: 11/Aug/08 12:04 PM


 Description  « Hide
MapReduce should skip records that throw exceptions.

If the exception is thrown under RecordReader.next() then RecordReader implementations should automatically skip to the start of a subsequent record.

Exceptions in map and reduce implementations can simply be logged, unless they happen under RecordWriter.write(). Cancelling partial output could be hard. So such output errors will still result in task failure.

This behaviour should be optional, but enabled by default. A count of errors per task and job should be maintained and displayed in the web ui. Perhaps if some percentage of records (>50%?) result in exceptions then the task should fail. This would stop jobs early that are misconfigured or have buggy code.

Thoughts?



 All   Comments   Work Log   Change History   Subversion Commits      Sort Order: Ascending order - Click to sort in descending order
Repository Revision Date User Message
ASF #684731 Mon Aug 11 12:03:37 UTC 2008 ddas HADOOP-153. Provides a way to skip bad records. Contributed by Sharad Agarwal.
Files Changed
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/Task.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaskTracker.java
ADD /hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestSortedRanges.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/LocalJobRunner.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/MapRunner.java
ADD /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/SkipBadRecords.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/Counters.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaskStatus.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaskUmbilicalProtocol.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/TaskInProgress.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/lib/MultithreadedMapRunner.java
MODIFY /hadoop/core/trunk/CHANGES.txt
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/InterTrackerProtocol.java
ADD /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/SortedRanges.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/MapTask.java
MODIFY /hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeReducer.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/pipes/PipesMapRunner.java
ADD /hadoop/core/trunk/src/contrib/streaming/src/test/org/apache/hadoop/streaming/TestStreamingBadRecords.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/pipes/PipesReducer.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/Task_Counter.properties
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/IsolationRunner.java
MODIFY /hadoop/core/trunk/src/contrib/streaming/src/java/org/apache/hadoop/streaming/PipeMapper.java
ADD /hadoop/core/trunk/src/test/org/apache/hadoop/mapred/TestBadRecords.java
MODIFY /hadoop/core/trunk/src/mapred/org/apache/hadoop/mapred/ReduceTask.java