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

Zhiyuan Yang commented on TEZ-3478:
-----------------------------------

Mostly looks good to me. Minor comments:  

\\1. SimpleFetchedInputAllocator.allocate()
{code}
Path diskFetchPath = diskFetchedInput.getInputPath().getParent();
{code}
This make structures of fetched files known to SimpleFetchedInputAllocator. We 
might want to hide this from SimpleFetchedInputAllocator in case the structure 
get changes in future.

\\2.ShuffleManager::FetchFutureCallback.onFailure()   
{code}
shutdown();
{code}
This one is not necessary because it will be invoked eventually during task 
cleanup.

> Cleanup fetcher data for failing task attempts (Unordered fetcher)
> ------------------------------------------------------------------
>
>                 Key: TEZ-3478
>                 URL: https://issues.apache.org/jira/browse/TEZ-3478
>             Project: Apache Tez
>          Issue Type: Improvement
>            Reporter: Rajesh Balamohan
>            Assignee: Rajesh Balamohan
>            Priority: Minor
>         Attachments: TEZ-3479.1.patch, TEZ-3479.branch-0.7.001.patch
>
>
> Env: 3 node AWS cluster with entire dataset in S3. Since data is in S3, it 
> does have not additional storage for HDFS (uses existing space available in 
> VMs). tez version is 0.7.
> With some workloads (e.g q29 in tpcds), unordered fetchers download data in 
> parallel for different vertices and runs out of disk space. However, 
> downloaded
> data related to these failed task attempts are not cleared. So subsequent 
> task attempts also encounter similar situation and fails with "No space" 
> exception. e.g stack trace
> {noformat}
> , errorMessage=Fetch failed:org.apache.hadoop.fs.FSError: 
> java.io.IOException: No space left on device
>         at 
> org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileOutputStream.write(RawLocalFileSystem.java:261)
>         at 
> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82)
>         at java.io.BufferedOutputStream.write(BufferedOutputStream.java:126)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
>         at java.io.DataOutputStream.write(DataOutputStream.java:107)
>         at 
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSOutputSummer.writeChunk(ChecksumFileSystem.java:426)
>         at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:206)
>         at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:124)
>         at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:110)
>         at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:58)
>         at java.io.DataOutputStream.write(DataOutputStream.java:107)
>         at 
> org.apache.tez.runtime.library.common.shuffle.ShuffleUtils.shuffleToDisk(ShuffleUtils.java:146)
>         at 
> org.apache.tez.runtime.library.common.shuffle.Fetcher.fetchInputs(Fetcher.java:771)
>         at 
> org.apache.tez.runtime.library.common.shuffle.Fetcher.doHttpFetch(Fetcher.java:497)
>         at 
> org.apache.tez.runtime.library.common.shuffle.Fetcher.doHttpFetch(Fetcher.java:396)
>         at 
> org.apache.tez.runtime.library.common.shuffle.Fetcher.callInternal(Fetcher.java:195)
>         at 
> org.apache.tez.runtime.library.common.shuffle.Fetcher.callInternal(Fetcher.java:70)
>         at org.apache.tez.common.CallableWithNdc.call(CallableWithNdc.java:36)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:262)
>         at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>         at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: No space left on device
>         at java.io.FileOutputStream.writeBytes(Native Method)
>         at java.io.FileOutputStream.write(FileOutputStream.java:345)
>         at 
> org.apache.hadoop.fs.RawLocalFileSystem$LocalFSFileOutputStream.write(RawLocalFileSy
> {noformat}
> This would also affect any other job running in the cluster at the same time. 
> It would be helpful to clean up the data downloaded for the failed task 
> attempts.
> Creating this ticket mainly for unordered fetcher case, though it could be 
> similar case for ordered shuffle case as well.
> e.g files
> {noformat}
> 17M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_0_10023_src_62_spill_-1.out
> 18M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_0_10023_src_63_spill_-1.out
> 16M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_0_10023_src_64_spill_-1.out
> ..
> ..
> 18M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_2_10003_src_0_spill_-1.out
> 17M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_2_10003_src_13_spill_-1.out
> 16M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_2_10003_src_15_spill_-1.out
> 16M   
> /hadoopfs/fs1/yarn/nodemanager/usercache/cloudbreak/appcache/application_1476667862449_0043/attempt_1476667862449_0043_1_07_000028_2_10003_src_17_spill_-1.ou
> {noformat}



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

Reply via email to