Re: Getting ClosedByInterruptException when DAG w/ edge executes

2016-07-01 Thread Scott McCarty
I'm still in the middle of design and code (learning along the way!) but
can certainly provide feedback.  Aside from commits (something I'm
definitely not thinking I should be doing), what's a good way for this?

As for the project I'm working on, it's basically front-ending Tez with an
XML graph definition that makes using it more config-driven (and, a bit
more abstract so that we can test/debug code separately from Tez--I know
that there's a local mode but we've wanted to abstract the concepts to
something more specific to our needs, and we don't want to have all
developers set up Tez).  Our experience with our data is that we sometimes
need to reprocess bunches of data and so the XML graph also allows starting
at some interior node.

It will be a service (daemon) that will read a queue of work requests and
apply very custom scheduling before submitting to Tez/YARN.  Certainly some
overlap with existing facilities in YARN...

--Scott

On Fri, Jul 1, 2016 at 10:52 AM, Hitesh Shah  wrote:

> Thanks for the update, Scott.
>
> Given that the APIs have mostly been used by other framework developers,
> there is probably quite a few things which may not be easily surfaced in
> javadocs, usage examples ( and their lack of ), etc. It would be great if
> you can provide feedback ( and patches ) to help address such shortcomings.
>
> Also, would you mind providing some more details on how you are using Tez?
>
> — Hitesh
>
> > On Jul 1, 2016, at 7:27 AM, Scott McCarty  wrote:
> >
> > Thanks for responding.
> >
> > After much hair pulling I found and fixed this.  It was due to my not
> calling setFromConfiguration(tezConf) on OrderedPartitionedKVEdgeConfig
> (other builders probably require the same call).  The comments in the
> sample code say that the call is optional (allowing override of the config
> with command line parameters) but that appears not to be the case, at least
> for my code :-(
> >
> > I also needed to make sure that the TezConfiguration I passed to it had
> been used in the call UserGroupInformation.setConfigurat(tezConf).  There's
> a lot of behind-the-scenes stuff I wasn't aware of...
> >
> > --Scott
> >
> > On Thu, Jun 30, 2016 at 3:48 PM, Siddharth Seth 
> wrote:
> > Scott,
> > Do you have logs for the entire job. I haven't seen this error before .
> The trace may be end result of an earlier failure / decision made to kill
> the task - which causes the task to be interrupted, and hence the trace.
> >
> > Thanks,
> > Sid
> >
> > On Wed, Jun 29, 2016 at 10:00 AM, Scott McCarty 
> wrote:
> > Hi,
> >
> > I am trying to get Tez 0.9.0-SNAPSHOT (latest commit as of this writing,
> but still fails with earlier 0.9.0 commits) working with vanilla hadoop
> 2.6.0 but it's failing with the following under certain conditions:
> >
> > java.lang.RuntimeException: java.io.IOException: Failed on local
> exception: java.nio.channels.ClosedByInterruptException; Host Details :
> local host is: "localhost/127.0.0.1"; destination host is:
> "localhost":9000;
> >   at
> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:209)
> >   at
> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initialize(TezGroupedSplitsInputFormat.java:156)
> >   at
> org.apache.tez.mapreduce.lib.MRReaderMapReduce.setupNewRecordReader(MRReaderMapReduce.java:157)
> >   at
> org.apache.tez.mapreduce.lib.MRReaderMapReduce.setSplit(MRReaderMapReduce.java:88)
> >   at
> org.apache.tez.mapreduce.input.MRInput.initFromEventInternal(MRInput.java:694)
> >   at
> org.apache.tez.mapreduce.input.MRInput.processSplitEvent(MRInput.java:622)
> >   at
> org.apache.tez.mapreduce.input.MRInput.handleEvents(MRInput.java:586)
> >   at
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.handleEvent(LogicalIOProcessorRuntimeTask.java:715)
> >   at
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.access$600(LogicalIOProcessorRuntimeTask.java:105)
> >   at
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask$1.runInternal(LogicalIOProcessorRuntimeTask.java:792)
> >   at
> org.apache.tez.common.RunnableWithNdc.run(RunnableWithNdc.java:35)
> >   at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.io.IOException: Failed on local exception:
> java.nio.channels.ClosedByInterruptException; Host Details : local host is:
> "localhost/127.0.0.1"; destination host is: "localhost":9000;
> >   at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:772)
> >   at org.apache.hadoop.ipc.Client.call(Client.java:1472)
> >   at org.apache.hadoop.ipc.Client.call(Client.java:1399)
> >   at
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
> >   at com.sun.proxy.$Proxy15.getFileInfo(Unknown Source)
> >   at
> 

