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

Josh Wills commented on CRUNCH-580:
-----------------------------------

[~jeffreyquinn] I get a bunch of these stack traces when I'm running the 
integration tests locally: 

SEVERE: RuntimeException while executing runnable 
com.google.common.util.concurrent.Futures$6@585644ef with executor 
com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService@5691b195
java.lang.RuntimeException: java.io.IOException: java.lang.InterruptedException
        at com.google.common.base.Throwables.propagate(Throwables.java:156)
        at 
org.apache.crunch.io.impl.FileTargetImpl$SuccessIndicatorCreator.onSuccess(FileTargetImpl.java:149)
        at 
org.apache.crunch.io.impl.FileTargetImpl$SuccessIndicatorCreator.onSuccess(FileTargetImpl.java:133)
        at com.google.common.util.concurrent.Futures$6.run(Futures.java:975)
        at 
com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:253)
        at 
com.google.common.util.concurrent.ExecutionList$RunnableExecutorPair.execute(ExecutionList.java:149)
        at 
com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:134)
        at 
com.google.common.util.concurrent.AbstractFuture.set(AbstractFuture.java:170)
        at 
com.google.common.util.concurrent.Futures$ListFuture.setOneValue(Futures.java:1394)
        at 
com.google.common.util.concurrent.Futures$ListFuture.access$200(Futures.java:1276)
        at 
com.google.common.util.concurrent.Futures$ListFuture$2.run(Futures.java:1343)
        at 
com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:253)
        at 
com.google.common.util.concurrent.ExecutionList$RunnableExecutorPair.execute(ExecutionList.java:149)
        at 
com.google.common.util.concurrent.ExecutionList.execute(ExecutionList.java:134)
        at 
com.google.common.util.concurrent.ListenableFutureTask.done(ListenableFutureTask.java:86)
        at java.util.concurrent.FutureTask.finishCompletion(FutureTask.java:384)
        at java.util.concurrent.FutureTask.set(FutureTask.java:233)
        at java.util.concurrent.FutureTask.run(FutureTask.java:274)
        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)
Caused by: java.io.IOException: java.lang.InterruptedException
        at org.apache.hadoop.util.Shell.runCommand(Shell.java:541)
        at org.apache.hadoop.util.Shell.run(Shell.java:455)
        at 
org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:702)
        at org.apache.hadoop.util.Shell.execCommand(Shell.java:791)
        at org.apache.hadoop.util.Shell.execCommand(Shell.java:774)
        at 
org.apache.hadoop.fs.RawLocalFileSystem.setPermission(RawLocalFileSystem.java:646)
        at 
org.apache.hadoop.fs.FilterFileSystem.setPermission(FilterFileSystem.java:472)
        at 
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:460)
        at 
org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:426)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:906)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:887)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:784)
        at 
org.apache.crunch.io.impl.FileTargetImpl$SuccessIndicatorCreator.onSuccess(FileTargetImpl.java:146)
        ... 19 more

> FileTargetImpl#handleOutputs Inefficiency on S3NativeFileSystem
> ---------------------------------------------------------------
>
>                 Key: CRUNCH-580
>                 URL: https://issues.apache.org/jira/browse/CRUNCH-580
>             Project: Crunch
>          Issue Type: Bug
>          Components: Core, IO
>    Affects Versions: 0.13.0
>         Environment: Amazon Elastic Map Reduce
>            Reporter: Jeffrey Quinn
>            Assignee: Josh Wills
>         Attachments: CRUNCH-580.patch
>
>
> We have run in to a pretty frustrating inefficiency inside of 
> org.apache.crunch.io.impl.FileTargetImpl#handleOutputs.
> This method loops over all of the partial output files and moves them to 
> their ultimate destination directories, calling 
> org.apache.hadoop.fs.FileSystem#rename(org.apache.hadoop.fs.Path, 
> org.apache.hadoop.fs.Path) on each partial output in a loop.
> This is no problem when the org.apache.hadoop.fs.FileSystem in question is 
> HDFS where #rename is a cheap operation, but when an implementation such as 
> S3NativeFileSystem is used it is extremely inefficient, as each iteration 
> through the loop makes a single blocking S3 API call, and this loop can be 
> extremely long when there are many thousands of partial output files.



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

Reply via email to