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

Gopal V commented on HIVE-4436:
-------------------------------

I have replied to Navis on RB. I tried to add more logging to the code, but it 
proved to be useless at giving any real debuggable info

I added some code to the main thread (running Driver) to check for 
interruptions and log it, but it prints the useless information that the 
Thread.sleep was interrupted from the following code

{code} 
 public TaskResult pollTasks(Set<TaskResult> results)
...
      // In this loop, nothing was found
      // Sleep 10 seconds and restart
      try {
        Thread.sleep(SLEEP_TIME);
      } catch (InterruptedException ie) {
        // Do Nothing
        ;
      }
{code}

I will have to modify that loop to exit cleanly on interruptions, which does 
change existing behaviour of the code.

But as for the other comment, it is a bad idea to reset interrupted status on 
threads unconditionally.

De-interrupting threads unconditionally by ignoring the cause of the 
interruption is not a fix to any problem we have today - almost like that empty 
catch-all block in the code.

It at best a wall-paper fix, which just ignores the cracks in the 
implementation & ignores all consequences of a valid interruption.

Since the patch fixes the bug described above cleanly and without 
test-failures, I have no more updates to this patch.
                
> hive.exec.parallel=true doesn't work on hadoop-2
> ------------------------------------------------
>
>                 Key: HIVE-4436
>                 URL: https://issues.apache.org/jira/browse/HIVE-4436
>             Project: Hive
>          Issue Type: Bug
>          Components: Query Processor
>    Affects Versions: 0.9.0, 0.12.0
>         Environment: Ubuntu LXC (hive-trunk), CDH 4 on Debian
>            Reporter: Gopal V
>            Assignee: Gopal V
>         Attachments: HIVE-4436.patch, HIVE-4436-test.tgz, parallel_sorted.q
>
>
> While running a hive query with multiple independent stages, 
> hive.exec.parallel is a valid optimization to use.
> The query tested has 3 MR jobs - the first job is the root dependency and the 
> 2 further job depend on the first one.
> When hive.exec.parallel is turned on, the job fails with the following 
> exception
> {code}
> java.io.IOException: java.lang.InterruptedException
>       at org.apache.hadoop.ipc.Client.call(Client.java:1214)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:202)
>       at $Proxy12.mkdirs(Unknown Source)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
>       at java.lang.reflect.Method.invoke(Method.java:597)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:164)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:83)
>       at $Proxy12.mkdirs(Unknown Source)
>       at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.mkdirs(ClientNamenodeProtocolTranslatorPB.java:447)
>       at org.apache.hadoop.hdfs.DFSClient.primitiveMkdir(DFSClient.java:2165)
>       at org.apache.hadoop.hdfs.DFSClient.mkdirs(DFSClient.java:2136)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.mkdirs(DistributedFileSystem.java:544)
>       at org.apache.hadoop.fs.FileSystem.mkdirs(FileSystem.java:1916)
>       at 
> org.apache.hadoop.hive.ql.exec.ExecDriver.createTmpDirs(ExecDriver.java:222)
>       at 
> org.apache.hadoop.hive.ql.exec.ExecDriver.execute(ExecDriver.java:444)
>       at 
> org.apache.hadoop.hive.ql.exec.MapRedTask.execute(MapRedTask.java:138)
>       at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:145)
>       at 
> org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:57)
>       at org.apache.hadoop.hive.ql.exec.TaskRunner.run(TaskRunner.java:47)
> Caused by: java.lang.InterruptedException
>       at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1279)
>       at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:218)
>       at java.util.concurrent.FutureTask.get(FutureTask.java:83)
>       at org.apache.hadoop.ipc.Client$Connection.sendParam(Client.java:921)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1208)
> {code}
> The query plan is as follows
> {code}
>   Stage-9 is a root stage
>   Stage-8 depends on stages: Stage-9
>   Stage-3 depends on stages: Stage-8
>   Stage-0 depends on stages: Stage-3
>   Stage-4 depends on stages: Stage-0
>   Stage-5 depends on stages: Stage-8
>   Stage-1 depends on stages: Stage-5
>   Stage-6 depends on stages: Stage-1
> STAGE PLANS:
>   Stage: Stage-9
>     Map Reduce Local Work
>   Stage: Stage-8
>     Map Reduce
>             Map Join Operator
>   Stage: Stage-3
>     Map Reduce
>   Stage: Stage-0
>     Move Operator
>   Stage: Stage-4
>     Stats-Aggr Operator
>   Stage: Stage-5
>     Map Reduce
>   Stage: Stage-1
>     Move Operator
>   Stage: Stage-6
>     Stats-Aggr Operator
> {code}
> -I cannot conclude that this is purely a hive issue, will file a bug on HDFS 
> if that does show up during triage.-
> *Triaged* - set hive.stats.autogather=false; removes the bug.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to