Re: Getting ClosedByInterruptException when DAG w/ edge executes

2016-07-01 Thread Hitesh Shah
Thanks for the update, Scott. 

Given that the APIs have mostly been used by other framework developers, there 
is probably quite a few things which may not be easily surfaced in javadocs, 
usage examples ( and their lack of ), etc. It would be great if you can provide 
feedback ( and patches ) to help address such shortcomings. 

Also, would you mind providing some more details on how you are using Tez? 

— Hitesh 

> On Jul 1, 2016, at 7:27 AM, Scott McCarty  wrote:
> 
> Thanks for responding.
> 
> After much hair pulling I found and fixed this.  It was due to my not calling 
> setFromConfiguration(tezConf) on OrderedPartitionedKVEdgeConfig (other 
> builders probably require the same call).  The comments in the sample code 
> say that the call is optional (allowing override of the config with command 
> line parameters) but that appears not to be the case, at least for my code :-(
> 
> I also needed to make sure that the TezConfiguration I passed to it had been 
> used in the call UserGroupInformation.setConfigurat(tezConf).  There's a lot 
> of behind-the-scenes stuff I wasn't aware of...
> 
> --Scott
> 
> On Thu, Jun 30, 2016 at 3:48 PM, Siddharth Seth  wrote:
> Scott,
> Do you have logs for the entire job. I haven't seen this error before . The 
> trace may be end result of an earlier failure / decision made to kill the 
> task - which causes the task to be interrupted, and hence the trace.
> 
> Thanks,
> Sid
> 
> On Wed, Jun 29, 2016 at 10:00 AM, Scott McCarty  wrote:
> Hi,
> 
> I am trying to get Tez 0.9.0-SNAPSHOT (latest commit as of this writing, but 
> still fails with earlier 0.9.0 commits) working with vanilla hadoop 2.6.0 but 
> it's failing with the following under certain conditions:
> 
> java.lang.RuntimeException: java.io.IOException: Failed on local exception: 
> java.nio.channels.ClosedByInterruptException; Host Details : local host is: 
> "localhost/127.0.0.1"; destination host is: "localhost":9000; 
>   at 
> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:209)
>   at 
> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initialize(TezGroupedSplitsInputFormat.java:156)
>   at 
> org.apache.tez.mapreduce.lib.MRReaderMapReduce.setupNewRecordReader(MRReaderMapReduce.java:157)
>   at 
> org.apache.tez.mapreduce.lib.MRReaderMapReduce.setSplit(MRReaderMapReduce.java:88)
>   at 
> org.apache.tez.mapreduce.input.MRInput.initFromEventInternal(MRInput.java:694)
>   at 
> org.apache.tez.mapreduce.input.MRInput.processSplitEvent(MRInput.java:622)
>   at org.apache.tez.mapreduce.input.MRInput.handleEvents(MRInput.java:586)
>   at 
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.handleEvent(LogicalIOProcessorRuntimeTask.java:715)
>   at 
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.access$600(LogicalIOProcessorRuntimeTask.java:105)
>   at 
> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask$1.runInternal(LogicalIOProcessorRuntimeTask.java:792)
>   at org.apache.tez.common.RunnableWithNdc.run(RunnableWithNdc.java:35)
>   at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: Failed on local exception: 
> java.nio.channels.ClosedByInterruptException; Host Details : local host is: 
> "localhost/127.0.0.1"; destination host is: "localhost":9000; 
>   at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:772)
>   at org.apache.hadoop.ipc.Client.call(Client.java:1472)
>   at org.apache.hadoop.ipc.Client.call(Client.java:1399)
>   at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
>   at com.sun.proxy.$Proxy15.getFileInfo(Unknown Source)
>   at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:752)
>   at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>   at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>   at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>   at java.lang.reflect.Method.invoke(Method.java:606)
>   at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
>   at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>   at com.sun.proxy.$Proxy16.getFileInfo(Unknown Source)
>   at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1988)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1118)
>   at 
> org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1114)
>   at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>   at 
> 

Re: Getting ClosedByInterruptException when DAG w/ edge executes

2016-07-01 Thread Scott McCarty
Thanks for responding.

