You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@tez.apache.org by "Rajesh Balamohan (JIRA)" <ji...@apache.org> on 2015/06/11 16:36:00 UTC

[jira] [Comment Edited] (TEZ-2552) CRC errors can cause job to run for very long time in large jobs

    [ https://issues.apache.org/jira/browse/TEZ-2552?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14581982#comment-14581982 ] 

Rajesh Balamohan edited comment on TEZ-2552 at 6/11/15 2:35 PM:
----------------------------------------------------------------

Attaching WIP patch which fixes the issue (basically account for the number of unique failures as well).  Will refine after further discussion in JIRA.

Attaching the job runtime with and without patch.  Note that without patch, job never finishes even after 13300 seconds & I killed the job (it was blocked in reducer 7 when I killed it).


was (Author: rajesh.balamohan):
Attaching WIP patch which fixes the issue (basically account for the number of unique failures as well).  Will refine after further discussion in JIRA.

Attaching the job runtime with and without patch.  Note that without patch, job never finishes even after 13300 seconds.

> CRC errors can cause job to run for very long time in large jobs
> ----------------------------------------------------------------
>
>                 Key: TEZ-2552
>                 URL: https://issues.apache.org/jira/browse/TEZ-2552
>             Project: Apache Tez
>          Issue Type: Bug
>            Reporter: Rajesh Balamohan
>            Assignee: Rajesh Balamohan
>         Attachments: Screen Shot 2015-06-11 at 3.07.47 PM.png, TEZ-2552.wip.patch, with_patch.png, without_patch.png
>
>
> Ran a fairly large job at 10 TB scale which had 1009 reducers.
> One of the machine had bad disk and NM did not delist that disk.  Machine hosting NM has disk issues (sdf & sde holds shuffle data).  exceptions.
> {noformat}
> Info fld=0x8960894
> sd 6:0:5:0: [sdf]  Add. Sense: Unrecovered read error
> sd 6:0:5:0: [sdf] CDB: Read(10): 28 00 08 96 08 90 00 00 08 00
> end_request: critical medium error, dev sdf, sector 144050320
> sd 6:0:5:0: [sdf]  Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
> sd 6:0:5:0: [sdf]  Sense Key : Medium Error [current]
> Info fld=0x895a2b9
> sd 6:0:5:0: [sdf]  Add. Sense: Unrecovered read error
> sd 6:0:5:0: [sdf] CDB: Read(10): 28 00 08 95 a2 b8 00 00 08 00
> end_request: critical medium error, dev sdf, sector 144024248
> sd 6:0:5:0: [sdf]  Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
> sd 6:0:5:0: [sdf]  Sense Key : Medium Error [current]
> Info fld=0x895a2b9
> sd 6:0:5:0: [sdf]  Add. Sense: Unrecovered read error
> sd 6:0:5:0: [sdf] CDB: Read(10): 28 00 08 95 a2 b8 00 00 08 00
> end_request: critical medium error, dev sdf, sector 144024248
> sd 6:0:5:0: [sdf]  Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
> sd 6:0:5:0: [sdf]  Sense Key : Medium Error [current]
> Info fld=0x8849edb
> sd 6:0:5:0: [sdf]  Add. Sense: Unrecovered read error
> sd 6:0:5:0: [sdf] CDB: Read(10): 28 00 08 84 9e d8 00 00 08 00
> end_request: critical medium error, dev sdf, sector 142909144
> sd 6:0:5:0: [sdf]  Result: hostbyte=DID_OK driverbyte=DRIVER_SENSE
> sd 6:0:5:0: [sdf]  Sense Key : Medium Error [current]
> Info fld=0x8849edb
> sd 6:0:5:0: [sdf]  Add. Sense: Unrecovered read error
> sd 6:0:5:0: [sdf] CDB: Read(10): 28 00 08 84 9e d8 00 00 08 00
> end_request: critical medium error, dev sdf, sector 142909144
> {noformat}
> In-memory fetches start throwing CRC as follows.  
> {noformat}
> 2015-06-11 01:01:03,728 INFO [ShuffleAndMergeRunner [Map_11]] orderedgrouped.ShuffleScheduler: PendingHosts=[]
> 2015-06-11 01:01:03,730 INFO [Fetcher [Map_11] #0] http.HttpConnection: for url=http://cn056-10.l42scl.hortonworks.com:13562/mapOutput?job=job_1433813751839_0124&reduce=3&map=attempt_1433813751839_0124_1_04_000446_0_10027&keepAlive=true sent hash and receievd reply 0 ms
> 2015-06-11 01:01:03,730 INFO [Fetcher [Map_11] #0] orderedgrouped.FetcherOrderedGrouped: fetcher#439 about to shuffle output of map InputAttemptIdentifier [inputIdentifier=InputIdentifier [inputIndex=446], attemptNumber=0, pathComponent=attempt_1433813751839_0124_1_04_000446_0_10027, fetchTypeInfo=FINAL_MERGE_ENABLED, spillEventId=-1] decomp: 45475 len: 23974 to MEMORY
> 2015-06-11 01:01:07,206 INFO [Fetcher [Map_11] #0] impl.IFileInputStream:  CurrentOffset=2510, offset=2510, off=2502, dataLength=23966, origLen=21456, len=21456, length=23970, checksumSize=4
> 2015-06-11 01:01:07,207 INFO [Fetcher [Map_11] #0] impl.IFileInputStream:  CurrentOffset=2510, offset=2510, off=0, dataLength=23966, origLen=21456, len=21456, length=23970, checksumSize=4
> 2015-06-11 01:01:07,207 WARN [Fetcher [Map_11] #0] orderedgrouped.FetcherOrderedGrouped: Failed to shuffle output of InputAttemptIdentifier [inputIdentifier=InputIdentifier [inputIndex=446], attemptNumber=0, pathComponent=attempt_1433813751839_0124_1_04_000446_0_10027, fetchTypeInfo=FINAL_MERGE_ENABLED, spillEventId=-1] from cn056-10.l42scl.hortonworks.com:13562
> org.apache.hadoop.fs.ChecksumException: Checksum Error:  CurrentOffset=2510, offset=2510, off=2502, dataLength=23966, origLen=21456, len=21456, length=23970, checksumSize=4
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.doRead(IFileInputStream.java:255)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFileInputStream.read(IFileInputStream.java:185)
> 	at org.apache.hadoop.io.compress.BlockDecompressorStream.getCompressedData(BlockDecompressorStream.java:127)
> 	at org.apache.hadoop.io.compress.BlockDecompressorStream.decompress(BlockDecompressorStream.java:98)
> 	at org.apache.hadoop.io.compress.DecompressorStream.read(DecompressorStream.java:85)
> 	at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:192)
> 	at org.apache.tez.runtime.library.common.sort.impl.IFile$Reader.readToMemory(IFile.java:619)
> 	at org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.shuffleToMemory(ShuffleUtils.java:113)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.copyMapOutput(FetcherOrderedGrouped.java:471)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.copyFromHost(FetcherOrderedGrouped.java:267)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.fetchNext(FetcherOrderedGrouped.java:164)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.callInternal(FetcherOrderedGrouped.java:177)
> 	at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.callInternal(FetcherOrderedGrouped.java:52)
> 	at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
> 	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> 	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> 	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> 	at java.lang.Thread.run(Thread.java:745)
> {noformat}
> TaskAttemptImpl didn't fail it due to the following code
> {noformat}
> float failureFraction = ((float) attempt.uniquefailedOutputReports.size())
>           / outputFailedEvent.getConsumerTaskNumber();
> {noformat}
> In this case, reducer ran in 180 slot waves.  So even if all 180 tasks report the error, it would be around 180/1009 = 0.17 (which is less than 0.25 MAX_ALLOWED_OUTPUT_FAILURES_FRACTION) and the job runs for ever (killed the job after 2 hours; normally run in couple of minutes)
> In fetcher side, reducer state would be healthy and it would continue to wait.
> Env: Tez master & Hive master
> Ref: Query_88 @ 10 TB scale.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)