Re: Getting ClosedByInterruptException while running 1-to-1 DAG

2016-08-29 Thread zhiyuan yang
The problem happens at task side instead of AM side. It looks like IPC client 
is interrupted while it tries to read HDFS. Without more information, it’s hard 
to see who’s interrupting(maybe from same process or external process). 
If it’s from same process, to find the source of interruption, ideally I would 
modify the related code to print the stack traces for all live threads, but I 
don’t know whether you can do that in your dev environment. 
Did you find any other thing suspicious in the logs? If you can share the log, 
that might also be helpful.

Thanks!
Zhiyuan

> On Aug 26, 2016, at 4:03 PM,Madhusudan Ramanna  wrote:
> 
> 
> Hello,
> 
> I'm trying to get a 1-to-1 edge connection working and running into this 
> exception on the yarn application master:
> 
> 
> Vertex failed, vertexName=v1, vertexId=vertex_1472251099946_0002_1_00, 
> diagnostics=[Task failed, taskId=task_1472251099946_0002_1_00_00, 
> diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( 
> failure ) : java.lang.RuntimeException: java.io.IOException: Failed on local 
> exception: java.nio.channels.ClosedByInterruptException; Host Details : local 
> host is: "shasta.local/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: 
> "shasta.local/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)
> 
> 
> Here is my setup:
> My Hadoop 2.6.4 is running in pseudo-cluster mode and I've subclassed 
> TezExamplesBase and setting up the DAG the following way:  v1 -> v2.
> 
> 
> >>
> DataSourceDescriptor dataSource = MRInput.createConfigBuilder(new 
> Configuration(tezConf), TextInputFormat.class, inputPath)
> .groupSplits(!isDisableSplitGrouping())
> .generateSplitsInAM(!isGenerateSplitInClient())
> .build();
> 
> DataSinkDescriptor dataSink = MROutput.createConfigBuilder(new 
> Configuration(tezConf), TextOutputFormat.class, outputPath)
> .build();
> 
> Vertex v1 = Vertex.create("v1", 
> ProcessorDescriptor.create(OnetoOne.V1Processor.class.getName()))
> .addDataSource(OnetoOne.INPUT, dataSource);
> 
> Vertex v2 = Vertex.create("v2", 
> ProcessorDescriptor.create(OnetoOne.V1Processor.class.getName())).addDataSink(OnetoOne.OUTPUT,
>  dataSink);
> 
> UnorderedKVEdgeConfig config = 
> UnorderedKVEdgeConfig.newBuilder(Text.class.getName(), 
> Text.class.getName()).setFromConfiguration(tezConf).build();
> 
> Edge e = Edge.create(v1, v2, 
> config.createDefaultOneToOneEdgeProperty());
> 
> <
> 
> 
> Pointers on resolving this exception are much appreciated!
> 
> 
> Madhu
> 



Re: Parallel queries to HS2/Tez

2016-08-29 Thread Hitesh Shah
I think there are some thread pool related settings in HiveServer2 which could 
be used to throttle the no. of concurrent queries down to 1. One quick search 
led me to https://issues.apache.org/jira/browse/HIVE-5229 but you may wish to 
ask the same question on the hive mailing lists for a definitive answer. 

thanks
— Hitesh 


> On Aug 27, 2016, at 1:02 AM, Chitragar, Uday (KMLWG) 
>  wrote:
> 
> Hi Hitesh,
> 
> Thank you for the advice. While I get dev help on TEZ-3420, are there any 
> recommendations in terms of configuring HIVE/HS2 to run the dags 
> sequentially? Interestingly this is not a problem with HDP deployment which 
> obviously has a 'fuller' setup.  Local mode really helps to test.
> 
> Thank you,
> Uday
> From: Hitesh Shah 
> Sent: 25 August 2016 20:06:30
> To: user@tez.apache.org
> Subject: Re: Parallel queries to HS2/Tez
>  
> Hello Uday,
> 
> I don’t believe anyone has tried running 2 dags in parallel in local mode 
> within the same TezClient ( and definitely not for HiveServer2 ). If this is 
> with 2 instances of Tez client, this could likely be a bug in terms of either 
> how Hive is setting up the TezClient for local mode with the same directories 
> or a bug somewhere in Tez where clashing directories for intermediate data 
> might be causing an issue. FWIW, the Tez AM does not support running 2 dags 
> in parallel and quite a bit of this code path is used with local mode. 
> 
> It would be great if you could file a JIRA for this with more detailed logs 
> and then take help of the dev community to come up with a patch that 
> addresses the issue in your environment.
> 
> thanks
> — Hitesh 
> 
> 
> 
>  
> 
> > On Aug 25, 2016, at 8:34 AM, Chitragar, Uday (KMLWG) 
> >  wrote:
> > 
> > Hello,
> >  
> > When running parallel queries (simultaneous connections by two beeline 
> > clients to HS2), I get the following exception (full debug attached), 
> > interestingly running the queries one after the other completes without any 
> > problem. 
> >  
> > The setup is Hive (1.2.1) and Tez (0.8.4) running in local mode.
> > Apologies in advance if this forum is not the right place for this 
> > question, thank you.
> >  
> > 2016-08-25 15:45:41,333 DEBUG 
> > [TezTaskEventRouter{attempt_1472136335089_0001_1_01_00_0}]: 
> > impl.ShuffleInputEventHandlerImpl 
> > (ShuffleInputEventHandlerImpl.java:processDataMovementEvent(127)) - DME 
> > srcIdx: 0, targetIndex: 9, attemptNum
> > : 0, payload: [hasEmptyPartitions: true, host: , port: 0, pathComponent: , 
> > runDuration: 0]
> > 2016-08-25 15:45:41,557 ERROR [TezChild]: tez.MapRecordSource 
> > (MapRecordSource.java:processRow(90)) - java.lang.IllegalStateException: 
> > Invalid input path 
> > file:/acorn/QC/OraExtract/20160131/Devices/Devices_extract_20160229T080613_3
> > at 
> > org.apache.hadoop.hive.ql.exec.MapOperator.getNominalPath(MapOperator.java:415)
> > at 
> > org.apache.hadoop.hive.ql.exec.MapOperator.cleanUpInputFileChangedOp(MapOperator.java:457)
> > at 
> > org.apache.hadoop.hive.ql.exec.Operator.cleanUpInputFileChanged(Operator.java:1069)
> > at 
> > org.apache.hadoop.hive.ql.exec.MapOperator.process(MapOperator.java:501)
> >  
> >  
> >  
> > 2016-08-25 15:45:41,817 INFO  [TezChild]: io.HiveContextAwareRecordReader 
> > (HiveContextAwareRecordReader.java:doNext(326)) –
> > Cannot get partition description from 
> > file:/acorn/QC/reportlib/VM_ValEdit.24656because cannot find dir = file:/ac
> > orn/QC/reportlib/VM_ValEdit.24656 in pathToPartitionInfo: 
> > [file:/acorn/QC/OraExtract/20160131/Devices]
> >  
> >  
> >  
> > Regards,
> > Uday
> >  
> >  
> > 
> > 
> > Kantar Disclaimer