After much hair pulling I found and fixed this.  It was due to my not
calling setFromConfiguration(tezConf) on OrderedPartitionedKVEdgeConfig
(other builders probably require the same call).  The comments in the
sample code say that the call is optional (allowing override of the config
with command line parameters) but that appears not to be the case, at least
for my code :-(

I also needed to make sure that the TezConfiguration I passed to it had
been used in the call UserGroupInformation.setConfigurat(tezConf).  There's
a lot of behind-the-scenes stuff I wasn't aware of...

--Scott

On Thu, Jun 30, 2016 at 3:48 PM, Siddharth Seth  wrote:

> Scott,
> Do you have logs for the entire job. I haven't seen this error before .
> The trace may be end result of an earlier failure / decision made to kill
> the task - which causes the task to be interrupted, and hence the trace.
>
> Thanks,
> Sid
>
> On Wed, Jun 29, 2016 at 10:00 AM, Scott McCarty 
> wrote:
>
>> Hi,
>>
>> I am trying to get Tez 0.9.0-SNAPSHOT (latest commit as of this writing,
>> but still fails with earlier 0.9.0 commits) working with vanilla hadoop
>> 2.6.0 but it's failing with the following under certain conditions:
>>
>> java.lang.RuntimeException: java.io.IOException: Failed on local
>> exception: java.nio.channels.ClosedByInterruptException; Host Details :
>> local host is: "localhost/127.0.0.1"; destination host is:
>> "localhost":9000;
>> at
>> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:209)
>> at
>> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initialize(TezGroupedSplitsInputFormat.java:156)
>> at
>> org.apache.tez.mapreduce.lib.MRReaderMapReduce.setupNewRecordReader(MRReaderMapReduce.java:157)
>> at
>> org.apache.tez.mapreduce.lib.MRReaderMapReduce.setSplit(MRReaderMapReduce.java:88)
>> at
>> org.apache.tez.mapreduce.input.MRInput.initFromEventInternal(MRInput.java:694)
>> at
>> org.apache.tez.mapreduce.input.MRInput.processSplitEvent(MRInput.java:622)
>> at org.apache.tez.mapreduce.input.MRInput.handleEvents(MRInput.java:586)
>> at
>> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.handleEvent(LogicalIOProcessorRuntimeTask.java:715)
>> at
>> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.access$600(LogicalIOProcessorRuntimeTask.java:105)
>> at
>> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask$1.runInternal(LogicalIOProcessorRuntimeTask.java:792)
>> at org.apache.tez.common.RunnableWithNdc.run(RunnableWithNdc.java:35)
>> at java.lang.Thread.run(Thread.java:745)
>> Caused by: java.io.IOException: Failed on local exception:
>> java.nio.channels.ClosedByInterruptException; Host Details : local host is:
>> "localhost/127.0.0.1"; destination host is: "localhost":9000;
>> at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:772)
>> at org.apache.hadoop.ipc.Client.call(Client.java:1472)
>> at org.apache.hadoop.ipc.Client.call(Client.java:1399)
>> at
>> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232)
>> at com.sun.proxy.$Proxy15.getFileInfo(Unknown Source)
>> at
>> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getFileInfo(ClientNamenodeProtocolTranslatorPB.java:752)
>> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>> at
>> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
>> at
>> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>> at java.lang.reflect.Method.invoke(Method.java:606)
>> at
>> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:187)
>> at
>> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:102)
>> at com.sun.proxy.$Proxy16.getFileInfo(Unknown Source)
>> at org.apache.hadoop.hdfs.DFSClient.getFileInfo(DFSClient.java:1988)
>> at
>> org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1118)
>> at
>> org.apache.hadoop.hdfs.DistributedFileSystem$18.doCall(DistributedFileSystem.java:1114)
>> at
>> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>> at
>> org.apache.hadoop.hdfs.DistributedFileSystem.getFileStatus(DistributedFileSystem.java:1114)
>> at org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1750)
>> at org.apache.hadoop.io.SequenceFile$Reader.(SequenceFile.java:1774)
>> at
>> org.apache.hadoop.mapreduce.lib.input.SequenceFileRecordReader.initialize(SequenceFileRecordReader.java:54)
>> at
>> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat$TezGroupedSplitsRecordReader.initNextRecordReader(TezGroupedSplitsInputFormat.java:207)
>> ... 11 more
>> Caused by: java.nio.channels.ClosedByInterruptException
>> at
>> java.nio.channels.spi.AbstractInterruptibleChannel.end(AbstractInterruptibleChannel.java:202)