Re: clarification regarding Tez DAGs
Hello Robert, Some of the questions may be better answered on the Hive list but I will take a first crack of some of them. From a Tez perspective, lets use vertices and ignore Maps and Reducers for now. Hive uses this as a convention to indicate that a vertex is either reading data from HDFS (map) or has an inbound shuffle edge (reduce). For a given vertex, each task in the vertex is composed of a set of inputs, a processor and a set of outputs. The key-value constructs are defined by the kind of Input and Output being used. Today, pretty much all I/Os are key-value based. The edge types defined how data is being transferred but they do not completely control how data is manipulated to be sent across the edge. A lot of that is defined within the Inputs and Outputs. To clarify, a broadcast edge implies an task from an upstream vertex will send all of its output to all tasks in the downstream vertex. However, a broadcast edge does not imply whether the data is sorted or non-sorted. Likewise for the scatter-gather edge. This edge allows for each task in an upstream vertex to generate partitioned data that can be distributed to all downstream tasks. This can be used to mimic the MR shuffle by having the Output in the upstream vertex generate partitioned and sorted data and be sent to a downstream Input which will do a merge+sort for all relevant partitions that it needs from all upstream tasks. This allows for plugging in a shuffle-like edge implementation that does not sort data but only partitions it ( or groups it ). To answer your questions: >>> for (2) and (3) Yes. The processor can generate a different key, val pair if it wants to. A simple usage of a MRR chain would be a case where you want to a group by X followed by an order by Y. It can be done in some form via a 2-stage DAG but a simplistic model would be a 3-stage dag where stage 2 does the grouping and stage 3 the order by. >>> for (4) and (5) I am not sure I understand the question. Could you clarify what M2 expects in terms of its input? If you combined the logic of M1 and M2 instead of a single task, would that retain the behavior that you want? If the reduce stage or a map stage in the middle of a DAG are both expecting an inbound shuffled input then there is no difference except for their logical names. Feel free to send more questions to the list to get more clarifications. thanks — Hitesh > On Nov 28, 2016, at 3:44 PM, Robert Grandlwrote: > > Hi all, > > I am trying to get a better understanding of the DAGs generated by Hive atop > Tez. However, I have some more fundamental questions about the types of > tasks/edges in a Tez DAG. > > 1) In case of MapReduce: > Map - takes records and generates pairs. > Reduce - takes pairs and reduce the list of the values for the > same Key. > Question:That means the reducer does not change the Keys right? > > In case of Tez, things can be more complex: > 2) For example, Map tasks can be in the middle of the DAG too. My > understanding is that in this case the input is a set of pairs > and the output can be a set of different value pairs. > Is this true for any type of input edge (scatter gather, broadcast, one to > one)? > > 3) Reduce tasks can be in the middle as well. Can I assume that the reducer > also can change the keys? For example, in case of Map -> Reduce_1 -> Reduce_2 > patterns, what is the main reason of having Reduce_2? It is because the keys > are changed by Reduce_2 while Reduce_1 preserve the ones from the Map? > > 4) On a related note. In case of Map_1 -> Map_2 patterns, it is possible > Map_2 to preserve the Keys generated by Map_1 or will be new keys? > > 4) If my guess that both Map and Reduce stages can eventually change the > keys, what is the main difference of having both Map and Reduce stages in the > middle of the DAG (i.e. not input stages or leaf stages). > > Thanks, > - Robert >
Re: Bad Log URL
For the logs to a container in the NM, the NM’s http address is obtained from YARN APIs. Is this the only page in which the “:” is missing or is it missing in other rows’ links within the task attempts table? Can you confirm that the links to the NodeManagers work correctly from the ResourceManager UI? thanks — Hitesh > On Nov 13, 2016, at 9:55 PM, Premal Shahwrote: > > Hi, > Looks like the log url does not have a colon in it. Is this bad config on my > end or should I open a ticket for this? > > > > -- > Regards, > Premal Shah.
Re: Vertex Parallelism
I suggest writing a custom InputFormat or modifying your existing InputFormat to generate more splits and at the same time, disable splits grouping for the vertex in question to ensure that you get the high level of parallelism that you want to achieve. The log snippet is just indicating that vertex had been setup with -1 tasks as the splits are being calculated in the AM and that the vertex parallelism will be set via the initializer/controller (based on the splits from the Input Format). — Hitesh > On Oct 31, 2016, at 3:33 PM, Madhusudan Ramannawrote: > > Hello Tez team, > > We have a native Tez application. The first vertex in the graph is a > downloader. This vertex takes a CSV or sequence file that contains the > "urls" as input, downloads content and passes the content on to the next > vertex. This input to vertex is smaller than the min split size. However, > we'd like to have more than one task for running for this vertex to help > throughput. How do we set the tasks on this particular vertex to be greater > than one ? Of course for other vertices in the graph, number of tasks as > computed by data size fits perfectly fine. > > Currently, we're seeing this in the logs: > > > > > Root Inputs exist for Vertex: download : {_initial={InputName=_initial}, > {Descriptor=ClassName=org.apache.tez.mapreduce.input.MRInput, > hasPayload=true}, > {ControllerDescriptor=ClassName=org.apache.tez.mapreduce.common.MRInputAMSplitGenerator, > hasPayload=false}} > Num tasks is -1. Expecting VertexManager/InputInitializers/1-1 split to set > #tasks for the vertex vertex_1477944280627_0004_1_00 [download] > Vertex will initialize from input initializer. vertex_1477944280627_0004_1_00 > [download] > < > > > > Thanks for your help ! > > Madhu > > >
Re: Tez containers and input splits
That is similar to MR’s approach. When a task requests for containers, it will specify that it needs to be scheduled on a particular set of hosts/racks via a TaskLocationHint. The TaskLocationHint is converted to a container ask from YARN i.e. one container on any of the hosts or racks specified in the location hint. See MRInputHelpers and TaskLocationHint classes for more details. Once a container is allocated to Tez, it will try its best to do host-level locality. After a certain time back-off, if rack level fall backs are enabled, it will try to match an unassigned task to a rack and then eventually fall back to a “any” match if that fallback option is enabled. — Hitesh > On Oct 28, 2016, at 12:10 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > Hello Hitesh, > > Thanks for that explanation ! Could you clarify about how locality of input > splits is used.. > > thanks, > Madhu > > > On Thursday, October 27, 2016 11:19 PM, Hitesh Shah <hit...@apache.org> wrote: > > > Hello Madhusudan, > > I will start with how container allocations work and make my way back to > explaining splits. > > At the lowest level, each vertex will have decided to run a number of tasks. > At a high level, when a task is ready to run, it tells the global DAG > scheduler about its requirements ( i.e. what kind of container resources it > needs, additional container specs such as env, local resources, etc. and also > where it wants to be executed for locality. > > The global scheduler then requests the ResourceManager for as many containers > as there are pending tasks. When YARN allocates a container to the Tez AM, > the Tez AM decides which is the highest priority task ( vertices at the top > of the tree run first ) that matches the container allocated and runs the > task on it. Re-used containers are given higher priority over new containers > due to JVM launch costs. And YARN may not give Tez all the containers it > requested so Tez will make do with whatever it has. It may end up releasing > containers which don’t match if there are non-matching tasks that need to be > run. > > Now, let us take a “map” vertex which is reading data from HDFS. In MR, each > task represented one split ( or a group if you use something like Hive’s > CombineFileInputFormat ). In Tez, there are a couple of differences: > > 1) The InputFormat is invoked in the AM i.e. splits are calculated in the AM > ( can be done on the client but most folks now run those in the AM) > 2) Splits are grouped based on the wave configurations ( > https://cwiki.apache.org/confluence/display/TEZ/How+initial+task+parallelism+works > ). > > Each grouped split will be mapped to one task. This will then define what > kind of container is requested. > > Let us know if you have more questions. > > thanks > — Hitesh > > > > On Oct 27, 2016, at 5:06 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > > > Hello Folks, > > > > We have a native Tez application. My question is mainly about MR inputs > > and tez allocated containers. How does tez grab containers ? Is it one per > > input split ? Could someone shed some light on this ? > > > > thanks, > > Madhu > >
Re: Tez Sessions
HiveServer2 today maintains Tez sessions ( when running with perimeter security i.e Ranger/Sentry ) and re-uses the session across queries. Tez AM recovery works for the most part. It will try to recover completed tasks of the last running DAG and complete the one that did not complete or were still running. It does not handle cases where the committer was in the middle of a commit though so those dags will abort when trying to recover. Given the complexity of recovery, there are probably bugs that we may not have discovered yet but for the most part, it does function well. There are a few issues you should consider when trying to use a single AM: - on secure clusters, the delegation token max lifetime is 7 days so you will need to re-cycle apps on a weekly basis. - YARN does not clean up data/logs for an app until the app completes so this can add space pressure on the yarn local dirs. That said, there is some work happening as part of TEZ-3334 to help clean up intermediate data on a regular basis. There have been a couple of other jiras filed recently too to look at being able to clean up data more frequently. thanks — Hitesh > On Oct 20, 2016, at 2:35 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > Ok, no worries. I agree that this single AM model would be very close to a > mini-job tracker. One of the options we're investigating having 1 yarn Tez > AM running all our DAGs. Given this AM already has all the > resources/containers, we were thinking this could save on the cost of AM, and > container initialization. > > We haven't looked into tez recovery as well. Durability is one of our big > concerns as well. > > > On Thursday, October 20, 2016 12:44 PM, Hitesh Shah <hit...@apache.org> wrote: > > > Not supported as of now. There are multiple aspects to supporting this > properly. One of the most important issues to address would be to do proper > QoS across various DAGs i.e. what kind of policies would need to be built out > to run multiple DAGs to completion within a limited amount of resources. The > model would become close to a mini-jobtracker or a spark-standalone cluster. > > Could you provide more details on what you are trying to achieve? We could > try and provide different viewpoints on trying to get you to a viable > solution. > > — Hitesh > > > On Oct 20, 2016, at 10:52 AM, Madhusudan Ramanna <m.rama...@ymail.com> > > wrote: > > > > Hello Folks, > > > > http://hortonworks.com/blog/introducing-tez-sessions/ > > > > From the above post it seems like DAGs can only be executed serially. > > Could DAGs be executed in parallel on one Tez AM ? > > > > thanks, > > Madhu > >
Re: Tez Sessions
Not supported as of now. There are multiple aspects to supporting this properly. One of the most important issues to address would be to do proper QoS across various DAGs i.e. what kind of policies would need to be built out to run multiple DAGs to completion within a limited amount of resources. The model would become close to a mini-jobtracker or a spark-standalone cluster. Could you provide more details on what you are trying to achieve? We could try and provide different viewpoints on trying to get you to a viable solution. — Hitesh > On Oct 20, 2016, at 10:52 AM, Madhusudan Ramannawrote: > > Hello Folks, > > http://hortonworks.com/blog/introducing-tez-sessions/ > > From the above post it seems like DAGs can only be executed serially. Could > DAGs be executed in parallel on one Tez AM ? > > thanks, > Madhu
Re: Container settings at vertex level
Hello Madhu, If you are using Tez via Hive, then this would need a fix in Hive. I don’t believe Hive supports different settings for each vertex in a given query today. However, for native jobs, Tez already supports different specs for each vertex: Vertex::setTaskResource() ( configuring yarn resources i.e. memory/cpu ) Vertex::setTaskLaunchCmdOpts() ( java opts, etc ) Does the above help? Or are you looking for something different? thanks — HItesh > On Oct 20, 2016, at 10:44 AM, Madhusudan Ramannawrote: > > Hello Folks, > > Some vertices require more memory than other vertices. These vertices are > memory intensive. The graph, in general, takes a long(ish) time to complete. > Default allocation of a huge chunk of memory to this one DAG/application > severely limits concurrent yarn containers that can be run. How can we > influence Tez Runtime to request and execute some vertices in specialized > containers ? What is a good solution to this problem? > > thanks, > Madhu
Re: Tez UI
Thanks for doing all the investigative work and patiently trying out the various options we provided. If you get a chance, feel free to try the following ( and hopefully provide a patch if it works better for you as a long term solution ): - compile the tez source against CDH’s distribution of hadoop ( would entail adding the cdh maven repo to the pom as well as updating hadoop.version hence the patch if you can put this under a maven profile ) - the above I believe will pull in the transitive dependencies of jackson from the hadoop version compiled against. - this allows you to retain CDH components as is and only modify tez instead of messing with the hadoop install. thanks — Hitesh > On Oct 18, 2016, at 9:28 PM, Stephen Sprague <sprag...@gmail.com> wrote: > > i'll be a monkey's uncle. that did it. > > this is what i did: > > * i untarred share/tez.tar.gz > > * i then set these two env vars: > export YARN_USER_CLASSPATH_FIRST=true > export YARN_USER_CLASSPATH=/usr/lib/apache-tez-0.8.4-bin/share/* > > * and started up the ATS as such: > > sudo -u yarn -- ./yarn-daemon.sh --config /etc/hadoop/conf start > timelineserver > > > hard to believe that jackson version from 1.8.8 to 1.9.13 made that > difference. > > great call Hitesh. many thanks. I've been wrestling with this for quite some > time. > > Cheers, > Stephen. > > On Tue, Oct 18, 2016 at 8:09 PM, Stephen Sprague <sprag...@gmail.com> wrote: > * hadoop version: Hadoop 2.6.0-cdh5.4.7 > > * tez version: 0.8.4 (i am using the bin distro from apache - so i didn't > "make" it) > > * i found these in the tez distro tarball. > >$ cd /home/dwr/downloads/apache-tez-0.8.4-bin >$ find . -name '*jers*' > ./lib/jersey-client-1.9.jar > ./lib/jersey-json-1.9.jar > > * i found these in the tez server-side tarball > $ tar ztf share/tez.tar.gz | grep jers > lib/jersey-json-1.9.jar > lib/jersey-core-1.9.jar > lib/jersey-client-1.9.jar > lib/jersey-guice-1.9.jar > > > $ tar ztf share/tez.tar.gz | grep jack > lib/jackson-core-asl-1.9.13.jar > lib/jackson-mapper-asl-1.9.13.jar > lib/jackson-jaxrs-1.9.13.jar > lib/jackson-xc-1.9.13.jar > > * i found these in the hadoop ATS timeline server classpath: > > * /usr/lib/hadoop/lib/jersey-server-1.9.jar > * /usr/lib/hadoop/lib/jersey-core-1.9.jar > * /usr/lib/hadoop/lib/jersey-json-1.9.jar > * /usr/lib/hadoop-yarn/lib/jersey-client-1.9.jar > * /usr/lib/hadoop-yarn/lib/jersey-guice-1.9.jar > > * /usr/lib/hadoop/lib/jackson-jaxrs-1.8.8.jar > * /usr/lib/hadoop/lib/jackson-xc-1.8.8.jar > * /usr/lib/hadoop/lib/jackson-core-asl-1.8.8.jar > * /usr/lib/hadoop/lib/jackson-mapper-asl-1.8.8.jar > > > > so jersey jars look to be in sync but the jackson ones look a step behind. > > what do you think? should i force the 1.9's into the ATS CLASSPATH? can't > hurt would be my guess. lemme try. > > Cheers, > Stephen. > > > On Mon, Oct 17, 2016 at 2:44 PM, Stephen Sprague <sprag...@gmail.com> wrote: > Thanks Hitesh. i'll look into this tonight. > > On Mon, Oct 17, 2016 at 10:31 AM, Hitesh Shah <hit...@apache.org> wrote: > Hello Stephen, > > I checked branch-2.4.0 of hadoop just to make sure - it does contain > “eventinfo” as a member of the TimelineEvent class so this does not seem to > indicate any issue in terms of a potential mismatch or a missing patch in the > version of hadoop that you are running. > > Based on the logs, YARN_APPLICATION_ATTEMPT is data being written by the YARN > RM into YARN Timeline and that seems to be working. What is not working is > the Tez AM talking to YARN Timeline. I have not come across the property not > found issue in the past. One guess I have is that this potentially could due > be either due to something incompatible with the timeline client class on Tez > AM’s classpath and/or a combination of the jackson/jersey jars in use. > > There are a few things you should look into and update this thread with the > following info: >- what version of hadoop you are running >- what version of Tez ( and also what version of hadoop it was compiled > against ) >- check the hadoop classpath for jackson/jersey jars and compare the > versions in it to the versions in the tez tarball. > > thanks > — Hitesh > > > On Oct 16, 2016, at 9:24 PM, Stephen Sprague <sprag...@gmail.com> wrote: > > > > thanks Allan. so i enabled DEBUG,console on the ATS. I see this in that > > log: > > > > 16/10/16 21:07:59 DEBUG mor
Re: Tez UI
Hello Stephen, yarn-site.xml needs to be updated wherever the Tez client is used. i.e if you are using Hive, then wherever you launch the Hive CLI and also where the HiveServer2 is installed ( HS2 will need a restart ). To see if the connection to timeline is/was an issue, please check the yarn app logs for any Tez application ( the application master logs to be more specific: syslog_dag* files) to see if there are any warnings/exceptions being logged related to history event handling. thanks — Hitesh > On Oct 15, 2016, at 9:58 PM, Stephen Spraguewrote: > > hmm... made that change to yarn-site.xml and retarted the timelineserver and > RM. > > $ sudo netstat -lanp | grep 31168 #timelineserver > > tcp0 0 172.19.103.136:102000.0.0.0:* LISTEN > 31168/java > tcp0 0 172.19.103.136:8188 0.0.0.0:* LISTEN > 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45299 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45298 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45322 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45297 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45316 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45318 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45317 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45321 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45326 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45314 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45315 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45313 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45320 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45324 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45325 > ESTABLISHED 31168/java > tcp0 0 172.19.103.136:8188 172.19.103.136:45319 > ESTABLISHED 31168/java > unix 2 [ ] STREAM CONNECTED 1455259739 31168/java > unix 2 [ ] STREAM CONNECTED 1455253313 31168/java > > > still no dice though. same error. i only changed yarn-site.xml on the > namenode though. you think i need to copy it to all the datanodes and > restart the NM's too? > > any other suggestions? > > 'ppreciate the help! > > > Cheers, > Stephen. > > On Sat, Oct 15, 2016 at 8:46 PM, Allan Wilson wrote: > Just saw Gopals response...that def needs updating too. > > Sent from my iPhone > > On Oct 15, 2016, at 9:31 PM, Stephen Sprague wrote: > >> thanks guys. lemme answer. >> >> Sreenath- >> 1. yarn.acl.enable = false (ie. i did not set it) >> 2. this: http://dwrdevnn1.sv2.trulia.com:9766 displays index.html with an >> *empty* list >> >> Gopal- >> 3. i'll replace 0.0.0.0 with dwrdevnn1.sv2.trulia.com and see happens... >> >> Allan- >> 4. yes, metrics are enabled. >> >> >> I'll let you know what happens with Gopal's suggestion. >> >> >> Cheers, >> Stephen. >> >> On Sat, Oct 15, 2016 at 8:20 PM, Allan Wilson wrote: >> Are you emitting metrics to the ATS? >> >> yarn.timeline-service.enabled=true >> >> Sent from my iPhone >> >> On Oct 15, 2016, at 8:36 PM, Sreenath Somarajapuram >> wrote: >> >>> Hi Stephen, >>> >>> The error message is coming from ATS, and it says that the application data >>> is not available. >>> And yes, tez_application_1476574340629_0001 is a legit value. It can be >>> considered as the id for Tez application details. >>> >>> Please help me with these: >>> 1. Are you having yarn.acl.enable = true in yarn-site.xml ? >>> 2. On going to http://dwrdevnn1.sv2.trulia.com:9766 from your browser >>> window, the UI is supposed to display a list of DAGs. Are you able to view >>> them? >>> >>> Thanks, >>> Sreenath >>> >>> From: Stephen Sprague >>> Reply-To: "user@tez.apache.org" >>> Date: Sunday, October 16, 2016 at 7:16 AM >>> To: "user@tez.apache.org" >>> Subject: Tez UI >>> >>> hey guys, >>> i'm having hard time getting the Tez UI to work. I'm sure i'm doing >>> something wrong but i can't seem to figure out. Here's my scenario. >>> >>> 1. i'm using nginx as the webserver. port 9766. using that port without >>> params correctly displays index.html. (i followed the instructions on >>> unzipping the war file - that seems
Re: Origin of failed tasks
If you have the logs for the application master, you can try the following: grep “[HISTORY]” | grep “TASK_ATTEMPT_FINISHED” This will give you info on any failed task attempts. The AM logs have history events being published to them. You can do grep “[HISTORY]” | grep “_” where entity type is one of DAG, VERTEX, TASK, TASK_ATTEMPT and event type is STARTED or FINISHED. The logs are also split into diff files. e.g. The AM logs use a syslog_dag… format to split across dags. Task/Container logs use syslog_attempt* format to split out logs for different task attempts. If you have YARN timeline enabled, you can use the analyzers to do more analysis on the dag specific data. These are more related to perf tuning and not failure diagnostics though. thanks — Hitesh > On Oct 11, 2016, at 5:09 PM, Allan Wilsonwrote: > > Use the yarn logs command. That's your only chance without the TEZ UI. I > setup the TEZ UI > In our shop and it is really nice. > > Allan > Sent from my iPhone > >> On Oct 11, 2016, at 5:05 PM, Jan Morlock wrote: >> >> Hi, >> >> currently failed tasks occur during the execution of my Hive/Tez job. >> However in the end, the overall job succeeds. Is it possible to find out >> afterwards about the origin of those failed tasks (without using the Tez >> UI) just by analyzing the output log files? >> >> Best regards >> Jan
Re: Debugging M/R job with tez
Thanks for filing the issues, Manuel. I took a quick look at trying to run the MR job in tez local mode. A native tez job running in local mode ( i.e. by running something like "hadoop jar ./tez/tez-examples-0.9.0-SNAPSHOT.jar wordcount -Dtez.local.mode=true …” ) works but local mode when trying to run an MR job via the tez framework does not. I don’t believe that this has really worked at all since the initial implementation of local mode was committed. There are some quirks of the MR to Tez translation layer which are still pending from an MR local mode perspective. If you can file a JIRA for the local mode issue, I can provide a small patch that allowed me to make some minor headway before I ended up hitting other ones. thanks — Hitesh > On Oct 5, 2016, at 5:44 AM, Manuel Godbert <manuel.godb...@gmail.com> wrote: > > Hello, > > I just opened TEZ-3459, with attached code adressing 3 of the issues I > encountered, including the embedded jars one. > > I did not manage yet to provide an example showing the issue I had with > multiple outputs. It would definitely help me if I could run my jobs locally > with Tez to understand the specificity of these jobs. Would it be possible to > get some support to set up my workstation to achieve this? > > Brgds > > Manuel > > On Wed, Sep 28, 2016 at 8:37 PM, Hitesh Shah <hit...@apache.org> wrote: > Thanks for the context, Manuel. > > Full compat with MR is something that has not really been fully tested with > Tez. We believe that it works for the most part but there are probably cases > out there which have either not been addressed or some which we are not aware > of. > > It is great that you are trying this out. We can definitely help you figure > out these issues and get the fixes into Tez to allow more users to seamlessly > run MR jobs on Tez. It will be great if you can file a jira for the MR > distributed cache handling of archives in Tez. A simple example to reproduce > it would help a lot too so as to allow any of the Tez contributors to quickly > debug and fix. I am assuming you are passing in archives/fat-jars to the > distributed cache which MR implicitly applies ./* + ./lib/* pattern against > to add to the runtime classpath? I am guessing this is something we may not > have handled correctly in the translation layer. > > thanks > — Hitesh > > > On Sep 28, 2016, at 9:38 AM, Manuel Godbert <manuel.godb...@gmail.com> > > wrote: > > > > Hello, > > > > In non local mode my M/R jobs generally behave as expected with Tez. > > However some still resist, and I am trying to have them running locally to > > understand if I they can work with some changes (either in my code or in > > Tez code, and in that latter case I planned to contribute some way to the > > Tez effort). Runnning the WordCount locally is only a first step. > > > > I won't be able to provide source code easily for the real problematic > > jobs, as we use a quite big home made framework on top of hadoop and that > > is not open source... in a few words most of my issues actually seem to > > come from the task attempts IDs management. We have subclassed the output > > committers to manage multiple outputs, and when we reach the commit task > > step the produced files are not always where expected in the temporary task > > attempt paths. It is hard to say what happens exactly, and this is why I > > wanted to reproduce the issue locally before sharing it. > > > > Besides this, another minor issue we got is that we used to package our > > applicative jars with nested dependencies in /lib and these are ignored by > > Tez. We could easily work around this expanding these and adapting our > > classpath. > > > > Regards > > > > On Wed, Sep 28, 2016 at 5:46 PM, Hitesh Shah <hit...@apache.org> wrote: > > Hello Manuel, > > > > Thanks for reporting the issue. Let me try and reproduce this locally to > > see what is going on. > > > > A quick question in general though - are you hitting issues when running in > > non-local mode too? Would you mind sharing that details on the issues you > > hit? > > > > thanks > > — Hitesh > > > > > > > On Sep 27, 2016, at 9:53 AM, Manuel Godbert <manuel.godb...@gmail.com> > > > wrote: > > > > > > Hello, > > > > > > I have map/reduce jobs that work as expected within YARN, and I want to > > > see if Tez can help me improving their performance. Alas, I am > > > experiencing issues and I want to understand what happens, to see if I > > > can adapt my code
Re: Zip Exception since commit da4098b9
To pinpoint the issue, one approach would be to change the history logger to SimpleHistoryLogger . i.e comment out the property for tez.history.logging.service.class in the configs so that it falls back to the default value. This should generate a history log file as part of the application logs which should help us understand whether tez itself is not generating the data or YARN timeline is somehow losing it. Any exceptions in the DAGAppMaster log and/or the yarn timeline logs when this job runs? — HItesh > On Sep 28, 2016, at 1:30 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > Hitesh, > > Some information like appId is getting through to timeline server, but not > all. See attached. > > Here is the output of > > http://timelinehost:port/ws/v1/timeline/TEZ_DAG_ID/ > {"entities":[{"events":[{"timestamp":1475094093409,"eventtype":"DAG_FINISHED","eventinfo":{}},{"timestamp":1475094062692,"eventtype":"DAG_STARTED","eventinfo":{}},{"timestamp":1475094062688,"eventtype":"DAG_INITIALIZED","eventinfo":{}},{"timestamp":1475094062055,"eventtype":"DAG_SUBMITTED","eventinfo":{}}],"entitytype":"TEZ_DAG_ID","entity":"dag_1475091857089_0007_1","starttime":1475094062055,"domain":"DEFAULT","relatedentities":{},"primaryfilters":{},"otherinfo":{}}]} > > http://host:8188/ws/v1/timeline/TEZ_DAG_ID/dag_1475091857089_0007_1 > > {"events":[{"timestamp":1475094093409,"eventtype":"DAG_FINISHED","eventinfo":{}},{"timestamp":1475094062692,"eventtype":"DAG_STARTED","eventinfo":{}},{"timestamp":1475094062688,"eventtype":"DAG_INITIALIZED","eventinfo":{}},{"timestamp":1475094062055,"eventtype":"DAG_SUBMITTED","eventinfo":{}}],"entitytype":"TEZ_DAG_ID","entity":"dag_1475091857089_0007_1","starttime":1475094062055,"domain":"DEFAULT","relatedentities":{},"primaryfilters":{},"otherinfo":{}} > > > > On Wednesday, September 28, 2016 8:44 AM, Hitesh Shah <hit...@apache.org> > wrote: > > > Hello Madhusudan, > > Thanks for the patience. Let us take this to a jira where once you attach > more logs, we can root cause the issue. > > A few things to attach to the jira: > - yarn-site.xml > - tez-site.xml > - hadoop version > - timeline server log for the time period in question > - application logs for any tez app which fails to display > - output of http://timelinehost:port/ws/v1/timeline/TEZ_DAG_ID// ( > e.g. dag_1475014682883_0027_1 ) > > thanks > — Hitesh > > > On Sep 27, 2016, at 10:42 PM, Madhusudan Ramanna <m.rama...@ymail.com> > > wrote: > > > > So I downloaded Tez commit 91a397b0ba and built the dist package. We're > > not seeing the zip exception anymore. > > > > However, now Tez UI is completely broken. Not at all sure what is happening > > here. Please see attached screenshots. > > > > > > 2016-09-28 05:11:40,903 [INFO] [main] |web.WebUIService|: Tez UI History > > URL: http://dev-cv2.aws:8080/tez-ui/#/tez-app/application_1475014682883_0027 > > 2016-09-28 05:11:40,908 [INFO] [main] |history.HistoryEventHandler|: > > Initializing HistoryEventHandler withrecoveryEnabled=true, > > historyServiceClassName=org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService > > 2016-09-28 05:11:41,474 [INFO] [main] |impl.TimelineClientImpl|: Timeline > > service address: http://ts-ip.aws:8188/ws/v1/timeline/ > > 2016-09-28 05:11:41,474 [INFO] [main] |ats.ATSHistoryLoggingService|: > > Initializing ATSHistoryLoggingService with maxEventsPerBatch=5, > > maxPollingTime(ms)=10, waitTimeForShutdown(ms)=-1, > > TimelineACLManagerClass=org.apache.tez.dag.history.ats.acls.ATSHistoryACLPolicyManager > > 2016-09-28 05:11:41,644 [INFO] [main] |impl.TimelineClientImpl|: Timeline > > service address: http://ts-ip.aws:8188/ws/v1/timeline/ > > > > > > >>> DAG Execution > > > > 2016-09-28 05:11:52,779 [INFO] [IPC Server handler 0 on 44039] > > |history.HistoryEventHandler|: > > [HISTORY][DAG:dag_1475014682883_0027_1][Event:DAG_SUBMITTED]: > > dagID=dag_1475014682883_0027_1, submitTime=1475039511185 > > > > > > Timeline server is up and running. Tez UI is however not able to display > > DAG and othe
Re: Debugging M/R job with tez
Hello Manuel, Thanks for reporting the issue. Let me try and reproduce this locally to see what is going on. A quick question in general though - are you hitting issues when running in non-local mode too? Would you mind sharing that details on the issues you hit? thanks — Hitesh > On Sep 27, 2016, at 9:53 AM, Manuel Godbertwrote: > > Hello, > > I have map/reduce jobs that work as expected within YARN, and I want to see > if Tez can help me improving their performance. Alas, I am experiencing > issues and I want to understand what happens, to see if I can adapt my code > or if I can suggest Tez enhancements. For this I need to be able to debug > jobs from within eclipse, with breakpoints in Tez source code etc. > > I am working on a linux (ubuntu) platform > I use the latest Tez version I found, i.e. 0.9.0-SNAPSHOT (also tried with > 0.7.0) > I have set up the hortonworks mini dev cluster > https://github.com/hortonworks/mini-dev-cluster > I am trying to run the basic WordCount2 code found here > https://hadoop.apache.org/docs/r2.7.2/hadoop-mapreduce-client/hadoop-mapreduce-client-core/MapReduceTutorial.html#Example:_WordCount_v2.0 > I added the following code to have tez running locally: > conf.set("mapreduce.framework.name", "yarn-tez"); > conf.setBoolean("tez.local.mode", true); > conf.set("fs.default.name", "file:///"); > conf.setBoolean("tez.runtime.optimize.local.fetch", true); > > And I am getting the following error: > > 2016-09-27 18:32:34 Running Dag: dag_1474992804027_0003_1 > 2016-09-27 18:32:34 Running Dag: dag_1474992804027_0003_1 > Exception in thread "main" java.lang.NullPointerException > at > org.apache.tez.client.LocalClient.getApplicationReport(LocalClient.java:153) > at > org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl.getAppReport(DAGClientRPCImpl.java:231) > at > org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl.createAMProxyIfNeeded(DAGClientRPCImpl.java:251) > at > org.apache.tez.dag.api.client.rpc.DAGClientRPCImpl.getDAGStatus(DAGClientRPCImpl.java:96) > at > org.apache.tez.dag.api.client.DAGClientImpl.getDAGStatusViaAM(DAGClientImpl.java:360) > at > org.apache.tez.dag.api.client.DAGClientImpl.getDAGStatusInternal(DAGClientImpl.java:220) > at > org.apache.tez.dag.api.client.DAGClientImpl.getDAGStatus(DAGClientImpl.java:268) > at > org.apache.tez.dag.api.client.MRDAGClient.getDAGStatus(MRDAGClient.java:58) > at > org.apache.tez.mapreduce.client.YARNRunner.getJobStatus(YARNRunner.java:710) > at > org.apache.tez.mapreduce.client.YARNRunner.submitJob(YARNRunner.java:650) > at > org.apache.hadoop.mapreduce.JobSubmitter.submitJobInternal(JobSubmitter.java:240) > at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1290) > at org.apache.hadoop.mapreduce.Job$10.run(Job.java:1287) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:422) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1657) > at org.apache.hadoop.mapreduce.Job.submit(Job.java:1287) > at org.apache.hadoop.mapreduce.Job.waitForCompletion(Job.java:1308) > at WordCount2.main(WordCount2.java:136) > > Please help me understanding what I am doing wrong! > > Regards
Re: Zip Exception since commit da4098b9
Hello Madhusudan, Thanks for the patience. Let us take this to a jira where once you attach more logs, we can root cause the issue. A few things to attach to the jira: - yarn-site.xml - tez-site.xml - hadoop version - timeline server log for the time period in question - application logs for any tez app which fails to display - output of http://timelinehost:port/ws/v1/timeline/TEZ_DAG_ID// ( e.g. dag_1475014682883_0027_1 ) thanks — Hitesh > On Sep 27, 2016, at 10:42 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > So I downloaded Tez commit 91a397b0ba and built the dist package. We're not > seeing the zip exception anymore. > > However, now Tez UI is completely broken. Not at all sure what is happening > here. Please see attached screenshots. > > > 2016-09-28 05:11:40,903 [INFO] [main] |web.WebUIService|: Tez UI History URL: > http://dev-cv2.aws:8080/tez-ui/#/tez-app/application_1475014682883_0027 > 2016-09-28 05:11:40,908 [INFO] [main] |history.HistoryEventHandler|: > Initializing HistoryEventHandler withrecoveryEnabled=true, > historyServiceClassName=org.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService > 2016-09-28 05:11:41,474 [INFO] [main] |impl.TimelineClientImpl|: Timeline > service address: http://ts-ip.aws:8188/ws/v1/timeline/ > 2016-09-28 05:11:41,474 [INFO] [main] |ats.ATSHistoryLoggingService|: > Initializing ATSHistoryLoggingService with maxEventsPerBatch=5, > maxPollingTime(ms)=10, waitTimeForShutdown(ms)=-1, > TimelineACLManagerClass=org.apache.tez.dag.history.ats.acls.ATSHistoryACLPolicyManager > 2016-09-28 05:11:41,644 [INFO] [main] |impl.TimelineClientImpl|: Timeline > service address: http://ts-ip.aws:8188/ws/v1/timeline/ > > > >>> DAG Execution > > 2016-09-28 05:11:52,779 [INFO] [IPC Server handler 0 on 44039] > |history.HistoryEventHandler|: > [HISTORY][DAG:dag_1475014682883_0027_1][Event:DAG_SUBMITTED]: > dagID=dag_1475014682883_0027_1, submitTime=1475039511185 > > > Timeline server is up and running. Tez UI is however not able to display DAG > and other details > > thanks, > Madhu > > > > On Saturday, September 24, 2016 12:01 PM, Hitesh Shah <hit...@apache.org> > wrote: > > > tez-dist tar balls are not published to maven today - only the module > specific jars are. But yes, you could just try a local build to see if you > can reproduce the issue with the commit in question. > > — Hitesh > > > > On Sep 23, 2016, at 6:23 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > > > Hitesh and Zhiyuan, > > > > Apache snapshots doesn't seem to have tez-dist > > > > http://repository.apache.org/content/groups/snapshots/org/apache/tez/tez-dist/ > > > > The last one seems to be 0.2.0-SNAPSHOT > > > > Should I just download based on the commit and recompile ? > > > > thanks, > > Madhu > > > > > > On Friday, September 23, 2016 5:19 PM, Hitesh Shah <hit...@apache.org> > > wrote: > > > > > > Hello Madhusudan, > > > > If you look at the MANIFEST.MF inside any of the tez jars, it will provide > > the commit hash via the SCM-Revision field. > > > > The tez client and the DAGAppMaster also log this info at runtime. > > > > — Hitesh > > > > > On Sep 23, 2016, at 4:08 PM, Madhusudan Ramanna <m.rama...@ymail.com> > > > wrote: > > > > > > Zhiyuan, > > > > > > We just pulled down the latest snapshot from Apache repository. > > > Question, is how can I figure out branch and commit information from the > > > snapshot artifact ? > > > > > > thanks, > > > Madhu > > > > > > > > > On Friday, September 23, 2016 10:38 AM, zhiyuan yang <sjtu@gmail.com> > > > wrote: > > > > > > > > > Hi Madhu, > > > > > > It looks like a Inflater-Deflater mismatch to me. From stack traces I see > > > you cherry-picked this patch instead of using master branch. > > > Would you mind double check whether the patch is correctly cherry-picked? > > > > > > Thanks! > > > Zhiyuan > > > > > >> On Sep 23, 2016, at 10:21 AM, Madhusudan Ramanna <m.rama...@ymail.com> > > >> wrote: > > >> > > >> Hello, > > >> > > >> We're using the Apache snapshot repository to pull latest tez snapshots. > > >> > > >> We've started seeing this exception: > > >> > > >> org.apache.tez.dag.api.TezUncheckedException: >
Re: Zip Exception since commit da4098b9
tez-dist tar balls are not published to maven today - only the module specific jars are. But yes, you could just try a local build to see if you can reproduce the issue with the commit in question. — Hitesh > On Sep 23, 2016, at 6:23 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > Hitesh and Zhiyuan, > > Apache snapshots doesn't seem to have tez-dist > > http://repository.apache.org/content/groups/snapshots/org/apache/tez/tez-dist/ > > The last one seems to be 0.2.0-SNAPSHOT > > Should I just download based on the commit and recompile ? > > thanks, > Madhu > > > On Friday, September 23, 2016 5:19 PM, Hitesh Shah <hit...@apache.org> wrote: > > > Hello Madhusudan, > > If you look at the MANIFEST.MF inside any of the tez jars, it will provide > the commit hash via the SCM-Revision field. > > The tez client and the DAGAppMaster also log this info at runtime. > > — Hitesh > > > On Sep 23, 2016, at 4:08 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > > > Zhiyuan, > > > > We just pulled down the latest snapshot from Apache repository. Question, > > is how can I figure out branch and commit information from the snapshot > > artifact ? > > > > thanks, > > Madhu > > > > > > On Friday, September 23, 2016 10:38 AM, zhiyuan yang <sjtu@gmail.com> > > wrote: > > > > > > Hi Madhu, > > > > It looks like a Inflater-Deflater mismatch to me. From stack traces I see > > you cherry-picked this patch instead of using master branch. > > Would you mind double check whether the patch is correctly cherry-picked? > > > > Thanks! > > Zhiyuan > > > >> On Sep 23, 2016, at 10:21 AM, Madhusudan Ramanna <m.rama...@ymail.com> > >> wrote: > >> > >> Hello, > >> > >> We're using the Apache snapshot repository to pull latest tez snapshots. > >> > >> We've started seeing this exception: > >> > >> org.apache.tez.dag.api.TezUncheckedException: java.util.zip.ZipException: > >> incorrect header check > >> at > >> org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager.handleVertexManagerEvent(ShuffleVertexManager.java:622) > >> at > >> org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager.onVertexManagerEventReceived(ShuffleVertexManager.java:579) > >> at > >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEventReceived.invoke(VertexManager.java:606) > >> at > >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent$1.run(VertexManager.java:647) > >> at > >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent$1.run(VertexManager.java:642) > >> at java.security.AccessController.doPrivileged(Native Method) > >> at javax.security.auth.Subject.doAs(Subject.java:422) > >> at > >> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) > >> at > >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent.call(VertexManager.java:642) > >> at > >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent.call(VertexManager.java:631) > >> at java.util.concurrent.FutureTask.run(FutureTask.java:266) > >> 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.util.zip.ZipException: incorrect header check > >> at java.util.zip.InflaterInputStream.read(InflaterInputStream.java:164) > >> at java.io.FilterInputStream.read(FilterInputStream.java:107) > >> at org.apache.commons.io.IOUtils.copyLarge(IOUtils.java:1792) > >> at org.apache.commons.io.IOUtils.copyLarge(IOUtils.java:1769) > >> at org.apache.commons.io.IOUtils.copy(IOUtils.java:1744) > >> at org.apache.commons.io.IOUtils.toByteArray(IOUtils.java:462) > >> > >> > >> since this commit > >> > >> https://github.com/apache/tez/commit/da4098b9d6f72e6d4aacc1623622a0875408d2ba > >> > >> > >> Wanted to bring this to your attention. For now we've locked the snapshot > >> version down. > >> > >> thanks, > >> Madhu > > > > > > > >
Re: Zip Exception since commit da4098b9
Hello Madhusudan, If you look at the MANIFEST.MF inside any of the tez jars, it will provide the commit hash via the SCM-Revision field. The tez client and the DAGAppMaster also log this info at runtime. — Hitesh > On Sep 23, 2016, at 4:08 PM, Madhusudan Ramannawrote: > > Zhiyuan, > > We just pulled down the latest snapshot from Apache repository. Question, > is how can I figure out branch and commit information from the snapshot > artifact ? > > thanks, > Madhu > > > On Friday, September 23, 2016 10:38 AM, zhiyuan yang > wrote: > > > Hi Madhu, > > It looks like a Inflater-Deflater mismatch to me. From stack traces I see you > cherry-picked this patch instead of using master branch. > Would you mind double check whether the patch is correctly cherry-picked? > > Thanks! > Zhiyuan > >> On Sep 23, 2016, at 10:21 AM, Madhusudan Ramanna wrote: >> >> Hello, >> >> We're using the Apache snapshot repository to pull latest tez snapshots. >> >> We've started seeing this exception: >> >> org.apache.tez.dag.api.TezUncheckedException: java.util.zip.ZipException: >> incorrect header check >> at >> org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager.handleVertexManagerEvent(ShuffleVertexManager.java:622) >> at >> org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager.onVertexManagerEventReceived(ShuffleVertexManager.java:579) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEventReceived.invoke(VertexManager.java:606) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent$1.run(VertexManager.java:647) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent$1.run(VertexManager.java:642) >> at java.security.AccessController.doPrivileged(Native Method) >> at javax.security.auth.Subject.doAs(Subject.java:422) >> at >> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent.call(VertexManager.java:642) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent.call(VertexManager.java:631) >> at java.util.concurrent.FutureTask.run(FutureTask.java:266) >> 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.util.zip.ZipException: incorrect header check >> at java.util.zip.InflaterInputStream.read(InflaterInputStream.java:164) >> at java.io.FilterInputStream.read(FilterInputStream.java:107) >> at org.apache.commons.io.IOUtils.copyLarge(IOUtils.java:1792) >> at org.apache.commons.io.IOUtils.copyLarge(IOUtils.java:1769) >> at org.apache.commons.io.IOUtils.copy(IOUtils.java:1744) >> at org.apache.commons.io.IOUtils.toByteArray(IOUtils.java:462) >> >> >> since this commit >> >> https://github.com/apache/tez/commit/da4098b9d6f72e6d4aacc1623622a0875408d2ba >> >> >> Wanted to bring this to your attention. For now we've locked the snapshot >> version down. >> >> thanks, >> Madhu > > >
Re: Zip Exception since commit da4098b9
Hello Madhusudan Thanks for reporting the issue. Would you mind filing a bug at https://issues.apache.org/jira/browse/tez with the application logs and tez configs attached? If you have a simple dag/job example that reproduces the behavior that would be great too. thanks — Hitesh > On Sep 23, 2016, at 10:38 AM, zhiyuan yangwrote: > > Hi Madhu, > > It looks like a Inflater-Deflater mismatch to me. From stack traces I see you > cherry-picked this patch instead of using master branch. > Would you mind double check whether the patch is correctly cherry-picked? > > Thanks! > Zhiyuan > >> On Sep 23, 2016, at 10:21 AM, Madhusudan Ramanna wrote: >> >> Hello, >> >> We're using the Apache snapshot repository to pull latest tez snapshots. >> >> We've started seeing this exception: >> >> org.apache.tez.dag.api.TezUncheckedException: java.util.zip.ZipException: >> incorrect header check >> at >> org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager.handleVertexManagerEvent(ShuffleVertexManager.java:622) >> at >> org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager.onVertexManagerEventReceived(ShuffleVertexManager.java:579) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEventReceived.invoke(VertexManager.java:606) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent$1.run(VertexManager.java:647) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent$1.run(VertexManager.java:642) >> at java.security.AccessController.doPrivileged(Native Method) >> at javax.security.auth.Subject.doAs(Subject.java:422) >> at >> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent.call(VertexManager.java:642) >> at >> org.apache.tez.dag.app.dag.impl.VertexManager$VertexManagerEvent.call(VertexManager.java:631) >> at java.util.concurrent.FutureTask.run(FutureTask.java:266) >> 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.util.zip.ZipException: incorrect header check >> at java.util.zip.InflaterInputStream.read(InflaterInputStream.java:164) >> at java.io.FilterInputStream.read(FilterInputStream.java:107) >> at org.apache.commons.io.IOUtils.copyLarge(IOUtils.java:1792) >> at org.apache.commons.io.IOUtils.copyLarge(IOUtils.java:1769) >> at org.apache.commons.io.IOUtils.copy(IOUtils.java:1744) >> at org.apache.commons.io.IOUtils.toByteArray(IOUtils.java:462) >> >> >> since this commit >> >> https://github.com/apache/tez/commit/da4098b9d6f72e6d4aacc1623622a0875408d2ba >> >> >> Wanted to bring this to your attention. For now we've locked the snapshot >> version down. >> >> thanks, >> Madhu >
Re: Parallel queries to HS2/Tez
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) > <uday.chitra...@kantarmedia.com> 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 <hit...@apache.org> > 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) > > <uday.chitra...@kantarmedia.com> 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
Re: Node unable to start vertex
Created https://cwiki.apache.org/confluence/display/TEZ/FAQ which might be a better fit for such content and other related questions down the line. > On Aug 25, 2016, at 1:16 PM, Hitesh Shah <hit...@apache.org> wrote: > > +1. Would you like to contribute the content? You should be able to add an > article under > https://cwiki.apache.org/confluence/display/TEZ/Troubleshooting+articles. > > If you hit any permission issues, feel free to reply back with your > confluence id. > > thanks > — Hitesh > > >> On Aug 25, 2016, at 12:59 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: >> >> Thanks, #2 worked ! >> >> Might be a good idea to add to confluence ? >> >> Madhu >> >> >> On Thursday, August 25, 2016 12:00 PM, Hitesh Shah <hit...@apache.org> wrote: >> >> >> Hello Madhu, >> >> There are 2 approaches for this: >> >> 1) Programmatically, for user code running in tasks, you would need to use >> either DAG::addTaskLocalFiles() or Vertex::addTaskLocalFiles() - former if >> the same jars are needed in all tasks of the DAG. >> TezClient::addAppMasterLocalFiles only impacts the ApplicationMaster. >> >> 2) Configure tez.aux.uris. This will ensure that all files specified here >> will be available in the AM and all tasks. >> >> thanks >> — Hitesh >> >>> On Aug 25, 2016, at 11:46 AM, Madhusudan Ramanna <m.rama...@ymail.com> >>> wrote: >>> >>> Hello, >>> >>> I'm trying to extend TezExamplesBase and get a dag running on yarn (pseudo >>> cluster mode on my host). >>> >>> For some reason, I'm running into class not found exception on the node >>> >>> Vertex failed, vertexName=v1, vertexId=vertex_1471907702278_0030_1_00, >>> diagnostics=[Task failed, taskId=task_1471907702278_0030_1_00_00, >>> diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( >>> failure ) : >>> attempt_1471907702278_0030_1_00_00_0:org.apache.tez.dag.api.TezReflectionException: >>> Unable to load class: sample.sampletez.OnetoOne$V1Processor >>> at org.apache.tez.common.ReflectionUtils.getClazz(ReflectionUtils.java:46) >>> at >>> org.apache.tez.common.ReflectionUtils.createClazzInstance(ReflectionUtils.java:87) >>> at >>> org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.createProcessor(LogicalIOProcessorRuntimeTask.java:668) >>> >>> >>> Should I be adding my application jar somewhere so that it can get >>> distributed ? I tried adding my jar via tezClient.addAppMasterLocalFiles() >>> but it didn't help. >>> >>> What am I not doing ? >>> >>> thanks! >>> Madhu >>> >>> >>> >>> >>> >> >> >
Re: Node unable to start vertex
+1. Would you like to contribute the content? You should be able to add an article under https://cwiki.apache.org/confluence/display/TEZ/Troubleshooting+articles. If you hit any permission issues, feel free to reply back with your confluence id. thanks — Hitesh > On Aug 25, 2016, at 12:59 PM, Madhusudan Ramanna <m.rama...@ymail.com> wrote: > > Thanks, #2 worked ! > > Might be a good idea to add to confluence ? > > Madhu > > > On Thursday, August 25, 2016 12:00 PM, Hitesh Shah <hit...@apache.org> wrote: > > > Hello Madhu, > > There are 2 approaches for this: > > 1) Programmatically, for user code running in tasks, you would need to use > either DAG::addTaskLocalFiles() or Vertex::addTaskLocalFiles() - former if > the same jars are needed in all tasks of the DAG. > TezClient::addAppMasterLocalFiles only impacts the ApplicationMaster. > > 2) Configure tez.aux.uris. This will ensure that all files specified here > will be available in the AM and all tasks. > > thanks > — Hitesh > > > On Aug 25, 2016, at 11:46 AM, Madhusudan Ramanna <m.rama...@ymail.com> > > wrote: > > > > Hello, > > > > I'm trying to extend TezExamplesBase and get a dag running on yarn (pseudo > > cluster mode on my host). > > > > For some reason, I'm running into class not found exception on the node > > > > Vertex failed, vertexName=v1, vertexId=vertex_1471907702278_0030_1_00, > > diagnostics=[Task failed, taskId=task_1471907702278_0030_1_00_00, > > diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( > > failure ) : > > attempt_1471907702278_0030_1_00_00_0:org.apache.tez.dag.api.TezReflectionException: > > Unable to load class: sample.sampletez.OnetoOne$V1Processor > > at org.apache.tez.common.ReflectionUtils.getClazz(ReflectionUtils.java:46) > > at > > org.apache.tez.common.ReflectionUtils.createClazzInstance(ReflectionUtils.java:87) > > at > > org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.createProcessor(LogicalIOProcessorRuntimeTask.java:668) > > > > > > Should I be adding my application jar somewhere so that it can get > > distributed ? I tried adding my jar via tezClient.addAppMasterLocalFiles() > > but it didn't help. > > > > What am I not doing ? > > > > thanks! > > Madhu > > > > > > > > > > > >
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
Re: Node unable to start vertex
Hello Madhu, There are 2 approaches for this: 1) Programmatically, for user code running in tasks, you would need to use either DAG::addTaskLocalFiles() or Vertex::addTaskLocalFiles() - former if the same jars are needed in all tasks of the DAG. TezClient::addAppMasterLocalFiles only impacts the ApplicationMaster. 2) Configure tez.aux.uris. This will ensure that all files specified here will be available in the AM and all tasks. thanks — Hitesh > On Aug 25, 2016, at 11:46 AM, Madhusudan Ramannawrote: > > Hello, > > I'm trying to extend TezExamplesBase and get a dag running on yarn (pseudo > cluster mode on my host). > > For some reason, I'm running into class not found exception on the node > > Vertex failed, vertexName=v1, vertexId=vertex_1471907702278_0030_1_00, > diagnostics=[Task failed, taskId=task_1471907702278_0030_1_00_00, > diagnostics=[TaskAttempt 0 failed, info=[Error: Error while running task ( > failure ) : > attempt_1471907702278_0030_1_00_00_0:org.apache.tez.dag.api.TezReflectionException: > Unable to load class: sample.sampletez.OnetoOne$V1Processor > at org.apache.tez.common.ReflectionUtils.getClazz(ReflectionUtils.java:46) > at > org.apache.tez.common.ReflectionUtils.createClazzInstance(ReflectionUtils.java:87) > at > org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.createProcessor(LogicalIOProcessorRuntimeTask.java:668) > > > Should I be adding my application jar somewhere so that it can get > distributed ? I tried adding my jar via tezClient.addAppMasterLocalFiles() > but it didn't help. > > What am I not doing ? > > thanks! > Madhu > > > > >
Re: Questions about Tez
When comparing just a simple MR job to a Tez dag with 2 vertices, the perf improvements are limited (as the plan is pretty much the same and data is transferred via a shuffle edge): - container re-use - pipelined sorter vs the MR sorter ( your mileage may vary here depending on the kind of work load ) - auto-reduce parallelism - dynamic splits grouping for the map vertex ( assuming splits are calculated in the client ) For the second question, the dag plan/structure and the processor are both user-code and therefore which output to write to is driven by user/business logic. If you write a tee processor, it could write to all outputs if needed. A processor which re-partitions data to different downstream vertices could be coded to write diff data to each output if needed. The MapProcessor and ReduceProcessor assume MR semantics which means that they always assume one input and one output (likewise for the WordCount example). thanks — Hitesh > On Aug 12, 2016, at 9:54 AM, Madhusudan Ramannawrote: > > Hello all, > > I've just started looking at Tez. I've setup Tez locally and have run sample > MapReduce job with Tez as a replacement for Yarn MRV2. I plan to use Tez > independently (No hive or pig). I have the following questions > > 1. Are there performance gains to use Tez for Mapreduce jobs apart from > container reuse ? Want to make sure I'm not missing anything > > 2. More technically, from what should be the behavior of a processor while > generating outputs. Should it write to all KeyValueWriters returned from > getOutputs() in the processor. The WordCount example retrieves an output by > name and writes to it. > > thanks, > Madhu
Re: Some resource about tez architecture and design document
If you want understand Tez in terms of how it is used by Hive or Pig, it might be best to send emails to the dev lists of hive/pig as needed. If you have any tez specific questions, you can take a look at the links I mentioned and example code under tez-examples. thanks — Hitesh > On Aug 10, 2016, at 9:56 AM, darion.yaphet <fly...@163.com> wrote: > > Hi Hitesh thank you for you relay . AFAIK Tez is the execute engine > about hive (It's also a series of API to build DAG). So when I want > to understand Tez I have to had a good know about Hive execute plan and AST > generation ? > 在 2016-08-10 01:02:31,"Hitesh Shah" <hit...@apache.org> 写道: >> The following 2 links should help you get started. Might be best to start >> with the sigmod paper and one of the earlier videos. >> >> https://cwiki.apache.org/confluence/display/TEZ/How+to+Contribute+to+Tez >> https://cwiki.apache.org/confluence/display/TEZ/Presentations%2C+publications%2C+and+articles+about+Tez >> >> thanks >> — Hitesh >> >>> On Aug 9, 2016, at 8:03 AM, darion.yaphet <fly...@163.com> wrote: >>> >>> Hi team : >>> I'm a beginner to learn tez source code . Is there any resource tez >>> architecture and design document to introduce it ? >>> thanks >>> >>> >>> >>
Re: Some resource about tez architecture and design document
The following 2 links should help you get started. Might be best to start with the sigmod paper and one of the earlier videos. https://cwiki.apache.org/confluence/display/TEZ/How+to+Contribute+to+Tez https://cwiki.apache.org/confluence/display/TEZ/Presentations%2C+publications%2C+and+articles+about+Tez thanks — Hitesh > On Aug 9, 2016, at 8:03 AM, darion.yaphetwrote: > > Hi team : > I'm a beginner to learn tez source code . Is there any resource tez > architecture and design document to introduce it ? > thanks > > >
Re: Word Count examples run failed with Tez 0.8.4
Hello I am assuming that this is the same issue as the one reported in TEZ-3396? Based on the logs in the jira: 2016-08-03 10:55:33,856 [INFO] [Thread-2] |app.DAGAppMaster|: DAGAppMasterShutdownHook invoked 2016-08-03 10:55:33,856 [INFO] [Thread-2] |app.DAGAppMaster|: DAGAppMaster received a signal. Signaling TaskScheduler It seems like the AM is getting killed. Can you provide the configs being used for: - tez.am.resource.memory.mb - tez.am.launch.cmd-opts You should also check the NodeManager logs for container_1470148111230_0011_01_01. It might shed light on whether the NM killed the AM for exceeding memory limits. thanks — Hitesh > On Aug 3, 2016, at 8:50 PM, HuXiwrote: > > Default configuration was used with yarn.resourcemanager.hostname set to > 0.0.0.0 and yarn.resourcemanager.address set to 0.0.0.0:8032. > > If what you mentioned is really the reason, please tell me what I should do > to fix it? > > > > Date: Wed, 3 Aug 2016 20:41:31 -0700 > > Subject: Re: Word Count examples run failed with Tez 0.8.4 > > From: gop...@apache.org > > To: user@tez.apache.org > > > > > > > 16/08/04 09:36:00 INFO client.TezClient: The url to track the Tez AM: > > >http://iZ25f2qedc7Z:8088/proxy/application_1470148111230_0014/ > > > 16/08/04 09:36:05 INFO client.RMProxy: Connecting to ResourceManager at > > >/0.0.0.0:8032 > > > > That sounds very strange - is the resource manager really running on > > localhost, but that resolves back to that strange hostname? > > > > Cheers, > > Gopal > > > > > > > > > > > >
Re: hung AM due to timeline timeout
It might worth filing a YARN jira to get it backported to 2.6.x and 2.7.x. At the very least, it will simplify rebuilding the timeline-server jar against the CDH version that you are running. — Hitesh > On Aug 3, 2016, at 4:42 PM, Slava Markeyev <slava.marke...@upsight.com> wrote: > > Thanks for the info Hitesh. Unfortunately it seems that RollingLevelDB is > only in trunk. I may have to backport it to 2.6.2 (version I use). I did > notice that the leveldb does grow to tens of gb which may be an indication of > pruning not happening often enough (or at all?). I also need to fix the > logging as the logs for the timeline server don't seem to be very active > beyond it starting up. > > For the job I posted before here is the associated eventQueueBacklog log line. > 2016-08-03 19:23:27,932 [INFO] [AMShutdownThread] > |ats.ATSHistoryLoggingService|: Stopping ATSService, eventQueueBacklog=17553 > I'll look into lowering tez.yarn.ats.event.flush.timeout.millis while trying > to look into the timelineserver. > > Thanks for your help, > Slava > > On Wed, Aug 3, 2016 at 2:45 PM, Hitesh Shah <hit...@apache.org> wrote: > Hello Slava, > > Can you check for a log line along the lines of "Stopping ATSService, > eventQueueBacklog=“ to see how backed up is the event queue to YARN timeline? > > I have noticed this in quite a few installs with YARN Timeline where YARN > Timeline is using the simple Level DB impl and not the RollingLevelDB storage > class. The YARN timeline ends up hitting some bottlenecks around the time > when the data purging happens ( takes a global lock on level db ). The > Rolling level db storage impl solved this problem by using separate level dos > for different time intervals and just throwing out the level db instead of > trying to do a full scan+purge. > > Another workaround though not a great one is to set > “tez.yarn.ats.event.flush.timeout.millis” to a value say 6 i.e. 1 min. > This implies that the Tez AM will try for at max 1 min to flush the queue to > YARN timeline before giving up and shutting down the Tez AM. > > A longer term option is the YARN Timeline version 1.5 work currently slated > to be released in hadoop 2.8.0 which uses HDFS for writes instead of the > current web service based approach. This has a far better perf throughput for > writes albeit with a delay on the read path as the Timeline server scans HDFS > for new updates. The tez changes for this are already available in the source > code under the hadoop28 profile though the documentation for this is still > pending. > > thanks > — Hitesh > > > > > > > On Aug 3, 2016, at 2:02 PM, Slava Markeyev <slava.marke...@upsight.com> > > wrote: > > > > I'm running into an issue that occurs fairly often (but not consistently > > reproducible) where yarn reports a negative value for memory allocation eg > > (-2048) and a 0 vcore allocation despite the AM actually running. For > > example the AM reports a runtime of 1hrs, 29mins, 40sec while the dag only > > 880 seconds. > > > > After some investigating I've noticed that the AM has repeated issues > > contacting the timeline server after the dag is complete (error trace > > below). This seems to be delaying the shutdown sequence. It seems to retry > > every minute before either giving up or succeeding but I'm not sure which. > > What's the best way to debug why this would be happening and potentially > > shortening the timeout retry period as I'm more concerned with job > > completion than logging it to the timeline server. This doesn't seem to be > > happening consistently to all tez jobs only some. > > > > I'm using hive 1.1.0 and tez 0.7.1 on cdh5.4.10 (hadoop 2.6). > > > > 2016-08-03 19:18:22,881 [INFO] [ContainerLauncher #112] > > |impl.ContainerManagementProtocolProxy|: Opening proxy : node:45454 > > 2016-08-03 19:18:23,292 [WARN] [HistoryEventHandlingThread] > > |security.UserGroupInformation|: PriviledgedActionException as:x > > (auth:SIMPLE) cause:java.net.SocketTimeoutException: Read timed out > > 2016-08-03 19:18:23,292 [ERROR] [HistoryEventHandlingThread] > > |impl.TimelineClientImpl|: Failed to get the response from the timeline > > server. > > com.sun.jersey.api.client.ClientHandlerException: > > java.net.SocketTimeoutException: Read timed out > > at > > com.sun.jersey.client.urlconnection.URLConnectionClientHandler.handle(URLConnectionClientHandler.java:149) > > at > > org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl$TimelineJerseyRetryFilter$1.run(TimelineClientImpl.java:226) > > at
Re: Guide to write map-reduce code using Tez API
Please check Step 7 on http://tez.apache.org/install.html thanks — Hitesh > On Aug 1, 2016, at 10:25 AM, zhiyuan yangwrote: > > The nice thing of Tez is it’s compatible with MapReduce API. So if you just > want to run MapReduce on Tez, you just learn how to write standard MapReduce > and change the execution engine to Tez. > > To change the execution engine of MapReduce, please change the configuration > mapreduce.framework.name. (Not 100% percent sure about this, correct me if > I’m wrong) > > Thanks! > Zhiyuan > >> On Aug 1, 2016, at 10:19 AM, Sudhir.Kumar wrote: >> >> Hello All, >> >> I have just started to read about Tez. >> >> Is there a document to understand the Tez Java APIs which can be used to >> write map-reduce code. >> >> Thanks, >> >> Sudhir >
Re: Tez error
If your cluster has a running YARN timeline server, setting up the UI is quite straightforward: http://tez.apache.org/tez-ui.html — Hitesh > On Jul 30, 2016, at 10:21 AM, Sandeep Khurana <skhurana...@gmail.com> wrote: > > Hitesh > > Tons of thanks. I am able to run Tez job from a hive query on mapR cluster. > I.e. hive query now fires a tez job on mapR. > > As pointed out by you I made 2 mistakes. Not configuring mapR repository in > tez pom.xml, somehow I thought I had done it but .. Second I was not using > minimal tar ball instead was using full tar, now using minimal tar. > > With these 2 changes, things worked. Also, I ran tez-examples.jar with hadoop > jar command, this also worked and executed a tez job. > > Now I will have to read tez docs to find out if there is a way to see the tez > graph in a tez view on mapR like I see in Ambari on HDP? > > > > > On Sat, Jul 30, 2016 at 10:25 PM, Sandeep Khurana <skhurana...@gmail.com> > wrote: > Sorry, by mistake sent half written message. > > I will try both now and update this thread. > > About trying tez bundle, I tried with the steps at > http://doc.mapr.com/display/MapR41/Installing+and+Configuring+Tez+0.5.3 .But > hive job gave some NumberFormatError and found out by googling that there is > version mismatch between tez and hadoop libs. > > On Sat, Jul 30, 2016 at 10:22 PM, Sandeep Khurana <skhurana...@gmail.com> > wrote: > Hitesh > > Both of things > > On Sat, Jul 30, 2016 at 10:20 PM, Hitesh Shah <hit...@apache.org> wrote: > Hello Sandeep, > > 2 things to check: >- When compiling Tez, is the hadoop.version in the top-level pom ( and > addition of mapr’s maven repo ) being used to compile against MapR’s hadoop > distribution and not the std. apache release? The Tez AM cannot seem to do a > handshake with the YARN RM. If MapR changed anything in that code path, this > issue could crop up. >- For tez.lib.uris, are you using the minimal tez tarball and not the full > one? The full one has hadoop jars in it so it should not be used in your > deployment setup as you are using MapR specific jars. You may want to check > for jar clashes in terms of the content of the tarball vs the jars from the > cluster. The only ones we do bundle in the minimal one are some mapreduce > jars I believe. > > FWIW, I think MapR does provide a Tez bundle for their clusters. You may wish > to check on their user lists too to see if they made some changes. If you do > find out anything, please do let us know in case this requires some fixes in > Tez for better compatibility across all distros. > > thanks > — Hitesh > > > > > On Jul 30, 2016, at 8:55 AM, Sandeep Khurana <skhurana...@gmail.com> wrote: > > > > Hello > > > > > > On mapR single node cluster, I compiled Tez as per the documentation on > > apache tez site. > > > > The only thing I changed from documentation was in tez-site.xml, where I > > gave the value of tez.use.cluster.hadoop-libs as true. I did it because > > mapR has its own jars for hadoop components. And without these tez might > > not work. > > > > When I run any MR job or hive query I get this below error in the job on RM > > link. (DIGEST-MD5: digest response format violation. Mismatched URI: > > default/; expecting: null/default) > > > > I am using mapr 4.1 and tez 0.7.0. mapR 4.1 uses hive 1.2.0. > > > > Any pointer or suggestion what might be causing this issue or where should > > I look? > > > > 2016-07-30 09:58:38,737 INFO [main] ipc.Server: Stopping server on 45868 > > 2016-07-30 09:58:38,737 INFO [IPC Server Responder] ipc.Server: Stopping > > IPC Server Responder > > 2016-07-30 09:58:38,739 INFO [IPC Server listener on 45868] ipc.Server: > > Stopping IPC Server listener on 45868 > > 2016-07-30 09:58:38,739 INFO [IPC Server Responder] ipc.Server: Stopping > > IPC Server Responder > > 2016-07-30 09:58:38,739 ERROR [main] app.DAGAppMaster: Error starting > > DAGAppMaster > > org.apache.tez.dag.api.TezUncheckedException: > > javax.security.sasl.SaslException: DIGEST-MD5: digest response format > > violation. Mismatched URI: default/; expecting: null/default [Caused by > > org.apache.hadoop.ipc.RemoteException(javax.security.sasl.SaslException): > > DIGEST-MD5: digest response format violation. Mismatched URI: default/; > > expecting: null/default] > > at > > org.apache.tez.dag.app.rm.YarnTaskSchedulerService.serviceStart(YarnTaskSchedulerService.java:384) > > at > > org.apache.hadoop.service.Abstra
Re: Tez error
Hello Sandeep, 2 things to check: - When compiling Tez, is the hadoop.version in the top-level pom ( and addition of mapr’s maven repo ) being used to compile against MapR’s hadoop distribution and not the std. apache release? The Tez AM cannot seem to do a handshake with the YARN RM. If MapR changed anything in that code path, this issue could crop up. - For tez.lib.uris, are you using the minimal tez tarball and not the full one? The full one has hadoop jars in it so it should not be used in your deployment setup as you are using MapR specific jars. You may want to check for jar clashes in terms of the content of the tarball vs the jars from the cluster. The only ones we do bundle in the minimal one are some mapreduce jars I believe. FWIW, I think MapR does provide a Tez bundle for their clusters. You may wish to check on their user lists too to see if they made some changes. If you do find out anything, please do let us know in case this requires some fixes in Tez for better compatibility across all distros. thanks — Hitesh > On Jul 30, 2016, at 8:55 AM, Sandeep Khuranawrote: > > Hello > > > On mapR single node cluster, I compiled Tez as per the documentation on > apache tez site. > > The only thing I changed from documentation was in tez-site.xml, where I gave > the value of tez.use.cluster.hadoop-libs as true. I did it because mapR has > its own jars for hadoop components. And without these tez might not work. > > When I run any MR job or hive query I get this below error in the job on RM > link. (DIGEST-MD5: digest response format violation. Mismatched URI: > default/; expecting: null/default) > > I am using mapr 4.1 and tez 0.7.0. mapR 4.1 uses hive 1.2.0. > > Any pointer or suggestion what might be causing this issue or where should I > look? > > 2016-07-30 09:58:38,737 INFO [main] ipc.Server: Stopping server on 45868 > 2016-07-30 09:58:38,737 INFO [IPC Server Responder] ipc.Server: Stopping IPC > Server Responder > 2016-07-30 09:58:38,739 INFO [IPC Server listener on 45868] ipc.Server: > Stopping IPC Server listener on 45868 > 2016-07-30 09:58:38,739 INFO [IPC Server Responder] ipc.Server: Stopping IPC > Server Responder > 2016-07-30 09:58:38,739 ERROR [main] app.DAGAppMaster: Error starting > DAGAppMaster > org.apache.tez.dag.api.TezUncheckedException: > javax.security.sasl.SaslException: DIGEST-MD5: digest response format > violation. Mismatched URI: default/; expecting: null/default [Caused by > org.apache.hadoop.ipc.RemoteException(javax.security.sasl.SaslException): > DIGEST-MD5: digest response format violation. Mismatched URI: default/; > expecting: null/default] > at > org.apache.tez.dag.app.rm.YarnTaskSchedulerService.serviceStart(YarnTaskSchedulerService.java:384) > at > org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) > at > org.apache.tez.dag.app.rm.TaskSchedulerEventHandler.serviceStart(TaskSchedulerEventHandler.java:353) > at > org.apache.hadoop.service.AbstractService.start(AbstractService.java:193) > at > org.apache.tez.dag.app.DAGAppMaster$ServiceWithDependency.start(DAGAppMaster.java:1573) > at > org.apache.tez.dag.app.DAGAppMaster$ServiceThread.run(DAGAppMaster.java:1591) > Caused by: javax.security.sasl.SaslException: DIGEST-MD5: digest response > format violation. Mismatched URI: default/; expecting: null/default [Caused > by org.apache.hadoop.ipc.RemoteException(javax.security.sasl.SaslException): > DIGEST-MD5: digest response format violation. Mismatched URI: default/; > expecting: null/default] > at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) > at > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) > at > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) > at java.lang.reflect.Constructor.newInstance(Constructor.java:526) > at > org.apache.hadoop.yarn.ipc.RPCUtil.instantiateException(RPCUtil.java:53) > at > org.apache.hadoop.yarn.ipc.RPCUtil.unwrapAndThrowException(RPCUtil.java:104) > at > org.apache.hadoop.yarn.api.impl.pb.client.ApplicationMasterProtocolPBClientImpl.registerApplicationMaster(ApplicationMasterProtocolPBClientImpl.java:109) > 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.$Proxy24.registerApplicationMaster(Unknown Source) > at >
Re: Could Tez 0.5.4 Integrate with Hive 2.X
That is highly unlikely to work as Hive-2.x requires APIs introduced in Tez 0.8.x. thanks — Hitesh > On Jul 28, 2016, at 8:56 PM, darion.yaphetwrote: > > Hi team : > > We are using hadoop 2.5.0 and hive 1.2.1 tez 0.5.4 . Now we want to upgrade > to hive 2.X . Could Tez 0.5.4 support Hive 2.X ? > > thanks ~~ > > >
Re: Getting ClosedByInterruptException when DAG w/ edge executes
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 McCartywrote: > > 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: Tez Job Counters
Hello Muhammad, Did you try any of the calls to YARN timeline as described by Rajesh in his earlier reply? thanks — Hitesh > On Jun 28, 2016, at 1:20 PM, Muhammad Haris >wrote: > > Hi, > Could anybody please guide me how to get all task level counters? Thanks a lot > > > > Regards > > On Mon, Jun 27, 2016 at 5:43 PM, Muhammad Haris > wrote: > Hi Rajesh, > Thank you for quick help > > > Regards > > On Mon, Jun 27, 2016 at 4:56 PM, Rajesh Balamohan > wrote: > You can refer to Tez-UI which makes use of these APIs extensively. > https://tez.apache.org/tez-ui.html provides > details on setting this up. > > AppID: > http://atsmachine:8088/ws/v1/cluster/apps/application_1466689310983_0024 > > Tez DAG: > http://atsmachine:8188/ws/v1/timeline/TEZ_DAG_ID?limit=10 > http://atsmachine:8188/ws/v1/timeline/TEZ_DAG_ID/dag_1466689310983_0021_409 > > This should have the counters in the otherInfo --> counters --> counterGroups > > Tez vertex: > http://atsmachine:8188/ws/v1/timeline/TEZ_VERTEX_ID?limit=10=TEZ_DAG_ID:dag_1466689310983_0023_408 > > Tez task: > http://atsmachine:8188/ws/v1/timeline/TEZ_TASK_ID?limit=1000=TEZ_DAG_ID:dag_1466689310983_0023_408 > > Tez task attempt: > http://atsmachine:8188/ws/v1/timeline/TEZ_TASK_ATTEMPT_ID?limit=1000=TEZ_DAG_ID:dag_1466689310983_0023_408 > > ~Rajesh.B > > On Mon, Jun 27, 2016 at 3:50 AM, Muhammad Haris > wrote: > Hi, > I have implemented an application that runs Hive jobs and pulls its job > counters using Job History server REST API, Job History server REST API that > i am using is: > http:// address:port>/ws/v1/history/mapreduce/jobs/{jobid}/counters > > I have been looking for similar REST API to get job counters for Hive on Tez > jobs, i understand Tez jobs history is pushed into YARN timeline server but i > failed to find any API in YARN timeline server through which i can pull the > job counters. > > Any help/suggestions will be highly appreciated > Regards > Haris Akhtar > > > >
Re: Tez 0.8.3 on EMR hanging with Hive task
If log aggregation is not enabled, the next best thing would be to download the application master logs from the RM UI for the apps in question. Those would provide a good starting point for figuring out what is going on. thanks — HItesh > On Jun 15, 2016, at 8:29 AM, Jose Rozanec <jose.roza...@mercadolibre.com> > wrote: > > Hello, > > We provide an update. Seems we understood something wrong: hive returned us > an error in the query, while Tez job was running not reporting progress. We > did not cancel it, since seemed that it hanged. After two hours reported as > finished on the UI; while still held running state when listed from YARN for > some time more and finished finally finished. > We have log aggregation enabled, but after the job finished, we still get the > same message as reported in the previous email. > > Now will research why Hive detached from Tez while still running; and if we > can improve query accept times, since is taking a while to start executing > complex queries. > > Thanks, > > > > > 2016-06-15 12:09 GMT-03:00 Jose Rozanec <jose.roza...@mercadolibre.com>: > Hello, > > I ran the command, and got the following message: > 16/06/15 15:07:35 INFO impl.TimelineClientImpl: Timeline service address: > http://ip-10-64-23-215.ec2.internal:8188/ws/v1/timeline/ > 16/06/15 15:07:35 INFO client.RMProxy: Connecting to ResourceManager at > ip-10-64-23-215.ec2.internal/10.64.23.215:8032 > /var/log/hadoop-yarn/apps/hadoop/logs/application_1465996511770_0001 does not > exist. > Log aggregation has not completed or is not enabled. > > I think we are missing some configuration that would help us get more insight? > > Thanks! > > Joze. > > 2016-06-15 12:03 GMT-03:00 Hitesh Shah <hit...@apache.org>: > Hello Joze, > > Would it be possible for you to provide the YARN application logs obtained > via “bin/yarn logs -applicationId ” for both of the cases you have > seen? Feel free to file JIRAs and attach logs to each of them. > > thanks > — Hitesh > > > On Jun 15, 2016, at 7:38 AM, Jose Rozanec <jose.roza...@mercadolibre.com> > > wrote: > > > > Hello, > > > > We are experiencing some issues with Tez 0.8.3 when we issue heavy queries > > from Hive. Seems some jobs hang on Tez and never return. Those jobs show up > > in the DAG web-ui, but no progress is reported on UI nor on Hive logs. Any > > ideas why this could happen? We detect happens with certain memory > > configurations, which if missing, the job dies soon (we guess due to OOM). > > > > Most probably not related to this, at some point we also got the following > > error: "org.apache.tez.dag.api.SessionNotRunning: TezSession has already > > shutdown. Application x failed 2 times due to AM Container". We are not > > sure can be related to TEZ-2663, which should be solved since version 0.7.1 > > onwards. > > > > Thanks in advance, > > > > Joze. > > >
Re: Tez 0.8.3 on EMR hanging with Hive task
Hello Joze, Would it be possible for you to provide the YARN application logs obtained via “bin/yarn logs -applicationId ” for both of the cases you have seen? Feel free to file JIRAs and attach logs to each of them. thanks — Hitesh > On Jun 15, 2016, at 7:38 AM, Jose Rozanec> wrote: > > Hello, > > We are experiencing some issues with Tez 0.8.3 when we issue heavy queries > from Hive. Seems some jobs hang on Tez and never return. Those jobs show up > in the DAG web-ui, but no progress is reported on UI nor on Hive logs. Any > ideas why this could happen? We detect happens with certain memory > configurations, which if missing, the job dies soon (we guess due to OOM). > > Most probably not related to this, at some point we also got the following > error: "org.apache.tez.dag.api.SessionNotRunning: TezSession has already > shutdown. Application x failed 2 times due to AM Container". We are not > sure can be related to TEZ-2663, which should be solved since version 0.7.1 > onwards. > > Thanks in advance, > > Joze.
Re: My first TEZ job fails
Sorry - I forgot to call that out in the instructions as the config templates are being generated only from 0.8.x onwards. Will update the wiki. — Hitesh > On May 20, 2016, at 4:51 PM, Mich Talebzadeh <mich.talebza...@gmail.com> > wrote: > > With reference to > > ... then do an export TEZ_CONF_DIR="apache-tez-{x.y.z}/conf/" > > By they way there is no conf directory under apache-tez-0.7.1-bin! There are > only lib, and share > > hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin> ltr|grep '^d' > drwxr-xr-x 2 hduser hadoop4096 May 20 22:58 share > drwxr-xr-x 2 hduser hadoop4096 May 20 22:58 lib > drwxr-xr-x 85 root root 40960 May 20 22:58 .. > drwxr-xr-x 4 hduser hadoop4096 May 20 23:18 . > hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin> > > Dr Mich Talebzadeh > > LinkedIn > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > > http://talebzadehmich.wordpress.com > > > On 21 May 2016 at 00:42, Mich Talebzadeh <mich.talebza...@gmail.com> wrote: > Please bear in mind that all I want is to test Hive with TEZ engine. I > already have Hive working OK with Spark 1.3.1 engine and I compiled it spark > from source code. so hopefully I can use TEZ as Spark engine as well. > > > thanks > > > > Dr Mich Talebzadeh > > LinkedIn > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > > http://talebzadehmich.wordpress.com > > > On 21 May 2016 at 00:39, Mich Talebzadeh <mich.talebza...@gmail.com> wrote: > This is the instruction? > > Created by Hitesh Shah, last modified on May 02, 2016 Go to start of metadata > Making use of the Tez Binary Release tarball > > • If the binary tarball's name does not include anything referring to a > hadoop version, then this implies that the tarball was compiled against the > hadoop version that the Tez release compiles against by default. For example, > for 0.7.0 and 0.8.0, the default hadoop version used is 2.6.0 ( this can be > found by looking for the hadoop.version property in the top-level pom.xml in > the source tarball for the release). > > • The tarball structure is as follows: > > > ? > apache-tez-{x.y.z}/ > /tez*.jar > /lib/*.jar > /conf/tez*.xml.template > /share/tez.tar.gz > • Set up Tez by following INSTALL.txt and use > apache-tez-{x.y.z}/share/tez.tar.gz as the full tarball to be uploaded to > HDFS. > • Use the config templates under apache-tez-{x.y.z}/conf/ to create the > tez-site.xml as needed in an appropriate conf directory. If you end up using > apache-tez-{x.y.z}/conf/, then do an export > TEZ_CONF_DIR="apache-tez-{x.y.z}/conf/" > • Add "apache-tez-{x.y.z}/*:apache-tez-{x.y.z}/lib/*:${TEZ_CONF_DIR}" > to HADOOP_CLASSPATH so as to get the tez client jars onto the classpath > invoked when using the "bin/hadoop jar" command to run an example job. > > > Dr Mich Talebzadeh > > LinkedIn > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > > http://talebzadehmich.wordpress.com > > > On 21 May 2016 at 00:37, Mich Talebzadeh <mich.talebza...@gmail.com> wrote: > Thanks both > > so this is the file that needs to go in hdfs correct? > > hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin/share> ltr tez-0.7.1.tar.gz > -rw-r--r-- 1 hduser hadoop 39694439 May 4 18:47 tez-0.7.1.tar.gz > > > In hdfs I have now > > hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin/share> hdfs dfs -ls > /usr/lib/apache-tez-0.7.1-bin > > -rw-r--r-- 2 hduser supergroup 39694439 2016-05-21 00:31 > /usr/lib/apache-tez-0.7.1-bin/tez-0.7.1.tar.gz > > > Now I only installed tez under > > /usr/lib/apache-tez-0.7.1-bin > > MY Hadoop is installed in > > echo $HADOOP_HOME > /home/hduser/hadoop-2.6.0 > > and my xml-site in in > > $HADOOP_HOME/etc/hadoop > > OK and this is my sml-site content > > hduser@rhes564: /home/hduser/hadoop-2.6.0/etc/hadoop> cat tez-site.xml > > > tez.version > 0.7.1 > > > tez.lib.uris > > /usr/lib/apache-tez-0.7.1-bin,/usr/lib/apache-tez-0.7.1-bin/lib > > > > Is the red correct please? > > > > Dr Mich Talebzadeh > > LinkedIn > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > > http://talebzadehmich.wordpress.com > > > On 21 May 2016 at 00:24, Bikas Saha <bi...@apache.org> wrote: > >> tez.lib.uris assumes that paths
Re: My first TEZ job fails
Sorry - missed the fact that the /usr/lib paths were also being used on HDFS. In any case, the contents of /usr/lib/apache-tez-0.7.1-bin,/usr/lib/apache-tez-0.7.1-bin/lib will be only from the minimal tarball and therefore will not work out of the box as the relevant hadoop-yarn jars would be missing. The Release FAQ wiki has the correct instructions on how to use the binary release. — Hitesh > On May 20, 2016, at 4:24 PM, Bikas Saha <bi...@apache.org> wrote: > >>> tez.lib.uris assumes that paths are based on the default fs and therefore >>> if your setup is using HDFS as default, the paths /usr/lib would be invalid > > Are you sure? The below paths looks right to me except that the contents of > the directories are wrong. > > tez.lib.uris > /usr/lib/apache-tez-0.7.1-bin,/usr/lib/apache-tez-0.7.1-bin/lib > > hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin> hdfs dfs -ls > /usr/lib/apache-tez-0.7.1-bin Found 2 items > -rw-r--r-- 2 hduser supergroup 53092828 2016-05-20 23:15 > /usr/lib/apache-tez-0.7.1-bin/apache-tez-0.7.1-bin.tar.gz > drwxr-xr-x - hduser supergroup 0 2016-05-20 23:27 > /usr/lib/apache-tez-0.7.1-bin/lib > > > -Original Message- > From: Hitesh Shah [mailto:hit...@apache.org] > Sent: Friday, May 20, 2016 4:18 PM > To: user@tez.apache.org > Subject: Re: My first TEZ job fails > > Can you try the instructions mentioned at > https://cwiki.apache.org/confluence/display/TEZ/Tez+Release+FAQ ? > > tez.lib.uris assumes that paths are based on the default fs and therefore if > your setup is using HDFS as default, the paths /usr/lib would be invalid. > > — HItesh > >> On May 20, 2016, at 3:39 PM, Mich Talebzadeh <mich.talebza...@gmail.com> >> wrote: >> >> Still failing with /apache-tez-0.7.1-bin I am afraid. >> >> OK this is my tez-site.xml >> >> hduser@rhes564: /home/hduser/hadoop-2.6.0/etc/hadoop> cat tez-site.xml >> >> >>tez.version >>0.7.1 >> >> >>tez.lib.uris >> >> /usr/lib/apache-tez-0.7.1-bin,/usr/lib/apache-tez-0.7.1-bin/lib >> >> >> >> This is what I have put in hdfs directory >> >> hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin> hdfs dfs -ls >> /usr/lib/apache-tez-0.7.1-bin Found 2 items >> -rw-r--r-- 2 hduser supergroup 53092828 2016-05-20 23:15 >> /usr/lib/apache-tez-0.7.1-bin/apache-tez-0.7.1-bin.tar.gz >> drwxr-xr-x - hduser supergroup 0 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib >> >> Also I put all /usr/lib/apache-tez-0.7.1-bin/lib/*.jar in >> /usr/lib/apache-tez-0.7.1-bin/lib >> >> hduser@rhes564: /usr/lib/apache-tez-0.7.1-bin> hdfs dfs -ls >> /usr/lib/apache-tez-0.7.1-bin/lib Found 22 items >> -rw-r--r-- 2 hduser supergroup 124846 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/RoaringBitmap-0.4.9.jar >> -rw-r--r-- 2 hduser supergroup 41123 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-cli-1.2.jar >> -rw-r--r-- 2 hduser supergroup 58160 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-codec-1.4.jar >> -rw-r--r-- 2 hduser supergroup 588337 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-collections-3.2.2.jar >> -rw-r--r-- 2 hduser supergroup 751238 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-collections4-4.1.jar >> -rw-r--r-- 2 hduser supergroup 185140 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-io-2.4.jar >> -rw-r--r-- 2 hduser supergroup 284220 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-lang-2.6.jar >> -rw-r--r-- 2 hduser supergroup1599627 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/commons-math3-3.1.1.jar >> -rw-r--r-- 2 hduser supergroup1648200 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/guava-11.0.2.jar >> -rw-r--r-- 2 hduser supergroup 664918 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/hadoop-mapreduce-client-common-2.6.0.jar >> -rw-r--r-- 2 hduser supergroup1509399 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/hadoop-mapreduce-client-core-2.6.0.jar >> -rw-r--r-- 2 hduser supergroup 130458 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/jersey-client-1.9.jar >> -rw-r--r-- 2 hduser supergroup 147952 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/jersey-json-1.9.jar >> -rw-r--r-- 2 hduser supergroup 81743 2016-05-20 23:27 >> /usr/lib/apache-tez-0.7.1-bin/lib/jettison-1.3.4.jar >> -rw-r--r-- 2 hduser supergroup
Re: My first TEZ job fails
.auth.Subject$ClassSet.populateSet(Subject.java:1409) > > at javax.security.auth.Subject$ClassSet.(Subject.java:1369) > > at javax.security.auth.Subject.getPublicCredentials(Subject.java:720) > > at > org.apache.hadoop.security.UserGroupInformation.getTokenIdentifiers(UserGroupInformation.java:1400) > > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.selectNMTokenIdentifier(ContainerManagerImpl.java:618) > > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.startContainers(ContainerManagerImpl.java:699) > > at > org.apache.hadoop.yarn.api.impl.pb.service.ContainerManagementProtocolPBServiceImpl.startContainers(ContainerManagementProtocolPBServiceImpl.java:60) > > at > org.apache.hadoop.yarn.proto.ContainerManagementProtocol$ContainerManagementProtocolService$2.callBlockingMethod(ContainerManagementProtocol.java:95) > > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619) > > at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962) > > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039) > > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035) > > at java.security.AccessController.doPrivileged(Native Method) > > at javax.security.auth.Subject.doAs(Subject.java:422) > > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) > > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2033) > > at org.apache.hadoop.ipc.Client.call(Client.java:1468) > > 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.$Proxy80.startContainers(Unknown Source) > > at > org.apache.hadoop.yarn.api.impl.pb.client.ContainerManagementProtocolPBClientImpl.startContainers(ContainerManagementProtocolPBClientImpl.java:96) > > at > org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.launch(AMLauncher.java:119) > > at > org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.run(AMLauncher.java:254) > > 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) > > . Failing the application. > > Thanks > > > > > > > > > > > > Dr Mich Talebzadeh > > LinkedIn > https://www.linkedin.com/profile/view?id=AAEWh2gBxianrbJd6zP6AcPCCdOABUrV8Pw > > http://talebzadehmich.wordpress.com > > > > On 20 May 2016 at 19:06, Hitesh Shah <hit...@apache.org> wrote: > > Logs from `bin/yarn logs -applicationId application_1463758195355_0002` would > be more useful to debug your setup issue. The RM logs usually do not shed > much light on why an application failed. > Can you confirm that you configured tez.lib.uris correctly to point to the > tez tarball on HDFS (tez tar should be the one obtained from > tez-dist/target/tez-0.8.3.tar.gz) ? > > — Hitesh > > > > On May 20, 2016, at 10:24 AM, Mich Talebzadeh <mich.talebza...@gmail.com> > > wrote: > > > > Hi, > > > > I have just compiled and installed TEZ, trying to do a test with > > > > hadoop jar ./tez-examples-0.8.3.jar orderedwordcount /tmp/input/test.txt > > /tmp/out > > > > The job fails as follows. This is from yarn log > > > > 2016-05-20 18:19:26,945 INFO SecurityLogger.org.apache.hadoop.ipc.Server: > > Auth successful for appattempt_1463758195355_0002_01 (auth:SIMPLE) > > 2016-05-20 18:19:26,950 WARN org.apache.hadoop.ipc.Server: IPC Server > > handler 0 on 59093, call > > org.apache.hadoop.yarn.api.ContainerManagementProtocolPB.startContainers > > from 50.140.197.217:46784 Call#2 Retry#0 > > java.lang.NoSuchMethodError: > > org.apache.hadoop.yarn.proto.YarnProtos$ApplicationIdProto.hashLong(J)I > > at > > org.apache.hadoop.yarn.proto.YarnProtos$ApplicationIdProto.hashCode(YarnProtos.java:2616) > > at > > org.apache.hadoop.yarn.proto.YarnProtos$ApplicationAttemptIdProto.hashCode(YarnProtos.java:3154) > > at > > org.apache.hadoop.yarn.proto.YarnSecurityTokenProtos$NMTokenIdentifierProto.hashCode(YarnSecurityTokenProtos.java:410) > > at > > org.apache.hadoop.yarn.security.NMTokenIdentifier.hashCode(NMTokenIdentifier.java:126) > > at java.util.HashMap.hash(HashMap.java:338) > > at java.util.HashMap.put(HashMap.java:611) &g
Re: data discrepancies related to parallelism
Thanks for the info, Kurt. You may wish to post this question to the Pig lists too to see if anyone has seen this. — Hitesh > On May 5, 2016, at 11:05 AM, Kurt Muehlner <kmuehl...@connexity.com> wrote: > > Hi Hitesh, > > We are using Pig 0.15.0 and Tez 0.8.2. > > Thanks, > Kurt > > > > On 5/5/16, 11:00 AM, "Hitesh Shah" <hit...@apache.org> wrote: > >> What version are you running with? >> >> thanks >> — Hitesh
Re: data discrepancies related to parallelism
What version are you running with? thanks — Hitesh > On May 5, 2016, at 10:31 AM, Kurt Muehlnerwrote: > > Hello, > > We have a Pig/Tez application which is exhibiting a strange problem. This > application was recently migrated from Pig/MR to Pig/Tez. We carefully > vetted during QA that both MR and Tez versions produced identical results. > However, after deploying to production, we noticed that occasionally, results > are not the same (either as compared to MR results, or results of Tez > processing the same data on a QA cluster). > > We’re still looking into the root cause, but I’d like to reach out to the > user group in case anyone has seen anything similar, or has suggestions on > what might be wrong/what to investigate. > > *** What we know so far *** > Results discrepancy occurs ONLY when the number of containers given to the > application by YARN is less than the number requested (we have disabled > auto-parallelism, and are using SET_DEFAULT_PARALLEL=50 in all pig scripts). > When this occurs, we also see a corresponding discrepancy in the the file > system counters HDFS_READ_OPS and HDFS_BYTES_READ (lower when number of > containers is low), despite the fact that in all cases number of records > processed is identical. > > Thus, when the production cluster is very busy, we get invalid results. We > have kept a separate instance of the Pig/Tez application running on another > cluster where it never competes for resources, so we have been able to > compare results for each run of the application, which has allowed us to > diagnose the problem this far. By comparing results on these two clusters, > we also know that the ratio (actual HDFS_READ_OPS)/(expected HDFS_READ_OPS) > correlates with the ratio (actual containers)/(requested containers). > Likewise, we see the same correlation between hdfs ops ratio and container > ratio. > > Below are some relevant counters. For each counter, the first line is the > value from the production cluster showing the problem, and the second line is > the value from the QA cluster running on the same data. > > Any hints/suggestions/questions are most welcome. > > Thanks, > Kurt > > org.apache.tez.common.counters.DAGCounter > > NUM_SUCCEEDED_TASKS=950 > NUM_SUCCEEDED_TASKS=950 > > TOTAL_LAUNCHED_TASKS=950 > TOTAL_LAUNCHED_TASKS=950 > > File System Counters > > FILE_BYTES_READ=7745801982 > FILE_BYTES_READ=8003771938 > > FILE_BYTES_WRITTEN=9725468612 > FILE_BYTES_WRITTEN=9675253887 > > *HDFS_BYTES_READ=9487600888 (when number of containers equals the number > requested, this counter is the same between the two clusters) > *HDFS_BYTES_READ=17996466110 > > *HDFS_READ_OPS=3080 (when number of containers equals the number requested, > this counter is the same between the two clusters) > *HDFS_READ_OPS=3600 > > HDFS_WRITE_OPS=900 > HDFS_WRITE_OPS=900 > > org.apache.tez.common.counters.TaskCounter > INPUT_RECORDS_PROCESSED=28729671 > INPUT_RECORDS_PROCESSED=28729671 > > > OUTPUT_RECORDS=33655895 > OUTPUT_RECORDS=33655895 > > OUTPUT_BYTES=28290888628 > OUTPUT_BYTES=28294000270 > > Input(s): > Successfully read 2254733 records (1632743360 bytes) from: "input1" > Successfully read 2254733 records (1632743360 bytes) from: "input1" > > > Output(s): > Successfully stored 0 records in: “output1” > Successfully stored 0 records in: "output1” > > Successfully stored 56019 records (10437069 bytes) in: “output2” > Successfully stored 56019 records (10437069 bytes) in: "output2” > > Successfully stored 2254733 records (1651936175 bytes) in: "output3” > Successfully stored 2254733 records (1651936175 bytes) in: "output3” > > Successfully stored 1160599 records (823479742 bytes) in: "output4” > Successfully stored 1160599 records (823480450 bytes) in: "output4” > > Successfully stored 28605 records (21176320 bytes) in: "output5” > Successfully stored 28605 records (21177552 bytes) in: "output5” > > Successfully stored 6574 records (4442933 bytes) in: "output6” > Successfully stored 6574 records (4442933 bytes) in: "output6” > > Successfully stored 111416 records (164375858 bytes) in: "output7” > Successfully stored 111416 records (164379800 bytes) in: "output7” > > Successfully stored 542 records (387761 bytes) in: "output8” > Successfully stored 542 records (387762 bytes) in: "output8" > > > > > > > > > > > > > > > >
Re: Description of tez counters
Take a look at TaskCounter and DAGCounter under https://git-wip-us.apache.org/repos/asf?p=tez.git;a=tree;f=tez-api/src/main/java/org/apache/tez/common/counters;h=df3784e54d1fa6075dcbbca8d1405e309bce1460;hb=HEAD and let us know if that is insufficient. thanks — Hitesh On Apr 11, 2016, at 4:42 AM, Nitin Kumarwrote: > Hi! > > I was going through the counters made available by the TEZ view in HDP > distribution (2.3.0). I tried to find descriptions for each of the available > counters but could not get a comprehensive list. I got the description for a > few of them in "Hadoop - The Definitive Guide 4th edition". > > I would be highly obliged if someone could direct me to a list of the > description for tez counters. > > Thanks and regards, > Nitin
Re: Unable To Build Apache Tez 0.7.1 on Hadoop 2.7
We have been seeing some recent failures due to some form of npm registry errors even though no code has changed ( https://issues.apache.org/jira/browse/TEZ-3201 ) - is this what you are seeing? BIGTOP-1826 is a different issue. That is due to the frontend-plugin not being compatible with certain versions of maven ( 0.0.22 is not compatible with mvn 3.3 ).This was fixed in Tez by switching the frontend plugin version based on the mvn version in use. If you check the top-level pom, it is defaulting to 0.0.23 but falls back to 0.0.22 if you have an older version of mvn. That said, a simpler approach for your case is to remove tez-ui from the module list at the top-level and re-compile ( i.e. not build tez-ui module). The tez-ui.war is not really used by the tez runtime but only used as part of the Tez UI. For this, you can download the war from https://repository.apache.org/content/groups/public/org/apache/tez/tez-ui/ and use as needed if you have the UI deployed. thanks — Hitesh On Apr 6, 2016, at 2:47 PM, Sam Joe <games2013@gmail.com> wrote: > Thanks for your valuable suggestion. I used -fae option to skip the errors > and proceed with the ones which is working. > sudo mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true > -Dfrontend-maven-plugin.version=0.0.22 -fae > > Now I'm stuck with only the following 2 errors: > > [ERROR] Failed to execute goal > com.github.eirslett:frontend-maven-plugin:0.0.16:npm (npm install) on project > tez-ui: Failed to run task: 'npm install --color=false' failed. (error code > 34) -> [Help 1] > [ERROR] Failed to execute goal > org.apache.maven.plugins:maven-assembly-plugin:2.4:single (package-tez) on > project tez-dist: Failed to create assembly: Artifact: > org.apache.tez:tez-ui:war:0.7.0 (included by module) does not have an > artifact with a file. Please ensure the package phase is run before the > assembly is generated. -> [Help 2] > > > [INFO] tez ... SUCCESS [3.861s] > [INFO] tez-api ... SUCCESS [37.493s] > [INFO] tez-common SUCCESS [3.843s] > [INFO] tez-runtime-internals . SUCCESS [5.211s] > [INFO] tez-runtime-library ... SUCCESS [9.675s] > [INFO] tez-mapreduce . SUCCESS [11.038s] > [INFO] tez-examples .. SUCCESS [1.355s] > [INFO] tez-dag ... SUCCESS [21.797s] > [INFO] tez-tests . SUCCESS [4.109s] > [INFO] tez-ui FAILURE [1:12.348s] > [INFO] tez-plugins ... SUCCESS [0.057s] > [INFO] tez-yarn-timeline-history . SUCCESS [1.816s] > [INFO] tez-yarn-timeline-history-with-acls ... SUCCESS [1.292s] > [INFO] tez-mbeans-resource-calculator SUCCESS [0.640s] > [INFO] tez-tools . SUCCESS [0.148s] > [INFO] tez-dist .. FAILURE [8.107s] > [INFO] Tez ... SUCCESS [0.063s] > > For the npm error I see a open JIRA : > https://issues.apache.org/jira/browse/BIGTOP-1826 > > Do you have any suggestion? > > Thanks. > > > On Wed, Apr 6, 2016 at 4:13 PM Hitesh Shah <hit...@apache.org> wrote: > Not sure why you are hitting those timeouts. Are you running this on a > small-sized VM which may be impacting the tests? Running locally on my > laptop, I don’t seem to be having any issues doing a mvn test on 0.7.0 > against hadoop-2.7.0. > > One option would be to just do a fail-at-end and then re-run the failed tests > separately to see if they are just flaky on your env. > > thanks > — Hitesh > > On Apr 6, 2016, at 12:48 PM, Sam Joe <games2013@gmail.com> wrote: > > > Hi Hitesh, > > > > My environment: Hadoop 2.7.0 > > As suggested by you I'm trying with apache-tez-0.7.0-src > > > > I've to use the following command to avoid npm and Dfrontend missing error: > > > > sudo mvn clean package -Dfrontend-maven-plugin.version=0.0.22 > > > > I keep getting timeout error while building Tez as shown in the error log > > below: > > > > --- > > Test set: org.apache.tez.client.TestTezClient > > --- > > Tests run: 8, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 9.013 sec > > <<< FAILURE! > >
Re: Unable To Build Apache Tez 0.7.1 on Hadoop 2.7
Not sure why you are hitting those timeouts. Are you running this on a small-sized VM which may be impacting the tests? Running locally on my laptop, I don’t seem to be having any issues doing a mvn test on 0.7.0 against hadoop-2.7.0. One option would be to just do a fail-at-end and then re-run the failed tests separately to see if they are just flaky on your env. thanks — Hitesh On Apr 6, 2016, at 12:48 PM, Sam Joe <games2013@gmail.com> wrote: > Hi Hitesh, > > My environment: Hadoop 2.7.0 > As suggested by you I'm trying with apache-tez-0.7.0-src > > I've to use the following command to avoid npm and Dfrontend missing error: > > sudo mvn clean package -Dfrontend-maven-plugin.version=0.0.22 > > I keep getting timeout error while building Tez as shown in the error log > below: > > --- > Test set: org.apache.tez.client.TestTezClient > --- > Tests run: 8, Failures: 0, Errors: 1, Skipped: 0, Time elapsed: 9.013 sec <<< > FAILURE! > testTezclientSession(org.apache.tez.client.TestTezClient) Time elapsed: > 5.062 sec <<< ERROR! > java.lang.Exception: test timed out after 5000 milliseconds > > (Earlier I was able to build apache-tez-0.5.0-src in the same environment > however had to upgrade to 0.7.0 in order to avoid some bugs fixed in 0.7.0 > version). > > Please help. > > > Thanks, > Joel > > On Wed, Apr 6, 2016 at 3:01 PM Hitesh Shah <hit...@apache.org> wrote: > Hello Sam, > > Couple of things to confirm: > - I assume you are building branch-0.7 of Tez for 0.7.1-SNAPSHOT as there > has not yet been a release of 0.7.1? > - For hadoop, are you using hadoop-2.7.0 or hadoop-2.7.1 ( though this > really should not be too relevant here )? > > I took branch-0.7 of the current code base and compiled it against > hadoop-2.7.1. I also ran the test that was failing and it did not fail. > > Picked up _JAVA_OPTIONS: -Djava.awt.headless=true > Running org.apache.tez.dag.app.TestMockDAGAppMaster > 2016-04-06 11:31:18.076 java[20695:1903] Unable to load realm info from > SCDynamicStore > Tests run: 17, Failures: 0, Errors: 0, Skipped: 3, Time elapsed: 5.788 sec > > I think this might be a flaky test that we need to fix. I would suggest > filing a bug for this with the surefire logs for the test in question so that > it will help us debug the flaky failure. Please update the jira with hadoop > version too. If this test is consistently failing for you, let us know ( if > not, a quick fix would be to just patch the code with an @Ignore to skip the > test ) > > thanks > — Hitesh > > > On Apr 6, 2016, at 10:57 AM, Sam Joe <games2013@gmail.com> wrote: > > > I am using the following command for building Tez. > > > > sudo mvn clean package -Dfrontend-maven-plugin.version=0.0.22 > > > > Thanks, > > Joel > > > > On Wed, Apr 6, 2016 at 1:51 PM, Sam Joe <games2013@gmail.com> wrote: > > Hi, > > > > Could any help me in resolving the following error which I am getting while > > building Apache Tez 0.7.1 on Hadoop 2.7: > > > > testCommitOutputOnDAGSuccess(org.apache.tez.dag.app.TestMockDAGAppMaster) > > Time elapsed: 0.453 sec <<< ERROR! > > org.apache.tez.dag.api.TezException: App master already running a DAG > > at > > org.apache.tez.dag.app.DAGAppMaster.submitDAGToAppMaster(DAGAppMaster.java:1256) > > > > I can see that there is a Jira found in the following link but not sure how > > to use the fix. > > > > https://issues.apache.org/jira/browse/TEZ-2307 > > > > Please help. > > > > Thanks, > > Sam > > >
Re: Unable To Build Apache Tez 0.7.1 on Hadoop 2.7
Hello Sam, Couple of things to confirm: - I assume you are building branch-0.7 of Tez for 0.7.1-SNAPSHOT as there has not yet been a release of 0.7.1? - For hadoop, are you using hadoop-2.7.0 or hadoop-2.7.1 ( though this really should not be too relevant here )? I took branch-0.7 of the current code base and compiled it against hadoop-2.7.1. I also ran the test that was failing and it did not fail. Picked up _JAVA_OPTIONS: -Djava.awt.headless=true Running org.apache.tez.dag.app.TestMockDAGAppMaster 2016-04-06 11:31:18.076 java[20695:1903] Unable to load realm info from SCDynamicStore Tests run: 17, Failures: 0, Errors: 0, Skipped: 3, Time elapsed: 5.788 sec I think this might be a flaky test that we need to fix. I would suggest filing a bug for this with the surefire logs for the test in question so that it will help us debug the flaky failure. Please update the jira with hadoop version too. If this test is consistently failing for you, let us know ( if not, a quick fix would be to just patch the code with an @Ignore to skip the test ) thanks — Hitesh On Apr 6, 2016, at 10:57 AM, Sam Joewrote: > I am using the following command for building Tez. > > sudo mvn clean package -Dfrontend-maven-plugin.version=0.0.22 > > Thanks, > Joel > > On Wed, Apr 6, 2016 at 1:51 PM, Sam Joe wrote: > Hi, > > Could any help me in resolving the following error which I am getting while > building Apache Tez 0.7.1 on Hadoop 2.7: > > testCommitOutputOnDAGSuccess(org.apache.tez.dag.app.TestMockDAGAppMaster) > Time elapsed: 0.453 sec <<< ERROR! > org.apache.tez.dag.api.TezException: App master already running a DAG > at > org.apache.tez.dag.app.DAGAppMaster.submitDAGToAppMaster(DAGAppMaster.java:1256) > > I can see that there is a Jira found in the following link but not sure how > to use the fix. > > https://issues.apache.org/jira/browse/TEZ-2307 > > Please help. > > Thanks, > Sam >
Re: Apply patches to Apache Tez
Apache releases are officially source-only releases. Some projects do provide binary jars for convenience but that depends on the project. Additionally, these will only be available for “releases” only and not for each and every patch applied on a branch. In your case, the only option is to download the source code for the release in question ( http://tez.apache.org/releases/index.html ). Download the patch file from JIRA and apply the patch against the source code. Build the source and deploy as explained in my previous email. If you are willing to live with the downtime, you can re-build the code and replace the jars/tarball in place for the current ones ( new locations are needed if you want to do a live/rolling upgrade with no downtime - however the new location approach will also allow you to do some testing to verify correctness of your newly applied patches ). Additionally, for cases like this, feel free to ask/push the project community for making a new release of 0.7.1 to make your life a bit simpler. 0.7.1 has been pending on our plate for quite some time and we have been a bit lax on making a release for it. thanks — Hitesh On Apr 6, 2016, at 11:41 AM, Sam Joe <games2013@gmail.com> wrote: > Hi Hitesh, > > That surely helps! > > However, how do I apply the .patch file to existing releases. For example, > Tez 0.7.0 has a bug which has been fixed through a JIRA with a .patch file > provided. No new set of jars are provided. > > How do I apply that .patch file to my existing setup of jars? > > Appreciate your help and time. > > Thanks, > Joel > > On Wed, Apr 6, 2016 at 2:28 PM, Hitesh Shah <hit...@apache.org> wrote: > Every component has a different approach to how it is deployed/upgraded. > > I can cover how you can go about patching Tez on an existing production > system. The steps should be similar to that described in INSTALL.md in the > source tree with a few minor gotchas to be aware of: > >- Deploying Tez has 2 aspects: > - installing the client jars on the local filesystem which can then be > added to the class paths of various components such as Hive/Pig, etc that use > Tez. These components need the tez-api, tez-common, tez-mapreduce, > tez-runtime-library jars in their classpath for the most part ( this set is > bundled as tez-minimal tarball in tez-dist when you build Tez ). The > classpath manipulation is usually done by adding the tez jars to > HADOOP_CLASSPATH. > - installing the tez tarball on HDFS and configuring the configs to > point to the location of the tez tarball on HDFS. > > Usually most bugs/patches tend to get applied to tez-dag and > tez-runtime-internals so for the most part you will likely only need to patch > the tez tarball. If you are moving to a new version, both the client side and > HDFS tarball need to be upgraded as there is an in-built check to ensure that > both sides are consistent/compatible. > > To upgrade client side jars, it should be a simple option to install the new > jars in an appropriate location and modifying HADOOP_CLASSPATH to point to > the new location. Likewise for the tez tarball - upload the new tarball to a > new location and modify configs to point to the new location. The exact steps > would be the following: >1) Upload new tez tarball to new location on HDFS >2) Backup tez configs to a new tez config dir and modify tez.lib.uris to > point to the new tarball location >3) Install new tez client side jars. >4) Update HADOOP_CLASSPATH to contain location of new tez client jars as > well as new tez config dir > > What the above does is ensure that existing jobs do not start failing in > between while things are being upgraded. As long as the old tarball is not > deleted while old jobs are runnings, existing jobs should not fail. New jobs > submitted with the new HADOOP_CLASSPATH will pick up the newly deployed bits. > > Hope that helps > — Hitesh > > > On Apr 6, 2016, at 10:34 AM, Sam Joe <games2013@gmail.com> wrote: > > > Hi, > > > > How do you apply patches to Tez or any other Hadoop component? For example > > if there is a bug in the existing classes used in a Hadoop component and > > it's resolved in a Jira, how do you apply that patch to the existing > > on-premise Hadoop setup? I think we should use Git but don't know the exact > > steps to do that. Please help. > > > > > > Thanks, > > Sam > >
Re: Tez UI in Pig
Hi Kurt, The Tez UI as documented should work with any version beyond 0.5.2 if the history logging is configured to use YARN timeline. As for scopes, some bits of the vertex description are currently not displayed in the UI though I am not sure if Pig has integrated with that API yet. Depending on the version of hadoop you are running and the scale at which you are running, there are some known issues with the YARN timeline impl from a scalability perspective but the Yahoo folks have implemented some fixes/config workarounds to get around those. @Jon Eagles, any chance of publishing a wiki for the configs that you recommend running with for YARN Timeline with the level db impl? ( and also the HDFS based impl though that this is not really available in any hadoop release as of now ). If you are trying out the UI, it would be good if you also try out tez-ui2 as it has some enhancements coming down the pipe such as a vertex swim lane which provides a better overall view of the vertices and how they progress/time they took. The UI2 version is fairly new so feedback will be highly appreciated. @Rohini, has Pig started setting the vertex info? @Sreenath, do we have an open jira for the vertex description to be displayed in the UI? thanks — Hitesh On Apr 5, 2016, at 11:04 AM, Kurt Muehlnerwrote: > I have a question about the availability of the Tez web UI in Pig on Tez. > The Pig ‘Performance and Efficiency’ doc states, "Tez specific GUI is not > available yet, there is no GUI to track task progress. However, log message > is available in GUI.” What does this mean, precisely? We have not deployed > and configured the Tez UI described here: > https://tez.apache.org/tez-ui.html. Will that UI work when running Tez on > Pig? If so, what does ‘Tez specific GUI is not available yet’ mean? > > What I am most specifically concerned about is the ability to see which Pig > aliases are being assigned to which Tez vertices, or failing that, which Pig > aliases are being processed by a particular Tez DAG. This is currently not > available in logs in pig 0.15.0, although I’m aware it is in master. > > What are best practices for Pig 0.15.0? > > Thanks, > Kurt
Re: tez and beeline and hs2
As Gopal mentioned, it is optional. The log message could probably be set to a WARN to not confuse users. — Hitesh On Feb 25, 2016, at 7:26 AM, Stephen Sprague <sprag...@gmail.com> wrote: > hey guys, > still not getting jobs to the running state via tez->beeline->hs2. > > lemme ask this first: Is is mandatory that the Tez UI be up an running for > this to work? > > this looks to be a hard error (see below) but given the "local cli" works i > don't think its *mandatory*. But maybe via HS2 it is? > > from yarn logs: > 2016-02-25 07:07:56,744 [ERROR] [main] |web.WebUIService|: Tez UI History URL > is not set > > Thanks, > Stephen > PS setting up the web service is just another step and complication i'm > hoping to avoid for just testing. :) > > On Mon, Feb 22, 2016 at 10:43 AM, Bikas Saha <bi...@apache.org> wrote: > Hi Stephen, > > > > Thanks for bearing with any delays on our side and keeping us updated. In the > end if we figure out that this is less about bugs in Tez/Hive and more about > lack of well documented best practices, then it would be useful to produce a > wiki page about this. > > > > Thanks > > Bikas > > > > From: Stephen Sprague [mailto:sprag...@gmail.com] > Sent: Monday, February 22, 2016 6:59 AM > To: user@tez.apache.org > Subject: Re: tez and beeline and hs2 > > > > just an update. i haven't given up! i've just been pulled into other things > this weekend and am hoping to pick it up again this week. > > > > On Fri, Feb 19, 2016 at 9:30 AM, Hitesh Shah <hit...@apache.org> wrote: > > Not exactly. I think the UI bits might be a red-herring. Bouncing YARN and > HS2 also should unlikely be needed unless you are modifying configs. > > There is likely a bug ( the NPE being logged ) in the shutting down code for > the org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService ( if > it was not started properly ) but the fact that it is shutting down means > that there is something else wrong ( which should have happened before the > shutdown sequence kicked in ). Feel free to file a bug with the logs attached > if you cannot attach them over the mailing list here. > > thanks > — Hitesh > > > > On Feb 19, 2016, at 7:16 AM, Stephen Sprague <sprag...@gmail.com> wrote: > > > Hi Gopal, > > nice. I should have known that command as you and Hitesh have given that > > advice in the past on various threads here! sorry for that. > > > > And sure enough the smoking gun reveals itself. thank you. > > > > {quote} > > 2016-02-18 18:40:19,672 [ERROR] [main] |web.WebUIService|: Tez UI History > > URL is not set > > 2016-02-18 18:40:19,731 [WARN] > > [ServiceThread:org.apache.tez.dag.history.HistoryEventHandler] > > |service.AbstractService|: When stopping the service > > org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService : > > java.lang.NullPointerException > > java.lang.NullPointerException > > {quote} > > > > So it looks like this Tez UI app is required for 'jdbc' mode. Lemme > > research that puppy and perhaps bounce Yarn and HS2. > > > > Thanks again for shining the light! > > > > Cheeers, > > Stephen. > > > > On Thu, Feb 18, 2016 at 10:09 PM, Gopal Vijayaraghavan <gop...@apache.org> > > wrote: > > > > > > > >http://dwrdevnn1.sv2.trulia.com:8088/proxy/application_1455811467110_0307/ > > > > > ><http://dwrdevnn1.sv2.trulia.com:8088/proxy/application_1455811467110_0307 > > >/> > > ... > > > So my question is whatya suppose is causing this? I'm pretty darn sure > > >the classpath is legit now. > > > > Two steps forward, one step back :) > > > > yarn logs -applicationId application_1455811467110_0307 for answers. > > > > Cheers, > > Gopal > > > > > > > > > >
Re: jansi dependendency?
One option may be to try using HADOOP_USER_CLASSPATH_FIRST with it set to true and adding the hive-exec.jar to the front of HADOOP_CLASSPATH. Using this ( and verifying by running “hadoop classpath”), you could try to get hive-exec.jar to the front of the classpath and see if that makes a difference to the class loading order. — Hitesh On Feb 15, 2016, at 4:43 PM, Stephen Spraguewrote: > Hi Jan, > ahhh. I see. my $HIVE_HOME/lib/hive-exec-1.2.1.jar does indeed have > TezJobMonitor in it. So this has really nothing to do with the server side > Tez jar files i built as part of the install instructions but rather the hive > client side code. > > got it. I'll dig away at that! > > Thank you! > > Cheers, > Stephen. > > On Mon, Feb 15, 2016 at 2:18 PM, Jan Morlock > wrote: > Hi Stephen, > > the code inside org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor intends > to use org.fusesource.jansi.Ansi. The Project hive-exec (where > TezJobMonitor is part of) therefore has a dependency on jline (where > Ansi is a part of). However on your class path a jar file containing an > older version of Ansi is found first. This version lacks the bold() > method and therefore the NoSuchMethodError is thrown. > > What you have to do is identifying the jar file containing the older > version of Ansi. Afterwards you have to organize the class path in a way > that the more up-to-date version is found first. > > I hope this helps. > Cheers > Jan > > > Am 15.02.2016 um 20:27 schrieb Stephen Sprague: > > hey guys, > > I'm looking to run Hive on Tez and have followed the instructions to a > > tee - but i just can't seem to get around this Jansi error despite > > everything i've tried. Now given i'm not exactly a Java programmer what > > may appear to you as something pretty trivial i'm at an impasse - but > > not for lack of trying! > > > > would anyone here have any tips? > > > > thanks, > > Stephen > > > > PS Here's my traceback and logging. > > > > $ hive > > SLF4J: Class path contains multiple SLF4J bindings. > > SLF4J: Found binding in > > [jar:file:/usr/lib/zookeeper/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class] > > SLF4J: Found binding in > > [jar:file:/home/spragues/downloads/apache-tez-0.7.0-src/tez-dist/target/tez-0.7.0-minimal/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class] > > SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an > > explanation. > > SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory] > > SLF4J: Class path contains multiple SLF4J bindings. > > SLF4J: Found binding in > > [jar:file:/usr/lib/zookeeper/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class] > > SLF4J: Found binding in > > [jar:file:/home/spragues/downloads/apache-tez-0.7.0-src/tez-dist/target/tez-0.7.0-minimal/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class] > > SLF4J: See http://www.slf4j.org/codes.html#multiple_bindings for an > > explanation. > > SLF4J: Actual binding is of type [org.slf4j.impl.Log4jLoggerFactory] > > > > Logging initialized using configuration in > > jar:file:/usr/lib/hive-1.2.1-standalone/lib/hive-common-1.2.1.jar!/hive-log4j.properties > > *hive> set hive.execution.engine=tez;* > > > > *hive> select count(*) from omniture.hit_data where date_key=20160210;* > > Query ID = spragues_20160215111912_f4b6bc39-d29d-42bb-b0bc-262f8c99f58c > > Total jobs = 1 > > Launching Job 1 out of 1 > > > > > > Status: Running (Executing on YARN cluster with App id > > application_1453472707474_6031) > > > > > > java.lang.NoSuchMethodError: > > org.fusesource.jansi.Ansi.bold()Lorg/fusesource/jansi/Ansi; > > at > > org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor.reprintLineWithColorAsBold(TezJobMonitor.java:205) > > at > > org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor.printStatusInPlace(TezJobMonitor.java:611) > > at > > org.apache.hadoop.hive.ql.exec.tez.TezJobMonitor.monitorExecution(TezJobMonitor.java:320) > > at > > org.apache.hadoop.hive.ql.exec.tez.TezTask.execute(TezTask.java:168) > > at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:160) > > at > > org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:88) > > at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1653) > > at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1412) > > at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1195) > > at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1059) > > at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1049) > > at > > org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:213) > > at > > org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:165) > > at > >
Re: Failing attemption at org.apache.tez.client.TezClient.waitTillReady
It seems to me that the Tez AM classpath somehow has a hadoop-common jar that does not have the Configuration.addDeprecations() api that YARN needs. For the Tez AM, the classpath is fully constructed based on the tez tarball ( from HDFS using distributed cache ) and additional jars that Hive adds ( hive-exec.jar, etc ). It does not use HADOOP_CLASSPATH or anything else from the cluster nodes. HADOOP_CLASSPATH is only used on the client node where the hive shell runs. Can you confirm that hive was also compiled against hadoop-2.6.0 as that might be pulling in a different version of hadoop-common? thanks — Hitesh On Feb 12, 2016, at 12:16 AM, no jihun <jees...@gmail.com> wrote: > Thanks Hitesh Shah. > > It claims > > 2016-02-12 14:59:07,388 [ERROR] [main] |app.DAGAppMaster|: Error starting > DAGAppMaster > > java.lang.NoSuchMethodError: > org.apache.hadoop.conf.Configuration.addDeprecations([Lorg/apache/hadoop/conf/Configuration$DeprecationDelta;)V > at > org.apache.hadoop.yarn.conf.YarnConfiguration.addDeprecatedKeys(YarnConfiguration.java:79) > at > org.apache.hadoop.yarn.conf.YarnConfiguration.(YarnConfiguration.java:73) > at org.apache.tez.dag.app.DAGAppMaster.main(DAGAppMaster.java:2271) > > I am not sure but according to this > thread(http://grokbase.com/t/cloudera/cdh-user/12765svj61/libjars-and-hadoop-jar-command) > this perhaps caused by $HADOOP_CLASSPATH problem. > > But I wander should I copy "tez-dist/target/tez-0.8.2" to all cluster then > export below? > export TEZ_JARS=/home1/apps/tez-0.8.2 > export TEZ_CONF_DIR=$TEZ_JARS/conf > export > HADOOP_CLASSPATH=$TEZ_CONF_DIR:$TEZ_JARS/*:$TEZ_JARS/lib/*:$HADOOP_CLASSPATH > > I did this only on name nodes. > > > 2016-02-12 16:47 GMT+09:00 Hitesh Shah <hit...@apache.org>: > Run the following command: “bin/yarn logs -applicationId > application_1452243782005_0292” . This should give you the logs for > container_1452243782005_0292_02_01 which may shed more light on why the > Tez ApplicationMaster is failing to launch when triggered via Hive. > > thanks > — Hitesh > > > > On Feb 11, 2016, at 10:48 PM, no jihun <jees...@gmail.com> wrote: > > > Hi all. > > > > When I execute a query on hive I got an error below.(so do in hive cli) > > no more detailed log found. > > > > what should I check? > > any advice will be appreciated. > > > > versions > > - tez-0.8.2 > > - hadoop 2.6.0 > > > > --- > > > > hive > set hive.execution.engine=tez; > > hive > select count(*) from contents; > > > > WARNING: org.apache.hadoop.metrics.jvm.EventCounter is deprecated. Please > > use org.apache.hadoop.log.metrics.EventCounter in all the log4j.properties > > files. > > > > Logging initialized using configuration in > > file:/home1/eco/hive/conf/hive-log4j.properties > > hive> set hive.execution.engine=tez; > > hive> select count(*) from agg_band_contents; > > Query ID = irteam_20160212145903_9300f3b2-3942-4423-8586-73d2eaff9e58 > > Total jobs = 1 > > Launching Job 1 out of 1 > > Exception in thread "Thread-10" java.lang.RuntimeException: > > org.apache.tez.dag.api.SessionNotRunning: TezSession has already shutdown. > > Application application_1452243782005_0292 failed 2 times due to AM > > Container for appattempt_1452243782005_0292_02 exited with exitCode: 1 > > For more detailed output, check application tracking > > page:http://xstathn003:8088/proxy/application_1452243782005_0292/Then, > > click on links to logs of each attempt. > > Diagnostics: Exception from container-launch. > > Container id: container_1452243782005_0292_02_01 > > Exit code: 1 > > Stack trace: ExitCodeException exitCode=1: > > at org.apache.hadoop.util.Shell.runCommand(Shell.java:538) > > at org.apache.hadoop.util.Shell.run(Shell.java:455) > > at > > org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:715) > > at > > org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:211) > > at > > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302) > > at > > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82) > > at java.util.concurrent.FutureTask.run(FutureTask.java:262) > > at > > java.util.concurrent.ThreadPoolExecutor.
Re: Failing attemption at org.apache.tez.client.TezClient.waitTillReady
r-- 1 6377448 2014-01-30 07:10 groovy-all-2.1.6.jar > -rw-rw-r-- 1 2189117 2015-04-30 03:08 guava-14.0.1.jar > -rw-rw-r-- 1 76643 2014-01-30 07:07 hamcrest-core-1.1.jar > -rw-rw-r-- 1 121403 2015-06-19 18:05 hive-accumulo-handler-1.2.1.jar > -rw-rw-r-- 1 47713 2015-06-19 18:04 hive-ant-1.2.1.jar > -rw-rw-r-- 1 138361 2015-06-19 18:05 hive-beeline-1.2.1.jar > -rw-rw-r-- 1 39019 2015-06-19 18:05 hive-cli-1.2.1.jar > -rw-rw-r-- 1 292290 2015-06-19 18:03 hive-common-1.2.1.jar > -rw-rw-r-- 1 121668 2015-06-19 18:05 hive-contrib-1.2.1.jar > -rw-rw-r-- 120599030 2015-06-19 18:04 hive-exec-1.2.1.jar > -rw-rw-r-- 1 115935 2015-06-19 18:05 hive-hbase-handler-1.2.1.jar > -rw-rw-r-- 1 28091 2015-06-19 18:06 hive-hwi-1.2.1.jar > -rw-rw-r-- 117360142 2015-06-19 18:05 hive-jdbc-1.2.1-standalone.jar > -rw-rw-r-- 1 100580 2015-06-19 18:05 hive-jdbc-1.2.1.jar > -rw-rw-r-- 1 5505100 2015-06-19 18:04 hive-metastore-1.2.1.jar > -rw-rw-r-- 1 916706 2015-06-19 18:03 hive-serde-1.2.1.jar > -rw-rw-r-- 1 1878543 2015-06-19 18:04 hive-service-1.2.1.jar > -rw-rw-r-- 1 32390 2015-06-19 18:03 hive-shims-0.20S-1.2.1.jar > -rw-rw-r-- 1 60070 2015-06-19 18:03 hive-shims-0.23-1.2.1.jar > -rw-rw-r-- 18949 2015-06-19 18:03 hive-shims-1.2.1.jar > -rw-rw-r-- 1 108914 2015-06-19 18:03 hive-shims-common-1.2.1.jar > -rw-rw-r-- 1 13065 2015-06-19 18:03 hive-shims-scheduler-1.2.1.jar > -rw-rw-r-- 1 14530 2015-06-19 18:06 hive-testutils-1.2.1.jar > -rw-rw-r-- 1 719304 2015-04-30 03:08 httpclient-4.4.jar > -rw-rw-r-- 1 321639 2015-04-30 03:08 httpcore-4.4.jar > -rw-rw-r-- 1 1282424 2015-04-30 03:25 ivy-2.4.0.jar > -rw-rw-r-- 1 611863 2015-04-30 03:25 janino-2.7.6.jar > -rw-rw-r-- 1 60527 2015-04-30 03:26 jcommander-1.32.jar > -rw-rw-r-- 1 201124 2014-01-30 07:09 jdo-api-3.0.1.jar > -rw-rw-r-- 1 1681148 2014-05-13 09:25 jetty-all-7.6.0.v20120127.jar > -rw-rw-r-- 1 1683027 2014-01-30 11:30 > jetty-all-server-7.6.0.v20120127.jar > -rw-rw-r-- 1 213854 2015-04-30 03:08 jline-2.12.jar > -rw-rw-r-- 1 588001 2015-04-30 03:23 joda-time-2.5.jar > -rw-rw-r-- 1 12131 2014-05-13 09:25 jpam-1.1.jar > -rw-rw-r-- 1 45944 2014-01-30 07:10 json-20090211.jar > -rw-rw-r-- 1 33031 2015-04-30 03:23 jsr305-3.0.0.jar > -rw-rw-r-- 1 15071 2014-01-30 07:09 jta-1.1.jar > -rw-rw-r-- 1 245039 2015-04-30 03:08 junit-4.11.jar > -rw-rw-r-- 1 313686 2015-04-30 03:23 libfb303-0.9.2.jar > -rw-rw-r-- 1 227712 2015-04-30 03:08 libthrift-0.9.2.jar > -rw-rw-r-- 1 481535 2014-01-30 07:06 log4j-1.2.16.jar > -rw-rw-r-- 1 447676 2014-01-30 11:30 mail-1.4.1.jar > -rw-rw-r-- 1 94421 2015-04-30 03:26 maven-scm-api-1.4.jar > -rw-rw-r-- 1 40066 2015-04-30 03:26 > maven-scm-provider-svn-commons-1.4.jar > -rw-rw-r-- 1 69858 2015-04-30 03:26 maven-scm-provider-svnexe-1.4.jar > -rw-rw-r-- 1 1208356 2015-04-30 03:08 netty-3.7.0.Final.jar > -rw-rw-r-- 1 19827 2015-04-30 03:23 opencsv-2.3.jar > -rw-rw-r-- 1 65261 2014-01-30 07:07 oro-2.0.8.jar > -rw-rw-r-- 1 29555 2014-01-30 07:08 paranamer-2.3.jar > -rw-rw-r-- 1 2796935 2015-04-30 03:23 parquet-hadoop-bundle-1.6.0.jar > -rw-rw-r-- 1 48557 2015-04-30 03:25 > pentaho-aggdesigner-algorithm-5.1.5-jhyde.jar > drwxrwxr-x 64096 2015-09-10 20:14 php > -rw-rw-r-- 1 250546 2014-01-30 11:29 plexus-utils-1.5.6.jar > drwxrwxr-x 104096 2015-09-10 20:14 py > -rw-rw-r-- 1 25429 2015-04-30 03:26 regexp-1.3.jar > -rw-rw-r-- 1 105112 2014-01-30 07:08 servlet-api-2.5.jar > -rw-rw-r-- 1 1251514 2014-01-30 07:08 snappy-java-1.0.5.jar > -rw-r--r-- 1 162976273 2015-09-10 20:16 > spark-assembly-1.4.1-hadoop2.6.0.jar > -rw-rw-r-- 1 26514 2014-01-30 07:08 stax-api-1.0.1.jar > -rw-rw-r-- 1 148627 2014-01-30 07:09 stringtemplate-3.2.1.jar > -rw-rw-r-- 1 93210 2015-04-30 03:26 super-csv-2.2.0.jar > -rw-rw-r-- 1 55953 2014-01-30 11:30 tempus-fugit-1.1.jar > -rw-rw-r-- 1 392124 2014-01-30 07:07 velocity-1.5.jar > -rw-rw-r-- 1 94672 2014-01-30 07:08 xz-1.0.jar > -rw-rw-r-- 1 792964 2015-04-30 03:08 zookeeper-3.4.6.jar > > * and hadoop-common-2.6.0.jar have addDeprecations(DeprecationDelta[]) method. > > > > > 2016-02-12 17:29 GMT+09:00 Hitesh Shah <hit...@apache.org>: > It seems to me that the Tez AM classpath somehow has a hadoop-common jar that > does not have the Configuration.addDeprecations() api that YARN needs. > > For the Tez AM, the classpath is fully constructed based on the tez tarball ( > from HDFS using distributed cache ) and
Re: Failing attemption at org.apache.tez.client.TezClient.waitTillReady
Run the following command: “bin/yarn logs -applicationId application_1452243782005_0292” . This should give you the logs for container_1452243782005_0292_02_01 which may shed more light on why the Tez ApplicationMaster is failing to launch when triggered via Hive. thanks — Hitesh On Feb 11, 2016, at 10:48 PM, no jihunwrote: > Hi all. > > When I execute a query on hive I got an error below.(so do in hive cli) > no more detailed log found. > > what should I check? > any advice will be appreciated. > > versions > - tez-0.8.2 > - hadoop 2.6.0 > > --- > > hive > set hive.execution.engine=tez; > hive > select count(*) from contents; > > WARNING: org.apache.hadoop.metrics.jvm.EventCounter is deprecated. Please use > org.apache.hadoop.log.metrics.EventCounter in all the log4j.properties files. > > Logging initialized using configuration in > file:/home1/eco/hive/conf/hive-log4j.properties > hive> set hive.execution.engine=tez; > hive> select count(*) from agg_band_contents; > Query ID = irteam_20160212145903_9300f3b2-3942-4423-8586-73d2eaff9e58 > Total jobs = 1 > Launching Job 1 out of 1 > Exception in thread "Thread-10" java.lang.RuntimeException: > org.apache.tez.dag.api.SessionNotRunning: TezSession has already shutdown. > Application application_1452243782005_0292 failed 2 times due to AM Container > for appattempt_1452243782005_0292_02 exited with exitCode: 1 > For more detailed output, check application tracking > page:http://xstathn003:8088/proxy/application_1452243782005_0292/Then, click > on links to logs of each attempt. > Diagnostics: Exception from container-launch. > Container id: container_1452243782005_0292_02_01 > Exit code: 1 > Stack trace: ExitCodeException exitCode=1: > at org.apache.hadoop.util.Shell.runCommand(Shell.java:538) > at org.apache.hadoop.util.Shell.run(Shell.java:455) > at > org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:715) > at > org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:211) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82) > 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) > > > Container exited with a non-zero exit code 1 > Failing this attempt. Failing the application. > at > org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:535) > at org.apache.hadoop.hive.ql.exec.TaskRunner.run(TaskRunner.java:74) > Caused by: org.apache.tez.dag.api.SessionNotRunning: TezSession has already > shutdown. Application application_1452243782005_0292 failed 2 times due to AM > Container for appattempt_1452243782005_0292_02 exited with exitCode: 1 > For more detailed output, check application tracking > page:http://xstathn003:8088/proxy/application_1452243782005_0292/Then, click > on links to logs of each attempt. > Diagnostics: Exception from container-launch. > Container id: container_1452243782005_0292_02_01 > Exit code: 1 > Stack trace: ExitCodeException exitCode=1: > at org.apache.hadoop.util.Shell.runCommand(Shell.java:538) > at org.apache.hadoop.util.Shell.run(Shell.java:455) > at > org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:715) > at > org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:211) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302) > at > org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82) > 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) > > > Container exited with a non-zero exit code 1 > Failing this attempt. Failing the application. > at org.apache.tez.client.TezClient.waitTillReady(TezClient.java:784) > at > org.apache.hadoop.hive.ql.exec.tez.TezSessionState.open(TezSessionState.java:205) > at > org.apache.hadoop.hive.ql.exec.tez.TezSessionState.open(TezSessionState.java:116) > at > org.apache.hadoop.hive.ql.session.SessionState.start(SessionState.java:532) > ... 1 more >
Re: Question regarding instability of EdgeProperty DataSourceType
There are 3 types defined as you have noticed: persisted_reliable: assumes a vertex output is stored in a reliable store like HDFS. This states that if the node on which the task ran disappears, the output is still available. persisted: vertex output stored on local disk where the task ran. ephemeral: vertex output stored in task memory. From a data transmission point of view, all data is always transmitted over network unless there is a case where the downstream task is running on same machine as the task that generated the output. In that case, it can read from local disk if needed. You are right that the in-memory support is not built out so a co-located task potentially reading from another task’s memory is therefore not supported today. The network channel requires a bit more explanation. Generally, all data is persisted to disk. This means that data transferred over the network is first written locally and then eventually pulled/pushed to a downstream task as needed. This does not mean that all the data needs to be generated first before being sent downstream. Data can be still generated in “chunks” and then sent downstream as when as a chunk becomes available. ( this functionality is internally called “pipelined shuffle” if you end up searching through the code/jiras ). However, again to clarify, there is no pure streaming support yet where data is kept in memory and pushed downstream. To add, the in-memory approach requires changing Tez to have a different fault tolerance model to be applied and it also needs more cluster capacity to ensure that both upstream and downstream tasks can run concurrently. Do you see this as a requirement for something that you are planning to use Tez for? thanks — Hitesh On Jan 31, 2016, at 12:30 AM, Gal Vinogradwrote: > Hey, > > I read though tez-api code and noticed that PERSISTED is the only stable data > source type. Does that mean that data isn't transmitted between vertices > though in-memory or network channels? > > Thanks :)
Re: What's the application scenario of Apache TEZ
Couple of other points to add to Bikas’s email: Regarding your question on small data: No - Tez is geared to work in both small data and extremely large data cases. Hive should likely perform better with Tez regardless of data size unless there is a bad query plan created that is non-optimal for Tez. For 3). Hive/Pig/Cascading when used with MR would deconstruct a single hive query/pig script into multiple MR jobs. This would end up reading/writing from/to HDFS multiple times. Furthermore, with MR, you are stuck to fitting all your code into a Mapper and Reducer ( each with only a single input and output ) and using Shuffle for data transfer. This introduces additional inefficiencies. With Tez, a single hive query can be converted into a single DAG. Vertices can run any kind of logic and the edges between vertices are not restricted to “shuffle-like” data transfer which allows more optimizations at the query planning stages. The fact that Tez allows Hive/Pig to use smarter ways of processing queries/scripts is what is usually the biggest win in terms of performance. Spark is similarly better than MR as it provides a richer operator library in some sense. As for comparing Spark vs Tez, to some extent, it is likely comparing apples to oranges as Tez is quite a low-level library. Depending on how an application is written to make use of Tez vs Spark, you will find different cases where one is faster than the other. — Hitesh On Jan 20, 2016, at 8:44 AM, LLBianwrote: > > Hello,Tez experts: > I have known that, tez is used in DAG cases. >Because it can control the intermediate results do not write to disk, > and container reuse, so it is more effective in processing small amount of > data than mr. So, mybe I will think that hive on tez is better than hive on > mr in processing small amount of data, am I right? > Well, now, my questions are: > (1)Even though there are main design themes in https://tez.apache.org/ , I am > still not very clear about its application scenarios,and If there are some > real and main enterprise applications,so much the better. > (2)I am still not very clear what question It is mainly used to solving? > (3) Why it is use for hive and pig? how is it better than spark or mr? > (4)I looked at your official PPT and paper “Apache Tez: A Unifying Framework > for Modeling and Building Data Processing Applications" , but still not very > clearly. > How to understand this :"Don’t solve problems that have already been solved. > Or else you will have to solve them again!"? Is there any real example? > > Apache tez is a great product , I hope to learn more about it. > > Any reply are very appreciated. > > Thankyou & Best Regards. > > ---LLBian > >
Re: Issue: Hive with Tez on CDH-5.4.2
I don’t believe I have seen this error reported before. The error mainly seems to be coming from somewhere in the Hive codebase so the hive mailing list might provide a more relevant answer. If you don’t get one, would you mind setting “tez.am.log.level” to DEBUG in your tez-site.xml, re-run the query and attach the yarn logs ( via bin/yarn logs -applicationId ) to a new jira? thanks — Hitesh On Dec 4, 2015, at 12:08 PM, Sudheer NVwrote: > Hi All, > > I have installed Tez (0.7.0) on Cloudera VM CDH 5.4.2 (hadoop-2.6.0, > hive-1.1.0). Execution of demo orderedWordCount example got executed > successfully. But hive queries on Tez execution engine is throwing below > error. Any help is appreciated. Thanks!! > > ERROR LOGS: > > hive> set hive.execution.engine=tez; > hive> select count(*) from sample_07; > Query ID = cloudera_2015120411_d37e3a42-6924-423b-9523-f43c1cca90e8 > Total jobs = 1 > Launching Job 1 out of 1 > Tez session was closed. Reopening... > Session re-established. > > > Status: Running (Executing on YARN cluster with App id > application_1449255750568_0004) > > > VERTICES STATUS TOTAL COMPLETED RUNNING PENDING FAILED > KILLED > > Map 1 FAILED -1 00 -1 0 > 0 > Reducer 2 KILLED 1 001 0 > 0 > > VERTICES: 00/02 [>>--] 0%ELAPSED TIME: 0.47 s > > > Status: Failed > Vertex failed, vertexName=Map 1, vertexId=vertex_1449255750568_0004_1_00, > diagnostics=[Vertex vertex_1449255750568_0004_1_00 [Map 1] killed/failed due > to:ROOT_INPUT_INIT_FAILURE, Vertex Input: sample_07 initializer failed, > vertex=vertex_1449255750568_0004_1_00 [Map 1], > java.lang.IllegalArgumentException: Illegal Capacity: -12185 > at java.util.ArrayList.(ArrayList.java:142) > at > org.apache.hadoop.mapred.FileInputFormat.getSplits(FileInputFormat.java:330) > at > org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:306) > at > org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:408) > at > org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:129) > at > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:245) > at > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:239) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1671) > at > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:239) > at > org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:226) > 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) > ] > Vertex killed, vertexName=Reducer 2, vertexId=vertex_1449255750568_0004_1_01, > diagnostics=[Vertex received Kill in INITED state., Vertex > vertex_1449255750568_0004_1_01 [Reducer 2] killed/failed due to:null] > DAG did not succeed due to VERTEX_FAILURE. failedVertices:1 killedVertices:1 > FAILED: Execution Error, return code 2 from > org.apache.hadoop.hive.ql.exec.tez.TezTask > > > Regards, > Sudheer
Re: Running Tez with Tachyon
iguration.java:2195) > at > org.apache.hadoop.fs.FileSystem.getFileSystemClass(FileSystem.java:2638) > at > org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:2651) > at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:92) > at > org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:2687) > at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:2669) > at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:371) > at org.apache.hadoop.fs.Path.getFileSystem(Path.java:295) > at > org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.(FileOutputCommitter.java:105) > at > org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.(FileOutputCommitter.java:80) > at > org.apache.hadoop.mapreduce.lib.output.FileOutputFormat.getOutputCommitter(FileOutputFormat.java:309) > at > org.apache.tez.mapreduce.committer.MROutputCommitter.getOutputCommitter(MROutputCommitter.java:137) > ... 24 more > Caused by: java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not > found > at > org.apache.hadoop.conf.Configuration.getClassByName(Configuration.java:2101) > at > org.apache.hadoop.conf.Configuration.getClass(Configuration.java:2193) > ... 35 more > , Vertex vertex_1447296197811_0003_1_02 [Sorter] killed/failed due > to:INIT_FAILURE], Vertex killed, vertexName=Summation, > vertexId=vertex_1447296197811_0003_1_01, diagnostics=[Vertex received Kill in > INITED state., Vertex vertex_1447296197811_0003_1_01 [Summation] > killed/failed due to:OTHER_VERTEX_FAILURE], DAG did not succeed due to > VERTEX_FAILURE. failedVertices:2 killedVertices:1] > > Best, > > -- > Jiří Šimša > > On Thu, Nov 12, 2015 at 8:52 AM, Hitesh Shah <hit...@apache.org> wrote: > The general approach for add-on jars requires 2 steps: > > 1) On the client host, where the job is submitted, you need to ensure that > the add-on jars are in the local classpath. This is usually done by adding > them to HADOOP_CLASSPATH. Please do pay attention to adding the jars via > "/*” instead of just "” > 2) Next, "tez.aux.uris”. This controls additional files/jars needed in the > runtime on the cluster. Upload the tachyon jar to HDFS and ensure that you > provide the path to either the dir on HDFS or the full path to the file and > specify that in tez.aux.uris. > > The last thing to note is that you may need to pull additional transitive > dependencies of tachyon if it is not self-contained jar. > > thanks > — HItesh > > On Nov 12, 2015, at 1:06 AM, Bikas Saha <bi...@apache.org> wrote: > > > Can you provide the full stack trace? > > > > Are you getting the exception on the client (while submitting the job) or > > in the cluster (after the job started to run)? > > > > For the client side, the fix would be to add tachyon jars to the client > > classpath. Looks like you tried some client side classpath fixes. You could > > run ‘hadoop classpath’ to print the classpath being picked up by the > > ‘hadoop jar’ command. And scan its output to check if your tachyon jars are > > being picked up correctly or not. > > > > Bikas > > > > From: Jiří Šimša [mailto:jiri.si...@gmail.com] > > Sent: Wednesday, November 11, 2015 6:54 PM > > To: user@tez.apache.org > > Subject: Running Tez with Tachyon > > > > Hello, > > > > I have followed the Tez installation instructions > > (https://tez.apache.org/install.html) and was able to successfully run the > > ordered word count example: > > > > $ hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar > > orderedwordcount /input.txt /output.txt > > > > Next, I wanted to see if I can do the same, this time reading from and > > writing to Tachyon (http://tachyon-project.org/) using: > > > > $ hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar > > orderedwordcount tachyon://localhost:19998/input.txt > > tachyon://localhost:19998/output.txt > > > > Unsurprisingly, this resulted in the "Class tachyon.hadoop.TFS not found" > > error because Tez needs the Tachyon client jar that defines the > > tachyon.hadoop.TFS class. To that end, I have tried several options (listed > > below) to provide this jar to Tez, none of which seems to have worked: > > > > 1) Adding the Tachyon client jar to HADOOP_CLASSPATH > > 2) Specifying the Tachyon client jar with the -libjars flag for the above > > command. > > 3) Copying the Tachyon client jar into the > > $HADOOP_HOME/share/hadoop/common/lib directory of my HADOOP installation. > > 4) Copying the Tachyon client jar into HDFS and specifying a path to it > > through the tez.aux.uris property in the tez-site.xml file (in a similar > > fashion the tez.lib.uris property specifies the path to the Tez tarball). > > 5) I modified the source code of the ordered word count example, adding a > > call to TezClient#addAppMasterLocalFiles(...), providing a URI for the > > Tachyon client jar uploaded to HDFS. > > > > Any advice on how to pass the Tachyon client jar to Tez to resolve this > > issue would be greatly appreciated. Thank you. > > > > Best, > > > > -- > > Jiří Šimša > > >
Re: Running Tez with Tachyon
The general approach for add-on jars requires 2 steps: 1) On the client host, where the job is submitted, you need to ensure that the add-on jars are in the local classpath. This is usually done by adding them to HADOOP_CLASSPATH. Please do pay attention to adding the jars via "/*” instead of just "” 2) Next, "tez.aux.uris”. This controls additional files/jars needed in the runtime on the cluster. Upload the tachyon jar to HDFS and ensure that you provide the path to either the dir on HDFS or the full path to the file and specify that in tez.aux.uris. The last thing to note is that you may need to pull additional transitive dependencies of tachyon if it is not self-contained jar. thanks — HItesh On Nov 12, 2015, at 1:06 AM, Bikas Sahawrote: > Can you provide the full stack trace? > > Are you getting the exception on the client (while submitting the job) or in > the cluster (after the job started to run)? > > For the client side, the fix would be to add tachyon jars to the client > classpath. Looks like you tried some client side classpath fixes. You could > run ‘hadoop classpath’ to print the classpath being picked up by the ‘hadoop > jar’ command. And scan its output to check if your tachyon jars are being > picked up correctly or not. > > Bikas > > From: Jiří Šimša [mailto:jiri.si...@gmail.com] > Sent: Wednesday, November 11, 2015 6:54 PM > To: user@tez.apache.org > Subject: Running Tez with Tachyon > > Hello, > > I have followed the Tez installation instructions > (https://tez.apache.org/install.html) and was able to successfully run the > ordered word count example: > > $ hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar > orderedwordcount /input.txt /output.txt > > Next, I wanted to see if I can do the same, this time reading from and > writing to Tachyon (http://tachyon-project.org/) using: > > $ hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar > orderedwordcount tachyon://localhost:19998/input.txt > tachyon://localhost:19998/output.txt > > Unsurprisingly, this resulted in the "Class tachyon.hadoop.TFS not found" > error because Tez needs the Tachyon client jar that defines the > tachyon.hadoop.TFS class. To that end, I have tried several options (listed > below) to provide this jar to Tez, none of which seems to have worked: > > 1) Adding the Tachyon client jar to HADOOP_CLASSPATH > 2) Specifying the Tachyon client jar with the -libjars flag for the above > command. > 3) Copying the Tachyon client jar into the > $HADOOP_HOME/share/hadoop/common/lib directory of my HADOOP installation. > 4) Copying the Tachyon client jar into HDFS and specifying a path to it > through the tez.aux.uris property in the tez-site.xml file (in a similar > fashion the tez.lib.uris property specifies the path to the Tez tarball). > 5) I modified the source code of the ordered word count example, adding a > call to TezClient#addAppMasterLocalFiles(...), providing a URI for the > Tachyon client jar uploaded to HDFS. > > Any advice on how to pass the Tachyon client jar to Tez to resolve this issue > would be greatly appreciated. Thank you. > > Best, > > -- > Jiří Šimša
Re: Tez Code 1 & Tez Code 2
Hello Dale, I think I can guess what is happening. Hue keeps connections open between itself and the HiveServer2. Now what happens is that the Tez session times itself out if queries are not submitted to it within a certain time window ( to stop wasting resources on a YARN cluster ). This can be inferred from the following log: 2015-10-13 15:39:43,938 INFO [Timer-1] app.DAGAppMaster: Session timed out, lastDAGCompletionTime=1444746866023 ms, sessionTimeoutInterval=30 ms There are a couple of fixes that were done in Tez and Hive so that such timeouts cause the Tez sessions to be re-cycled correctly. Due to a combination of Tez not throwing a proper error in earlier versions to Hive not handling such submission failures correctly, Hive continues to try to use the “dead” sessions causing submitted queries to fail. In your case, for the short term, a restart of the HiveServer should address the issue but until the HiveServer ( and/or Tez ) is patched, you will see such issues crop up from time to time. thanks — Hitesh On Oct 19, 2015, at 5:52 AM, Bradman, Dalewrote: > Hi Jeff, > > The RM is fairly stable and reliable. As I said, the command works when > passing it through Beeline. Just not in Hue. > > Resource manager log snippet: > > 2015-10-19 09:51:51,811 INFO ipc.Server (Server.java:run(2060)) - IPC Server > handler 32 on 8032, call > org.apache.hadoop.yarn.api.ApplicationClientProtocolPB.getApplicationReport > from 10.10.7.223:33554 Call#625674 Retry#0 > org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException: Application > with id 'application_1444742140034_0009' doesn't exist in RM. > at > org.apache.hadoop.yarn.server.resourcemanager.ClientRMService.getApplicationReport(ClientRMService.java:324) > at > org.apache.hadoop.yarn.api.impl.pb.service.ApplicationClientProtocolPBServiceImpl.getApplicationReport(ApplicationClientProtocolPBServiceImpl.java:170) > at > org.apache.hadoop.yarn.proto.ApplicationClientProtocol$ApplicationClientProtocolService$2.callBlockingMethod(ApplicationClientProtocol.java:401) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619) > at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2033) > 2015-10-19 09:52:46,900 INFO ipc.Server (Server.java:run(2060)) - IPC Server > handler 25 on 8032, call > org.apache.hadoop.yarn.api.ApplicationClientProtocolPB.getApplicationReport > from 10.10.7.223:33599 Call#625743 Retry#0 > org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException: Application > with id 'application_1444742140034_0009' doesn't exist in RM. > at > org.apache.hadoop.yarn.server.resourcemanager.ClientRMService.getApplicationReport(ClientRMService.java:324) > at > org.apache.hadoop.yarn.api.impl.pb.service.ApplicationClientProtocolPBServiceImpl.getApplicationReport(ApplicationClientProtocolPBServiceImpl.java:170) > at > org.apache.hadoop.yarn.proto.ApplicationClientProtocol$ApplicationClientProtocolService$2.callBlockingMethod(ApplicationClientProtocol.java:401) > at > org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:619) > at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:962) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2039) > at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2035) > at java.security.AccessController.doPrivileged(Native Method) > at javax.security.auth.Subject.doAs(Subject.java:415) > at > org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) > at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2033) > 2015-10-19 09:53:45,479 INFO ipc.Server (Server.java:run(2060)) - IPC Server > handler 10 on 8032, call > org.apache.hadoop.yarn.api.ApplicationClientProtocolPB.getApplicationReport > from 10.10.7.223:33651 Call#625830 Retry#0 > > > Thanks, > Dale > >> On 19 Oct 2015, at 13:46, Jianfeng (Jeff) Zhang >> wrote: >> >> >> Hi Dale, >> >> Does it happen frequently ? Does the RM work normally (can still accept new >> jobs) when this happens ? >> From the logs, it seems AM meet errors when heartbeat with RM. And it switch >> between 2 RM for a long time. It might be the RM issue, could you check the >> RM logs ? >> >> >> Best Regard, >> Jeff Zhang >> >> >> From: , Dale >>
Re: Tez View in Ambari
In previous releases of the Tez view, it did not have the integration in place for the Hive query info. There is one approach that you can try: Ambari has something called standalone mode. You can deploy a new ambari-server version 2.1 ( on a different host ) and just instantiate a Tez view on it. This ambari instance does not need to be used to manage/install a cluster but the Tez view can be configured to point to the YARN ResourceManager and YARN timeline server from your original cluster. This will likely get you the new set of features ( some of the hive-tez integrations came in newer versions of Hive/Tez so some features might not be available depending on your deployed cluster versions). There are is another approach which would involve compiling a new version of the Tez view against the old ambari-server code and loading into your existing ambari-server instance too. I am not too sure on whether the view from ambari-server 2.1 will work on ambari-server 2.0 ( might be worth a try but not sure if the ambari view apis are fully compatible across both releases ) thanks — Hitesh On Oct 15, 2015, at 3:04 AM, Bradman, Dalewrote: > Thanks. I am using Ambari 2.0. I am guessing that you are referring to Ambari > 2.1? > > > >> On 15 Oct 2015, at 10:59, Rajesh Balamohan wrote: >> >> In recent versions, hive queries are directly available in DAG details >> (which has the app and DAG id). Example screenshot is given below. >> >> >> >> >> Which versions of Hive/Tez are you using? >> >> ~Rajesh.B >> >> >> On Thu, Oct 15, 2015 at 3:08 PM, Bradman, Dale >> wrote: >> Hi, >> >> I am heavily using the Tez UI within Ambari and it's superb - really helps >> tune your queries. >> >> Anyway, I was wondering if there is an easier way to determine which >> applicationID belongs to which query. I’m running 100’s of queries in >> parallel and need a better way to determine what query has been run rather >> than trying to second guess the UI. >> >> Is there a way (either via the UI or otherwise) to do this? >> >> Thanks, >> Dale >> >> >> >> Capgemini is a trading name used by the Capgemini Group of companies which >> includes Capgemini UK plc, a company registered in England and Wales (number >> 943935) whose registered office is at No. 1, Forge End, Woking, Surrey, GU21 >> 6DB. >> This message contains information that may be privileged or confidential and >> is the property of the Capgemini Group. It is intended only for the person >> to whom it is addressed. If you are not the intended recipient, you are not >> authorized to read, print, retain, copy, disseminate, distribute, or use >> this message or any part thereof. If you receive this message in error, >> please notify the sender immediately and delete all copies of this message. >> > > > > Capgemini is a trading name used by the Capgemini Group of companies which > includes Capgemini UK plc, a company registered in England and Wales (number > 943935) whose registered office is at No. 1, Forge End, Woking, Surrey, GU21 > 6DB.
Re: Getting dot files for DAGs
Hello Andre, For ATS, in the TEZ_DAG_ID entity, the dagPlan is already serialized and available in the otherInfo section. It is not in the .dot format but it is used by the Tez UI to come up with the graphical view of the dag plan. thanks — Hitesh On Oct 1, 2015, at 1:28 AM, Andre Kelpe <ake...@concurrentinc.com> wrote: > Maybe it would be a good idea to send the dot file to the ATS along with the > other information you are sending. I too wanted to look at a dot file the > other day and had problem finding it back. > > - André > > On Thu, Oct 1, 2015 at 4:00 AM, Hitesh Shah <hit...@apache.org> wrote: > The .dot file is generated into the Tez Application Master’s container log > dir. Firstly, you need to figure out the yarn application in which the > query/Tez DAG ran. Once you have the applicationId, you can use one of these > 2 approaches: > > 1) Go to the YARN ResourceManager UI, find the application and click through > to the Application Master logs. The .dot file for the dag should be visible > there. > 2) Using the application Id ( if the application has completed), get the yarn > logs using “bin/yarn logs -applicationId ” - once you have the logs, > you will be able to find the contents of the .dot file within them. This > approach only works if you have YARN log aggregation enabled. > > thanks > — Hitesh > > > On Sep 30, 2015, at 5:29 PM, James Pirz <james.p...@gmail.com> wrote: > > > I am using Tez 0.7.0 on Hadopp 2.6 to run Hive queries. > > I am interested in checking DAGs for my queries visually, and I realized > > that I can do that by graphviz once I can get "dot" files of my DAGs. My > > issue is I can not find those files, they are not in the log directory of > > Yarn or Hadoop or under /tmp . > > > > Any hint as where I can find those files would be great. Do I need to add > > any settings to my tez-site.xml in-order to enable generating them ? > > > > Thanks. > > > > > -- > André Kelpe > an...@concurrentinc.com > http://concurrentinc.com
Re: Getting dot files for DAGs
I don’t believe the binary should need changing at all unless you need enhancements from recent commits. It should just be setting up the UI and configuring Tez for using YARN Timeline. The instructions that you can follow: http://tez.apache.org/tez-ui.html http://tez.apache.org/tez_yarn_timeline.html thanks — Hitesh On Oct 1, 2015, at 11:07 AM, James Pirzwrote: > Thanks for suggesting, I never used Tez UI before, and learned about it > yesterday. > I am trying to find out how I can enable/use it. Apparently it needs some > changes in the binary that I am using (I had built the binary for tez 0.7 > almost 2 months ago). > > > > > On Wed, Sep 30, 2015 at 10:27 PM, Jörn Franke wrote: > Why not use tez ui? > > Le jeu. 1 oct. 2015 à 2:29, James Pirz a écrit : > I am using Tez 0.7.0 on Hadopp 2.6 to run Hive queries. > I am interested in checking DAGs for my queries visually, and I realized that > I can do that by graphviz once I can get "dot" files of my DAGs. My issue is > I can not find those files, they are not in the log directory of Yarn or > Hadoop or under /tmp . > > Any hint as where I can find those files would be great. Do I need to add any > settings to my tez-site.xml in-order to enable generating them ? > > Thanks. >
Re: Getting dot files for DAGs
The .dot file is generated into the Tez Application Master’s container log dir. Firstly, you need to figure out the yarn application in which the query/Tez DAG ran. Once you have the applicationId, you can use one of these 2 approaches: 1) Go to the YARN ResourceManager UI, find the application and click through to the Application Master logs. The .dot file for the dag should be visible there. 2) Using the application Id ( if the application has completed), get the yarn logs using “bin/yarn logs -applicationId ” - once you have the logs, you will be able to find the contents of the .dot file within them. This approach only works if you have YARN log aggregation enabled. thanks — Hitesh On Sep 30, 2015, at 5:29 PM, James Pirzwrote: > I am using Tez 0.7.0 on Hadopp 2.6 to run Hive queries. > I am interested in checking DAGs for my queries visually, and I realized that > I can do that by graphviz once I can get "dot" files of my DAGs. My issue is > I can not find those files, they are not in the log directory of Yarn or > Hadoop or under /tmp . > > Any hint as where I can find those files would be great. Do I need to add any > settings to my tez-site.xml in-order to enable generating them ? > > Thanks.
Re: Getting DAG Id from Hive on tez
This is a question that is probably meant for the Hive mailing list. I believe either the Hive query Id or the information from the query plan ( as used in https://github.com/apache/hive/blob/master/ql/src/java/org/apache/hadoop/hive/ql/hooks/ATSHook.java ) should probably be able to give you that info but the Hive devs (d...@hive.apache.org) will likely have a better answer for this. — HItesh On Sep 15, 2015, at 1:56 AM, Dharmesh Kakadiawrote: > Thanks Hitesh. > I am able to filter out the particular dag, now the only problem is how to > get the DAG name. I see from a previous mail on the list[1] that Tez uses > Hive queryId + counter as the dag name. I have access to the hive query Id, > but how do I know the name from it? Is there a way to get the name/Id of the > DAG directly ? Just trying out counter=1..N will be pretty bad for me in > pre-hook. > [1] > http://mail-archives.apache.org/mod_mbox/hive-user/201408.mbox/%3c1409015953.23241.yahoomail...@web161603.mail.bf1.yahoo.com%3E > > Thanks, > Dharmesh > > The Hive query id maps to the Tez dag name. You can try the following call > against timeline: > > > /ws/v1/timeline/TEZ_DAG_ID?primaryFilter=dagName:{tezDagName} > > thanks > — Hitesh > > On Sep 14, 2015, at 10:45 PM, Dharmesh Kakadia < > dharmesh.kaka...@research.iiit.ac.in > > > wrote: > > > Hi, > > > > I am running Hive on Tez, with timeline server. We have a pre-hook in hive > > to maintain > the statistics of what jobs ran by whom and how much resource it tool etc. > that we had been > using it with Hive-MR. Now I am trying to port this pre-hook to work with > Hive on Tez. > > > > I plan to using timeline server for querying the stat, but I am not able to > > get the DAG > ID in the hook. How to get the DAG Id from Tez? Any help will be great. > > > > Thanks, > > Dharmesh > >
Re: Getting DAG Id from Hive on tez
The Hive query id maps to the Tez dag name. You can try the following call against timeline: /ws/v1/timeline/TEZ_DAG_ID?primaryFilter=dagName:{tezDagName} thanks — Hitesh On Sep 14, 2015, at 10:45 PM, Dharmesh Kakadiawrote: > Hi, > > I am running Hive on Tez, with timeline server. We have a pre-hook in hive to > maintain the statistics of what jobs ran by whom and how much resource it > tool etc. that we had been using it with Hive-MR. Now I am trying to port > this pre-hook to work with Hive on Tez. > > I plan to using timeline server for querying the stat, but I am not able to > get the DAG ID in the hook. How to get the DAG Id from Tez? Any help will be > great. > > Thanks, > Dharmesh
Re: Missing libraries.
“tez.aux.uris” supports a comma separated list of files and directories on HDFS or any distributed filesystem ( no tar balls, archives, etc and no support for file:// ). When Hive submits a query to Tez, it adds its hive-exec.jar to the tez runtime ( similar to MR distributed cache ). If you are hitting a class not found again, you are missing some jars from the classpath. If you are using your own InputFormat, try doing an “add jar “ in your hive script or add that custom jar to tez.aux.uris after uploading it to HDFS. If you are modifying the hive-exec.jar, you may need to ensure that Hive is using that jar and not overriding your custom hive-exec.jar. — Hitesh On Sep 11, 2015, at 12:56 AM, Raajaywrote: > Yeah. I added the hive-exec.jar that contains HiveSpltGenerator to HDFS. I > still hit the exception > > On Fri, Sep 11, 2015 at 2:43 AM, Jianfeng (Jeff) Zhang > wrote: > > Have you try using jar rather than tar.gz ? > > > Best Regard, > Jeff Zhang > > > From: Raajay > Reply-To: "user@tez.apache.org" > Date: Friday, September 11, 2015 at 3:15 PM > To: "user@tez.apache.org" > Subject: Missing libraries. > > I am running DAGs generated by Hive for Tez in offline mode; as in I store > the DAGs to disk and then run them later using my own Tez Client. > > I have been able to get this setup going in local mode. However, while > running on the cluster, I hit Processor class not found exception (snippet > below). I figure this is because, custom processor classes defined in Hive > (eg: HiveSplitGenerator) is not visible while executing a mapper. > > I have uploaded, hive exec jar (apache-hive-2.0.0-SNAPSHOT-bin.tar.gz) to > HDFS and pointed ${tez.aux.uris} to that location. Not sure what more is > needed to make hive Classes visible to tez tasks ? "tar.gz" does not work ? > > > 2015-09-11 00:59:02,973 INFO [Dispatcher thread: Central] impl.VertexImpl: > Recovered Vertex State, vertexId=vertex_1441949856963_0006_1_02 [Map 1], > state=NEW, numInitedSourceVertices=0, numStartedSourceVertices=0, > numRecoveredSourceVertices=0, recoveredEvents=0, tasksIsNull=false, numTasks=0 > 2015-09-11 00:59:02,974 INFO [Dispatcher thread: Central] impl.VertexImpl: > Root Inputs exist for Vertex: Map 4 : {a={InputName=a}, > {Descriptor=ClassName=org.apache.tez.mapreduce.input.MRInputLegacy, > hasPayload=true}, > {ControllerDescriptor=ClassName=org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator, > hasPayload=false}} > 2015-09-11 00:59:02,974 INFO [Dispatcher thread: Central] impl.VertexImpl: > Starting root input initializer for input: a, with class: > [org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator] > 2015-09-11 00:59:02,974 INFO [Dispatcher thread: Central] impl.VertexImpl: > Setting vertexManager to RootInputVertexManager for > vertex_1441949856963_0006_1_00 [Map 4] > 2015-09-11 00:59:02,979 INFO [Dispatcher thread: Central] impl.VertexImpl: > Num tasks is -1. Expecting VertexManager/InputInitializers/1-1 split to set > #tasks for the vertex vertex_1441949856963_0006_1_00 [Map 4] > 2015-09-11 00:59:02,979 INFO [Dispatcher thread: Central] impl.VertexImpl: > Vertex will initialize from input initializer. vertex_1441949856963_0006_1_00 > [Map 4] > 2015-09-11 00:59:02,980 INFO [Dispatcher thread: Central] impl.VertexImpl: > Vertex will initialize via inputInitializers vertex_1441949856963_0006_1_00 > [Map 4]. Starting root input initializers: 1 > 2015-09-11 00:59:02,981 ERROR [Dispatcher thread: Central] > common.AsyncDispatcher: Error in dispatcher thread > org.apache.tez.dag.api.TezUncheckedException: Unable to load class: > org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator > at > org.apache.tez.common.ReflectionUtils.getClazz(ReflectionUtils.java:45) > at > org.apache.tez.common.ReflectionUtils.createClazzInstance(ReflectionUtils.java:96) > at > org.apache.tez.dag.app.dag.RootInputInitializerManager.createInitializer(RootInputInitializerManager.java:137) > at > org.apache.tez.dag.app.dag.RootInputInitializerManager.runInputInitializers(RootInputInitializerManager.java:114) > > >
Re: Over writing files
This is probably a question for the Hive dev mailing list on how the staging/output directory name is determined. i.e. ".hive-staging_hive_2015-09-11_00-07-40_043_6365145769624003668-1”. You may need to change this value in the config being used to configure the output of the vertex that is doing the write to HDFS. — Hitesh On Sep 11, 2015, at 1:09 PM, Raajaywrote: > I am running DAGs generated by Hive using my custom Tez Client. So I > serialize a DAG, load it back and submit it later. Everything works great the > first time; however, on second runs the I get a RunTime exception (snippet > below) > > My guess, it since the same DAG is run again, the output tables (have same > id) and that prevents overwrite. > > Where should i introduce randomness in the file name ? Should I change some > name field in FileSinkDescriptor every time I re-run the dag ? > > Thanks > Raajay > > > Vertex failed, vertexName=Reducer 3, > vertexId=vertex_1441949856963_0011_1_04, diagnostics=[Task failed, > taskId=task_1441949856963_0011_1_04_00, diagnostics=[TaskAttempt 0 > failed, info=[Error: Failure while running task:java.lang.RuntimeException: > java.lang.RuntimeException: Hive Runtime Error while closing operators: > Unable to rename output from: > hdfs://10.10.1.2:8020/apps/hive/output_tab/.hive-staging_hive_2015-09-11_00-07-40_043_6365145769624003668-1/_task_tmp.-ext-1/_tmp.00_0 > to: > hdfs://10.10.1.2:8020/apps/hive/output_tab/.hive-staging_hive_2015-09-11_00-07-40_043_6365145769624003668-1/_tmp.-ext-1/00_0 > at > org.apache.hadoop.hive.ql.exec.tez.TezProcessor.initializeAndRunProcessor(TezProcessor.java:171) > at org.apache.hadoop.hive.ql.exec.tez.TezProcessor.run(TezProcessor.java:137) > at > org.apache.tez.runtime.LogicalIOProcessorRuntimeTask.run(LogicalIOProcessorRuntimeTask.java:345) > at > org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:179) > at > org.apache.tez.runtime.task.TezTaskRunner$TaskRunnerCallable$1.run(TezTaskRunner.java:171
Re: Creating TaskLocationHints
Tez tries to obey them but as you call out, it also depends on YARN. Tez follows a simple heuristic. It tries a best effort to do data local allocation. After a certain delay expires, it tries to then allow a task to be assigned to either a data local or rack local container and then after another timeout picks any available container. These fallbacks are configurable ( i.e. whether to allow fall backs ) as well as the time delay. There is also some additional priority given to already launched containers as compared to new allocations from YARN. Search for FALLBACK in TezConfiguration.java or check the attachments in https://issues.apache.org/jira/browse/TEZ-2294 for documentation. — Hitesh On Sep 11, 2015, at 12:05 AM, Raajay <raaja...@gmail.com> wrote: > I was able to get it working with "hostnames". thanks! > > To dig deeper, how much does Tez obey the hints provided? How are Vertex > Location Hints handled ? What if YARN is not able to provide containers in > requested locations ? > > Raajay > > On Thu, Sep 10, 2015 at 10:19 AM, Hitesh Shah <hit...@apache.org> wrote: > In almost all cases, this is usually hostnames. The general flow is find the > block locations for the data source, extract the hostname from there and > provide it to YARN so that it can provide a container on the same host as the > datanode having the data. As long as YARN is using hostnames, the container > locality matching should work correctly. I will need to go and check the YARN > codebase to see if it does some additional reverse dns lookups for IPs to > also function correctly but to be safe, hostnames should work. > > I don’t believe Tez has yet introduced support for working with > application-level YARN node labels. > > thanks > — Hitesh > > On Sep 10, 2015, at 12:43 AM, Raajay <raaja...@gmail.com> wrote: > > > While creating TaskLocationHints, using the static function > > > > TaskLocationHint.createTaskLocationHint(Set nodes, Set > > racks) > > > > what should the Strings be ? IP address of the nodes ? Node labels ? Or > > hostnames ? > > > > Thanks > > Raajay > >
Re: Error of setting vertex location hints
There are 2 aspects to using Vertex Location Hints and parallelism. All of this depends on how you define the work that needs to be done by a particular task. I will take the MR approach and compare it to the more dynamic approach that Jeff has been explaining. For MR, all the work was decided upfront on the client-side. i.e. how many tasks are needed and which task will process what split. From a Tez point of view, what this means is that you can configure the vertex with a fixed parallelism ( i.e. not -1 ) and set up the Vertex location hints as needed. This also implies that you need to configure the Input for that vertex with all the necessary information on what work it needs to do via its user payload. tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java has an option to generate the splits on the client. You can follow this code path to see how the DAG is setup. The same approach is also used for running any MapReduce job via Tez using the yarn-tez config knob ( MR always generates splits on the client ). The dynamic approach that Tez follows is that for vertices which are taking input from HDFS ( or any other source for that matter ) will have parallelism set to -1 ( and no location hints defined at dag plan creation time ). The Input has an Initializer attached to it which runs in the ApplicationMaster, looks at the data to be processed, figures out how many tasks to run, where to run the tasks and also what shard/partition of work to assign to each task. There are multiple facets to this which have been mostly covered by Jeff in his earlier replies. thanks — Hitesh On Sep 10, 2015, at 1:15 AM, Jianfeng (Jeff) Zhangwrote: > >>> I am trying to create a scenario where the mappers (root tasks) are > >>> necessarily not executed at the data location > Not sure your purpose. Usually data locality can improve performance. > > > >>> Can the number of tasks for the tokenizer be a value *NOT* equal to the > >>> number of HDFS blocks of the file ? > Yes, it can. Two ways > * MRInput internally use InputFormat to determine how to split. So all the > methods in InputFormat are applied to MRInput too. >Like mapreduce.input.fileinputformat.split.minsize & > mapreduce.input.fileinputformat.split.maxsize > > * Another way is to use TezGroupedSplitsInputFormat which is provided by tez. > This InputFormat will group several splits together as a new split to be > consumed by one mapper. > You can use the following parameters to tune that, and please refer > MRInputConfigBuilder.groupSplits > • tez.grouping.split-waves > • tez.grouping.max-size > • tez.grouping.min-size > > >>> Can a mapper be scheduled at a location different than the location of > >>> its input block ? If yes, how ? > Yes, it is possible. Tez will always use the split info, there’s no > option to disable it. If you really want to, you need to create new > InputInitializer. I think you just need to make a little changes on > MRInputAMSplitGenerator > > https://github.com/zjffdu/tez/blob/a3a7700dea0a315ad613aa2d8a7223eb73878cb5/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/common/MRInputAMSplitGenerator.java > > > You just need to make a little changes on the following code snippet > > InputConfigureVertexTasksEvent configureVertexEvent = > InputConfigureVertexTasksEvent.create( > inputSplitInfo.getNumTasks(), > VertexLocationHint.create(inputSplitInfo.getTaskLocationHints()), > // make code changes here > InputSpecUpdate.getDefaultSinglePhysicalInputSpecUpdate()); > events.add(configureVertexEvent); > > > > Best Regard, > Jeff Zhang > > > From: Raajay > Reply-To: "user@tez.apache.org" > Date: Thursday, September 10, 2015 at 2:07 PM > To: "user@tez.apache.org" > Subject: Re: Error of setting vertex location hints > > The input is a hdfs file. I am trying to create a scenario where the mappers > (root tasks) are necessarily not executed at the data location. So for now, I > chose the Location Hint for the tasks in a random fashion. I figured by > populating VertexLocation hint, with address of random nodes, I could achieve > it. > > This requires setting parallelism to be the number of elements in > VertexLocation hint; which led to the errors. > > Summarizing, for the work count example, > > 1. Can the number of tasks for the tokenizer be a value *NOT* equal to the > number of HDFS blocks of the file ? > > 2. Can a mapper be scheduled at a location different than the location of its > input block ? If yes, how ? > > Raajay > > > > > On Thu, Sep 10, 2015 at 12:30 AM, Jianfeng (Jeff) Zhang > wrote: > >>> In the WordCount example, while creating the Tokenizer Vertex, neither > >>> the parallelism or VertexLocation hints is specified. My guess is that
Re: Pig(0.14.0) on Tez(0.7.0)
Based on the stack trace, the following issue seems to be the cause: Caused by: java.lang.NoSuchMethodError: com.google.common.base.Stopwatch.elapsedTime(Ljava/util/concurrent/TimeUnit;)J at org.apache.tez.runtime.library.common.shuffle.HttpConnection.validate(HttpConnection.java:221) at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.setupConnection(FetcherOrderedGrouped.java:328) at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.copyFromHost(FetcherOrderedGrouped.java:245) at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.run(FetcherOrderedGrouped.java:167) This means that the guava library version is picking up a new version at runtime. A quick test is to run say orderedwordcount from tez-examples to verify standalone Tez has no issues. Also, you may wish to check the contents of “tez.lib.uris” to verify that it has guava-11.0.2. If you are familiar with using/debugging YARN, set "yarn.nodemanager.delete.debug-delay-sec” to a value such as 1200. Now, pick a host on which the Tez container ran in which a task failed. Using the directories specified in "yarn.nodemanager.local-dirs” in yarn-site.xml, search for a “launch_container.sh” under the container-specific directory mapping to the tez container above. The contents of this will tell you which guava library is being symlinked into the container space and used in the classpath. Having 2 guava jars is also a problem as either could be picked. thanks — Hitesh On Sep 2, 2015, at 5:18 AM, Sandeep Kumarwrote: > Thanks for your responses. I was mistaken that there is any compatibility > issue. Its the same error when i run PIG-0.14.0 over Tez-0.5.2. > > HadoopVersion: 2.6.0-cdh5.4.4 > > PigVersion: 0.14.0 > > TezVersion: 0.5.2 > > > PFA the exception stack trace. > > > On Wed, Sep 2, 2015 at 3:11 PM, Jianfeng (Jeff) Zhang > wrote: > >>> I could not use tez-0.5.2 because it was not compatible with > >>> Hadoop-2.6.0. > > What incompatible do you see ? > > > Best Regard, > Jeff Zhang > > > From: Sandeep Kumar > Reply-To: "user@tez.apache.org" > Date: Wednesday, September 2, 2015 at 5:18 PM > > To: "user@tez.apache.org" > Subject: Re: Pig(0.14.0) on Tez(0.7.0) > > Yes i did change PIG/ivy/libraries.propeties to compile it with tez-0.7.0 and > also changed pig to compile with Hadoop-core-2.6.0. > > I could not use tez-0.5.2 because it was not compatible with Hadoop-2.6.0. > > I'm compiling my code of PIG using same command: ant clean jar > -Dhadoopversion=23 > > > > On Wed, Sep 2, 2015 at 2:36 PM, Jianfeng (Jeff) Zhang > wrote: > > Not sure how did you compile pig with tez 0.7.0, did you change the tez > version in PIG/ivy/libraries.propeties ? > > And make sure you build pig with hadoop version, by default, pig build with > hadoop-1.x. Use the following command to build pig with hadoop-2.x > > >> ant clean jar -Dhadoopversion=23 > > > > Best Regard, > Jeff Zhang > > > From: Sandeep Kumar > Reply-To: "user@tez.apache.org" > Date: Wednesday, September 2, 2015 at 4:27 PM > To: "user@tez.apache.org" > Subject: Re: Pig(0.14.0) on Tez(0.7.0) > > Hi Jeff, > > The cloudera Hadoop is using guava-11.0.2.jar. > I've also exported one environment variable before running pig: > > export HADOOP_USER_CLASSPATH_FIRST=true > > Should i recompile my PIG code with new guava jar? > > Regards, > Sandeep > > On Wed, Sep 2, 2015 at 1:20 PM, Jianfeng (Jeff) Zhang > wrote: > > It looks like classpath issue due guava version. Could you check the guava > version CDH 5.4.4 is using ? AFAIK, Tez use guava 11.02 and pig use guava > 11.0 > > > > Best Regard, > Jeff Zhang > > > From: Sandeep Kumar > Reply-To: "user@tez.apache.org" > Date: Wednesday, September 2, 2015 at 2:28 PM > To: "user@tez.apache.org" > Subject: Pig(0.14.0) on Tez(0.7.0) > > Hi Guys, > > I've a pig script which is as follows: > > map = LOAD 'input/' using > com.RawPigLoader('conf/Map.xml','conf/R360MapSignalling.json','csv'); > normalized_map_data = foreach r360map generate flatten(com.Map($0..)); > > data_grouped_over_event_time = GROUP normalized_map_data by > (((startTime/1000) / 3600) * 3600) PARTITION BY com.CustomTimePartitioner; > final_data = foreach data_grouped_over_event_time generate flatten($1); > >
Re: Pig(0.14.0) on Tez(0.7.0)
Pig 0.14 was released around the time when tez-0.5 was the stable release. Tez 0.7 is compatible with tez 0.5 so pig 0.14 should work with it. This is a question which you should also ask on the pig mailing lists ( I don’t believe anyone from the Pig community has raised any bugs in this area ). thanks — Hitesh On Sep 2, 2015, at 9:48 PM, Sandeep Kumar <sandeepdas@gmail.com> wrote: > As you correctly pointed out there was issue of guava library only. In my > code there were some UDFs which were using guava-0.16.0.jar. > > I've removed it and now there are no exceptions. > > Just for curiosity. Can i use tez-0.7.0 with latest PIG-0.14.0? Is it tested > earlier? > > Regards, > Sandeep > > On Wed, Sep 2, 2015 at 8:44 PM, Hitesh Shah <hit...@apache.org> wrote: > Based on the stack trace, the following issue seems to be the cause: > > Caused by: java.lang.NoSuchMethodError: > com.google.common.base.Stopwatch.elapsedTime(Ljava/util/concurrent/TimeUnit;)J > at > org.apache.tez.runtime.library.common.shuffle.HttpConnection.validate(HttpConnection.java:221) > at > org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.setupConnection(FetcherOrderedGrouped.java:328) > at > org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.copyFromHost(FetcherOrderedGrouped.java:245) > at > org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.run(FetcherOrderedGrouped.java:167) > > This means that the guava library version is picking up a new version at > runtime. > > A quick test is to run say orderedwordcount from tez-examples to verify > standalone Tez has no issues. Also, you may wish to check the contents of > “tez.lib.uris” to verify that it has guava-11.0.2. > > If you are familiar with using/debugging YARN, set > "yarn.nodemanager.delete.debug-delay-sec” to a value such as 1200. Now, pick > a host on which the Tez container ran in which a task failed. > > Using the directories specified in "yarn.nodemanager.local-dirs” in > yarn-site.xml, search for a “launch_container.sh” under the > container-specific directory mapping to the tez container above. The contents > of this will tell you which guava library is being symlinked into the > container space and used in the classpath. Having 2 guava jars is also a > problem as either could be picked. > > thanks > — Hitesh > > > On Sep 2, 2015, at 5:18 AM, Sandeep Kumar <sandeepdas@gmail.com> wrote: > > > Thanks for your responses. I was mistaken that there is any compatibility > > issue. Its the same error when i run PIG-0.14.0 over Tez-0.5.2. > > > > HadoopVersion: 2.6.0-cdh5.4.4 > > PigVersion: 0.14.0 > > TezVersion: 0.5.2 > > > > PFA the exception stack trace. > > > > > > On Wed, Sep 2, 2015 at 3:11 PM, Jianfeng (Jeff) Zhang > > <jzh...@hortonworks.com> wrote: > > >>> I could not use tez-0.5.2 because it was not compatible with > > >>> Hadoop-2.6.0. > > > > What incompatible do you see ? > > > > > > Best Regard, > > Jeff Zhang > > > > > > From: Sandeep Kumar <sandeepdas@gmail.com> > > Reply-To: "user@tez.apache.org" <user@tez.apache.org> > > Date: Wednesday, September 2, 2015 at 5:18 PM > > > > To: "user@tez.apache.org" <user@tez.apache.org> > > Subject: Re: Pig(0.14.0) on Tez(0.7.0) > > > > Yes i did change PIG/ivy/libraries.propeties to compile it with tez-0.7.0 > > and also changed pig to compile with Hadoop-core-2.6.0. > > > > I could not use tez-0.5.2 because it was not compatible with Hadoop-2.6.0. > > > > I'm compiling my code of PIG using same command: ant clean jar > > -Dhadoopversion=23 > > > > > > > > On Wed, Sep 2, 2015 at 2:36 PM, Jianfeng (Jeff) Zhang > > <jzh...@hortonworks.com> wrote: > > > > Not sure how did you compile pig with tez 0.7.0, did you change the tez > > version in PIG/ivy/libraries.propeties ? > > > > And make sure you build pig with hadoop version, by default, pig build with > > hadoop-1.x. Use the following command to build pig with hadoop-2.x > > > > >> ant clean jar -Dhadoopversion=23 > > > > > > > > Best Regard, > > Jeff Zhang > > > > > > From: Sandeep Kumar <sandeepdas@gmail.com> > > Reply-To: "user@tez.apache.org" <user@tez.apache.org> > > Date: Wednesday, September 2, 2015 at 4:27 PM > > To: "user@tez.apache.org" <user@tez.a
Re: building Tez master branch fail
Actually protobuf version is still 2.5.0 on master. Hadoop forces everyone to use 2.5.0 for compatibility. 2.5.0 Please check to see whether protoc is firstly in your path and confirm that "protoc —version” returns 2.5.0 thanks — Hitesh On Sep 2, 2015, at 1:47 AM, Sandeep Kumarwrote: > In case you are trying to compile master branch then its pom.xml refers to > 2.6.0 protoc. So, you have to install protoc 2.6.0 on your host to compile > tez master branch. AFAIK tez 0.7.0 uses older version of protoc 2.5.0. > > > > On Wed, Sep 2, 2015 at 2:08 PM, Xiaoyong Zhu wrote: > Hi experts > > > > > > I am trying to build the tez master branch but it always gives me the error > message (it seems that the pom file has something wrong?) > > > > I am sure I am using protobuf 2.5, and not sure why there’s such kind of > issue…the log is also attached. > > > > Thanks! > > > > [INFO] --- hadoop-maven-plugins:2.6.0:protoc (compile-protoc) @ tez-api --- > > [DEBUG] org.apache.hadoop:hadoop-maven-plugins:jar:2.6.0: > > [DEBUG]org.apache.maven:maven-plugin-api:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-model:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-artifact:jar:3.0:compile > > [DEBUG] org.sonatype.sisu:sisu-inject-plexus:jar:1.4.2:compile > > [DEBUG] org.sonatype.sisu:sisu-inject-bean:jar:1.4.2:compile > > [DEBUG] org.sonatype.sisu:sisu-guice:jar:noaop:2.1.7:compile > > [DEBUG]org.apache.maven:maven-core:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-settings:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-settings-builder:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-repository-metadata:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-model-builder:jar:3.0:compile > > [DEBUG] org.apache.maven:maven-aether-provider:jar:3.0:runtime > > [DEBUG] org.sonatype.aether:aether-impl:jar:1.7:compile > > [DEBUG] org.sonatype.aether:aether-spi:jar:1.7:compile > > [DEBUG] org.sonatype.aether:aether-api:jar:1.7:compile > > [DEBUG] org.sonatype.aether:aether-util:jar:1.7:compile > > [DEBUG] org.codehaus.plexus:plexus-interpolation:jar:1.14:compile > > [DEBUG] org.codehaus.plexus:plexus-utils:jar:2.0.5:compile (version > managed from 2.0.4) > > [DEBUG] org.codehaus.plexus:plexus-classworlds:jar:2.2.3:compile > > [DEBUG] > org.codehaus.plexus:plexus-component-annotations:jar:1.5.5:compile > > [DEBUG] org.sonatype.plexus:plexus-sec-dispatcher:jar:1.3:compile > > > > > > > > [ERROR] Failed to execute goal > org.apache.hadoop:hadoop-maven-plugins:2.6.0:protoc (compile-protoc) on > project tez-api: org.apache.maven.plugin.MojoExecutionException: protoc > failure -> [Help 1] > > org.apache.maven.lifecycle.LifecycleExecutionException: Failed to execute > goal org.apache.hadoop:hadoop-maven-plugins:2.6.0:protoc (compile-protoc) on > project tez-api: org.apache.maven.plugin.MojoExecutionException: protoc > failure > > at > org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:217) > > at > org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:153) > > at > org.apache.maven.lifecycle.internal.MojoExecutor.execute(MojoExecutor.java:145) > > at > org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:84) > > at > org.apache.maven.lifecycle.internal.LifecycleModuleBuilder.buildProject(LifecycleModuleBuilder.java:59) > > at > org.apache.maven.lifecycle.internal.LifecycleStarter.singleThreadedBuild(LifecycleStarter.java:183) > > at > org.apache.maven.lifecycle.internal.LifecycleStarter.execute(LifecycleStarter.java:161) > > at org.apache.maven.DefaultMaven.doExecute(DefaultMaven.java:320) > > at org.apache.maven.DefaultMaven.execute(DefaultMaven.java:156) > > at org.apache.maven.cli.MavenCli.execute(MavenCli.java:537) > > at org.apache.maven.cli.MavenCli.doMain(MavenCli.java:196) > > at org.apache.maven.cli.MavenCli.main(MavenCli.java:141) > > 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.codehaus.plexus.classworlds.launcher.Launcher.launchEnhanced(Launcher.java:290) > > at > org.codehaus.plexus.classworlds.launcher.Launcher.launch(Launcher.java:230) > > at > org.codehaus.plexus.classworlds.launcher.Launcher.mainWithExitCode(Launcher.java:409) > > at > org.codehaus.plexus.classworlds.launcher.Launcher.main(Launcher.java:352) > >
Re: Add tez dependency
You need to add tez-api as a dependency i.e set artifactId to tez-api. Also, you are better off using a released version as a dependency e.g 0.7.0 instead of a SNAPSHOT which will change regularly. — Hitesh On Sep 1, 2015, at 12:40 PM, Raajaywrote: > How to add Tez as a dependency to another project ? > > I have a Tez client in a separate project and want to use Tez classes in it. > I added the dependency to pom.xml for the application project as below > > > org.apache.tez > tez > 0.8.0-SNAPSHOT > > > > The commmand "mvn compile" on the client gives errors. > > "Could not find artifact org.apache.tez:tez:jar:0.8.0-SNAPSHOT -> [Help 1]" > > > I am not able to install tez libraries to the local maven repo. > > Thanks > Raajay
Re: How to profile tez task?
@Rajesh, could you please this helpful set of points to the Tez wiki? thanks — Hitesh On Aug 25, 2015, at 1:36 AM, Rajesh Balamohan rbalamo...@apache.org wrote: Another example: tez.task-specific.launch.cmd-opts.list=Map 1[1,10,20] would profile tasks 1, 10, 20 tasks in Map 1. If you need to specify for multiple vertices, just add it with ,. E.g Map 1[1,10,20],Map 2[5,10] and so on. ~Rajesh.B On Tue, Aug 25, 2015 at 2:04 PM, Rajesh Balamohan rbalamo...@apache.org wrote: You can use tez.task-specific.launch.cmd-opts.list and tez.task-specific.launch.cmd-opts to specify which tasks in which vertices should be profiled. E.g --hiveconf tez.task-specific.launch.cmd-opts.list=“M5[0] --hiveconf tez.task-specific.launch.cmd-opts=-agentpath:/opt/yourkit/bin/linux-x86-64/libyjpagent.so=disablej2ee,tracing,alloceach=1000,onexit=snapshot,tracing_settings_path=/tmp/walltime.txt,dir=/tmp/__VERTEX_NAME__/__TASK_INDEX__ Here, Task 0 in vertex M5 would be profiled (i.e the set of profiling options would be added to JVM start up). Once the profiling is done, it would store the snapshot in /tmp/M5/0 directory in the node where M5-task0 got executed. Basically __VERTEX_NAME__ would be replaced by vertexName and __TASK_INDEX__ would be replaced by task number. ~Rajesh.B On Tue, Aug 25, 2015 at 1:57 PM, r7raul1...@163.com r7raul1...@163.com wrote: When I use mr , I can enable profile by set follow config: set mapreduce.task.profile=true; set mapreduce.task.profile.maps=0-; set mapreduce.task.profile.reduces=0-; set mapreduce.task.profile.params=-Xprof; or set mapreduce.task.profile.params=-agentpath:/usr/jprofile/jprofiler9/bin/linux-x64/libjprofilerti.so=port=8849,nowait; In tez how to enable profile? r7raul1...@163.com
Re: Tez UI Kerberos
You will first need to do a kinit and then start a new firefox session with the following config “network.negotiate-auth.trusted-uris“ set up as needed. Ref on setting up firefox: http://people.redhat.com/mikeb/negotiate/ http://docs.oracle.com/cd/E41633_01/pt853pbh1/eng/pt/tsec/task_EnablingKerberosAuthenticationinFirefox-836673.html — Hitesh On Aug 20, 2015, at 1:00 PM, Gagan Brahmi gaganbra...@gmail.com wrote: Does anyone has an idea how to enable Tez UI in a kerberos enabled environment? I am hosting tez UI on apache and the ATS is secured. I am getting errors where Tez UI is not able to retrieve the data from Timeline server. Couldn't find any documentation which can provide help with this one. Regards, Gagan Brahmi
Re: tez compile
For the most part, yes - the compile should likely work given that the CDH release is based off Apache Hadoop 2.3. To build Tez, you will need to add the CDH maven repo to the pom.xml at the top-level. Modify hadoop.version to 2.3.0-cdh5.1.0” or something similar based on your requirements. And then run mvn package -P\!hadoop24 -P\!hadoop26 so that you disable all the features/modules that need a minimum of hadoop-2.4 and hadoop-2.6 respectively. Let us know if you run into any problems. thanks — Hitesh On Aug 20, 2015, at 3:29 AM, hanked...@emar.com wrote: Hi~ all Can use cdh-hadoop 2.3 to compile tez 7.0 ? Wirtten in the building.txt is support, lowest hadoop version 2.2 ? Now, my cluster with CDH 5.1 version, can use tez ? Thanks ! hanked...@emar.com
Re: Tez UI Kerberos
The other approach to this is to have an intermediate proxy that is authenticated and can make calls to the ATS server. The Tez UI makes 2 kind of calls to the backend services ( one set to YARN ResourceManager and the other to Timeline server ) so both of these calls would need to be proxied through a server/process that is kerberos authenticated. A simple option for this would be try using Ambari in standalone mode. In this mode, the ambari-server acts as an authenticated proxy for the UI. This involves installing the Ambari Server, setting it up to work in a secure mode and instantiating the Tez View within it. All your users can then use the Tez UI within Ambari without needing any kerberos auth. Ambari-server would be started with kerberos auth and you need to configure that user as a hadoop proxy user. To be clear, this does *not* require you to set up your Hadoop cluster using Ambari. http://docs.hortonworks.com/HDPDocuments/Ambari-2.1.0.0/bk_ambari_views_guide/content/ch_running_ambari_standalone.html http://docs.hortonworks.com/HDPDocuments/Ambari-2.1.0.0/bk_ambari_views_guide/content/ch_configuring_views_for_kerberos.html http://docs.hortonworks.com/HDPDocuments/Ambari-2.1.0.0/bk_ambari_views_guide/content/ch_using_tez_view.html thanks — Hitesh On Aug 20, 2015, at 2:41 PM, Gagan Brahmi gaganbra...@gmail.com wrote: Thanks Hitesh, but I don't want the UI to be accessed through Kerberos. Client to Tez UI communication should be without kerberos and Tez UI to ATS will be over Kerberos. Anyone accomplished this before? On Thu, Aug 20, 2015 at 1:57 PM, Hitesh Shah hit...@apache.org wrote: You will first need to do a kinit and then start a new firefox session with the following config “network.negotiate-auth.trusted-uris“ set up as needed. Ref on setting up firefox: http://people.redhat.com/mikeb/negotiate/ http://docs.oracle.com/cd/E41633_01/pt853pbh1/eng/pt/tsec/task_EnablingKerberosAuthenticationinFirefox-836673.html — Hitesh On Aug 20, 2015, at 1:00 PM, Gagan Brahmi gaganbra...@gmail.com wrote: Does anyone has an idea how to enable Tez UI in a kerberos enabled environment? I am hosting tez UI on apache and the ATS is secured. I am getting errors where Tez UI is not able to retrieve the data from Timeline server. Couldn't find any documentation which can provide help with this one. Regards, Gagan Brahmi
Re: Logical to Physical DAG conversion
If you are mainly looking at this from a “mapper” and “reducer” perspective, there are 2 main ways in which Tez affects parallelism: i) For the mapper, the no. of maps is effectively governed by how many splits are created and how are splits assigned to tasks. The initial split creation is controlled by the InputSplitFormat. Tez then looks at the cluster capacity to decide whether the no. of splits are optimal. For example, the InputFormat could create 1000 splits which would imply a 1000 tasks. However, if the max no. of containers that can be launched is only 50, this would imply 20 waves of tasks for the full stage/vertex to complete. So in this case, if grouping is enabled, Tez tries and re-shuffles the splits to around 1.5 waves ( 1.5 is configurable ) but also ensures that splits are kept within a bounded range with respect to the amount of data being processed. For example, if each split was processing 1 TB of data, there is no need to group splits as 1 TB is already too large. Whereas if each split was 1 MB, then Tez will likely end bundling up more and more splits to be processed by a single task until a min limit is reached on a per task basis. Disabling grouping would allow you to control no. of tasks by configuring the InputSplit as needed to a more static nature. 2) For reducers, the main problem everyone has is that today you could configure a 1000 reducers but the mappers generated only 10 MB of data in total. This could be processed by a single reducer. Tez monitors the data being generated from the mappers and dynamically tries to decide how many reducers it can scale down to. This is the auto-reduce parallelism that Jeff referred to in his email earlier. Again, a static configuration for the reducer stage i.e setParallelism() and disabling auto-reduce would stop Tez from making any runtime changes. To answer your first question, each stage/vertex is governed by its own VertexManager which in the end is user-code ( most of the common ones are implemented within the framework but can be overridden if needed). These VMs can be as dumb or as complex as possible and can make runtime decisions. There are certain points in the processing up to which the VM can tweak parallelism. After a certain point, parallelism becomes final ( due to the requirement that failures need to be handled in a deterministic manner such that re-running a task results in the same data output each time around ). thanks — Hitesh On Aug 19, 2015, at 3:24 AM, Raajay Viswanathan raaja...@gmail.com wrote: Hello, I am just getting started with Tez and need some clarification about the logical to physical DAG conversion in Tez. Does Tez convert a Logical DAG to physical DAG at one go or are the number of mappers and reducers for a stage determined only when the stage is ready to run ? Also, what are some rules of thumb regarding the number of mappers/reducers for a stage ? I would ideally like to fix the #mappers / #reducers in the application itself rather than let Tez determine it. What are the common pitfalls in doing so ? Thanks, Raajay
Re: Error compiling tez (0.6.1 and 0.7.0) from scratch
Thanks for digging into the issue, Rajat. Mind filing a jira for this and uploading your patch ( without the frontend plugin change)? thanks — Hitesh On Jul 15, 2015, at 3:53 PM, Rajat Jain rajat...@gmail.com wrote: I run this command: mvn clean install -DskipTests It automatically picks up Hadoop 2.6.0 from the pom.xml. I also thought hadoop-auth would be implicitly included, but apparently thats not the case. On Wed, Jul 15, 2015 at 3:51 PM, Jianfeng (Jeff) Zhang jzh...@hortonworks.com wrote: hadoop-auth should be implicitly included as a dependency of hadoop-common. Do you specify hadoop version when building it ? Best Regard, Jeff Zhang From: Rajat Jain rajat...@gmail.com Reply-To: Rajat Jain rajat...@gmail.com Date: Wednesday, July 15, 2015 at 3:26 PM To: Jianfeng Zhang jzh...@hortonworks.com Cc: user@tez.apache.org user@tez.apache.org Subject: Re: Error compiling tez (0.6.1 and 0.7.0) from scratch @@ -424,6 +424,11 @@ /dependency dependency groupIdorg.apache.hadoop/groupId +artifactIdhadoop-auth/artifactId +version${hadoop.version}/version + /dependency + dependency +groupIdorg.apache.hadoop/groupId artifactIdhadoop-yarn-common/artifactId version${hadoop.version}/version exclusions
Re: fails to alter table concatenate
Move to user@hive. BCC’ed user@tez. — Hitesh On Jun 30, 2015, at 1:44 AM, patcharee patcharee.thong...@uni.no wrote: Hi, I am using hive 0.14 + tez 0.5. It fails to alter table concatenate occasionally (see the exception below). It is strange that it fails from time to time not predictable. However, it works on mr. Is there any suggestion/clue? hive alter table 4dim partition(zone=2,z=15,year=2005,month=4) CONCATENATE; VERTICES STATUS TOTAL COMPLETED RUNNING PENDING FAILED KILLED File MergeFAILED -1 00 -1 0 0 VERTICES: 00/01 [--] 0%ELAPSED TIME: 1435651968.00 s Status: Failed Vertex failed, vertexName=File Merge, vertexId=vertex_1435307579867_0041_1_00, diagnostics=[Vertex vertex_1435307579867_0041_1_00 [File Merge] killed/failed due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: [hdfs://service-10-0.local:8020/apps/hive/warehouse/wrf_tables/4dim/zone=2/z=15/year=2005/month=4] initializer failed, vertex=vertex_1435307579867_0041_1_00 [File Merge], java.lang.NullPointerException at org.apache.hadoop.hive.ql.io.HiveInputFormat.init(HiveInputFormat.java:265) at org.apache.hadoop.hive.ql.io.CombineHiveInputFormat.getSplits(CombineHiveInputFormat.java:452) at org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateOldSplits(MRInputHelpers.java:441) at org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateInputSplitsToMem(MRInputHelpers.java:295) at org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.initialize(MRInputAMSplitGenerator.java:124) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:245) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:239) at java.security.AccessController.doPrivileged(Native Method) at javax.security.auth.Subject.doAs(Subject.java:415) at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1628) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:239) at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:226) 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) ] DAG failed due to vertex failure. failedVertices:1 killedVertices:0 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.DDLTask BR, Patcharee
Re: Error while running Hive queries over tez
A couple of things to do: 1) (optional) For tez.lib.uris, set it to ${fs.defaultFS}/apps/tez/tez.tar.gz” - this tez.tar.gz should come from tez-dist/target/. Given that your basic job is working, you can ignore this for now but it is the recommended way to deploy tez. 2) On a fresh setup, do a hadoop dfs -mkdir -p /tmp/hive and set permissions to 777 for both /tmp and /tmp/hive before running any job. You should probably check whether your setup is using a different dfs.umask that is causing write permissions later on. thanks — Hitesh On Jun 24, 2015, at 9:20 PM, amit kumar aks...@gmail.com wrote: Below is the tez-site.xml configuration property nametez.lib.uris/name value${fs.defaultFS}/apps/tez-0.7.0,${fs.defaultFS}/apps/tez-0.7.0/lib//value /property /configuration I ran OrderWordCount job and that run successfully, only with hive queries i am facing this error. Please let me know if i missing something. Thanks. On Thu, Jun 25, 2015 at 4:04 AM, Jianfeng (Jeff) Zhang jzh...@hortonworks.com wrote: Did you configure tez.lib.uris in tez-site.xml and put it under classpath ? It looks like you didn’t configure it. Best Regard, Jeff Zhang From: amit kumar aks...@gmail.com Reply-To: user@tez.apache.org user@tez.apache.org Date: Thursday, June 25, 2015 at 11:58 AM To: user@tez.apache.org user@tez.apache.org Subject: Re: Error while running Hive queries over tez No, I didn't add anything after first query, its like the hive query runs on fresh cluster and after that below error is shown for all queries. Below are the hive logs: 2015-06-24 05:12:50,163 INFO [main]: log.PerfLogger (PerfLogger.java:PerfLogBegin(108)) - PERFLOG method=task.TEZ.Stage-1 from=org.apache.hadoop.hive.ql.Driver 2015-06-24 05:12:50,163 INFO [main]: ql.Driver (SessionState.java:printInfo(824)) - Launching Job 1 out of 1 2015-06-24 05:12:50,163 INFO [main]: ql.Driver (Driver.java:launchTask(1602)) - Starting task [Stage-1:MAPRED] in serial mode 2015-06-24 05:12:50,163 INFO [main]: tez.TezSessionPoolManager (TezSessionPoolManager.java:canWorkWithSameSession(217)) - The current user: amitkumar, session user: amitkumar 2015-06-24 05:12:50,163 INFO [main]: tez.TezSessionPoolManager (TezSessionPoolManager.java:canWorkWithSameSession(234)) - Current queue name is null incoming queue name is null 2015-06-24 05:12:50,190 INFO [main]: ql.Context (Context.java:getMRScratchDir(266)) - New scratch dir is hdfs://localhost:9000/tmp/hive/amitkumar/5dcd0855-107b-471e-8ae6-a332c8b7c919/hive_2015-06-24_05-12-50_013_722448017482877635-1 2015-06-24 05:12:50,204 INFO [main]: exec.Task (TezTask.java:updateSession(236)) - Session is already open 2015-06-24 05:12:50,207 INFO [main]: tez.DagUtils (DagUtils.java:localizeResource(951)) - Localizing resource because it does not exist: file:/home/amitkumar/Apache_tez/TEZ/tez-0.7.0 to dest: hdfs://localhost:9000/tmp/hive/amitkumar/_tez_session_dir/25e6f3e0-b913-46c8-9d87-83962f1dc073/tez-0.7.0 2015-06-24 05:12:50,212 INFO [main]: tez.DagUtils (DagUtils.java:localizeResource(955)) - Looks like another thread is writing the same file will wait. 2015-06-24 05:12:50,212 INFO [main]: tez.DagUtils (DagUtils.java:localizeResource(962)) - Number of wait attempts: 5. Wait interval: 5000 2015-06-24 05:13:15,239 ERROR [main]: tez.DagUtils (DagUtils.java:localizeResource(978)) - Could not find the jar that was being uploaded 2015-06-24 05:13:15,240 ERROR [main]: exec.Task (TezTask.java:execute(184)) - Failed to execute tez graph. java.io.IOException: Previous writer likely failed to write hdfs://localhost:9000/tmp/hive/amitkumar/_tez_session_dir/25e6f3e0-b913-46c8-9d87-83962f1dc073/tez-0.7.0. Failing because I am unlikely to write too. at org.apache.hadoop.hive.ql.exec.tez.DagUtils.localizeResource(DagUtils.java:979) at org.apache.hadoop.hive.ql.exec.tez.DagUtils.addTempResources(DagUtils.java:860) at org.apache.hadoop.hive.ql.exec.tez.DagUtils.localizeTempFilesFromConf(DagUtils.java:803) at org.apache.hadoop.hive.ql.exec.tez.TezSessionState.refreshLocalResourcesFromConf(TezSessionState.java:228) at org.apache.hadoop.hive.ql.exec.tez.TezTask.updateSession(TezTask.java:245) at org.apache.hadoop.hive.ql.exec.tez.TezTask.execute(TezTask.java:136) at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:160) at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:85) at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1604) at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1364) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1177) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1004) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:994) at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:201) at
Re: Error while running Hive queries over tez
The error seems to indicate that the create dir/write to HDFS failed. Can you check compare what user you are running as and whether the user has permissions to create/write to the directories in the path below? Furthermore, you may wish to check if the datanodes are alive and also look for errors earlier in the logs to see what failed ( due to the bit related to Previous writer likely failed” ) . thanks — Hitesh On Jun 24, 2015, at 5:19 AM, amit kumar aks...@gmail.com wrote: java.io.IOException: Previous writer likely failed to write hdfs://localhost:9000/tmp/hive/amitkumar/_tez_session_dir/25e6f3e0-b913-46c8-9d87-83962f1dc073/tez-0.7.0.
Re: Error while building Tez UI
Another option would be https://repository.apache.org/content/repositories/releases/org/apache/tez/tez-ui/ to get the tez-ui.war — Hitesh On Jun 22, 2015, at 9:52 PM, amit kumar aks...@gmail.com wrote: Hi, I resolved the nodejs issue but now stuck in bower installtion as error shows below: bower ember-data#1.0.0-beta.11ECMDERR Failed to execute git ls-remote --tags --heads git://github.com/components/ember-data.git, exit code of #128. I just need to explore Tez to understand if it can serve the purpose of our product, So if there is any location from where I can download the jar (artifact) directly instead of building it. It would be very helpful. Thanks. On Fri, Jun 19, 2015 at 3:31 PM, Hitesh Shah hit...@apache.org wrote: The build tries to download node and then install it. Did the download fail for some reason? You can try running in debug mode i.e. “mvn -X” to see why it could be failing. — Hitesh On Jun 19, 2015, at 8:17 AM, amit kumar aks...@gmail.com wrote: Hi, I am trying to build tez-0.7.0, but can not build Tez UI. [ERROR] Failed to execute goal com.github.eirslett:frontend-maven-plugin:0.0.22:install-node-and-npm (install node and npm) on project tez-ui: Could not extract the Node archive: Could not extract archive: '/home/amitkumar/Apache_tez/setUp/tez/tez-ui/src/main/webapp/node_tmp/node.tar.gz': EOFException - [Help 1] what could be the reason for this this failure? Thanks.
Re: hive 1.1.0 on tez0.53 error
Yes something along those lines. This might help a bit more: http://techtonka.com/?p=174 thanks — Hitesh On Jun 18, 2015, at 6:44 PM, r7raul1...@163.com wrote: log like this hdfs-audit.log.9 ? r7raul1...@163.com From: Hitesh Shah Date: 2015-06-19 02:28 To: user Subject: Re: hive 1.1.0 on tez0.53 error Also, if you have access to the name node audit logs, can you search for all accesses of /tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/“ directory and see if/when someone tried to delete it? thanks — Hitesh On Jun 17, 2015, at 7:57 PM, r7raul1...@163.com wrote: Here is hive log: Status: Running (Executing on YARN cluster with App id application_1433219182593_180456) Map 1: -/- Reducer 2: 0/5 Reducer 3: 0/5 Map 1: 0(+1)/1 Reducer 2: 0/5 Reducer 3: 0/5 Map 1: 1/1 Reducer 2: 0(+1)/5 Reducer 3: 0/5 Map 1: 1/1 Reducer 2: 0(+5)/5 Reducer 3: 0/5 Map 1: 1/1 Reducer 2: 2(+3)/5 Reducer 3: 0/5 Map 1: 1/1 Reducer 2: 4(+1)/5 Reducer 3: 0(+5)/5 Map 1: 1/1 Reducer 2: 5/5 Reducer 3: 0(+5)/5 Map 1: 1/1 Reducer 2: 5/5 Reducer 3: 5/5 Loading data to table testtmp.tmp_pm_cpttr_hot_srch partition (cur_flg=0, ds=2015-06-16) Partition testtmp.tmp_pm_cpttr_hot_srch{cur_flg=0, ds=2015-06-16} stats: [numFiles=5, numRows=0, totalSize=0, rawDataSize=0] OK Time taken: 3.885 seconds OK Time taken: 0.266 seconds OK Time taken: 0.067 seconds Query ID = lujian_2015061718_f048ad51-d72f-458f-8480-bef366606a68 Total jobs = 1 Launching Job 1 out of 1 Status: Running (Executing on YARN cluster with App id application_1433219182593_180456) Map 1: 0/1 Map 2: -/- Map 1: 0/1 Map 2: 0/1 Map 1: 0/1 Map 2: 0/1 Map 1: 0(+0,-1)/1 Map 2: 0(+0,-1)/1 Map 1: 0(+0,-1)/1 Map 2: 0(+0,-1)/1 Map 1: 0(+0,-2)/1 Map 2: 0(+0,-2)/1 Map 1: 0(+0,-2)/1 Map 2: 0(+0,-2)/1 Map 1: 0(+0,-3)/1 Map 2: 0(+0,-3)/1 Status: Failed Vertex failed, vertexName=Map 2, vertexId=vertex_1433219182593_180456_3_01, diagnostics=[Task failed, taskId=task_1433219182593_180456_3_01_00, diagnostics=[TaskAttempt 0 failed, info=[Container container_1433219182593_180456_01_14 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 1 failed, info=[Container container_1433219182593_180456_01_16 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 2 failed, info=[Container container_1433219182593_180456_01_18 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 3 failed, info=[Container container_1433219182593_180456_01_20 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1433219182593_180456_3_01 [Map 2] killed/failed due to:null] Vertex killed, vertexName=Map 1, vertexId=vertex_1433219182593_180456_3_00, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1433219182593_180456_3_00 [Map 1] killed/failed due to:null] DAG failed due to vertex failure. failedVertices:1 killedVertices:1 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask I think maybe first successfule job delete tez-conf.pb ? r7raul1...@163.com From: Hitesh Shah Date: 2015-06-18 10:46 To: user Subject: Re: hive 1.1.0 on tez0.53 error That particular log is a red herring and not really an issue that is causing the failure. The main problem based on the log is this: 2015-06-17 18:00:43,543 INFO [AsyncDispatcher event handler] history.HistoryEventHandler: [HISTORY][DAG:dag_1433219182593_180456_3][Event:DAG_FINISHED]: dagId=dag_1433219182593_180456_3, startTime=1434535228467, finishTime=1434535243529, timeTaken=15062, status=FAILED, diagnostics=Vertex failed, vertexName=Map 2, vertexId=vertex_1433219182593_180456_3_01, diagnostics=[Task failed, taskId=task_1433219182593_180456_3_01_00, diagnostics=[TaskAttempt 0 failed, info=[Container container_1433219182593_180456_01_14 finished with diagnostics set to [Container failed. File does
Re: Error while building Tez UI
The build tries to download node and then install it. Did the download fail for some reason? You can try running in debug mode i.e. “mvn -X” to see why it could be failing. — Hitesh On Jun 19, 2015, at 8:17 AM, amit kumar aks...@gmail.com wrote: Hi, I am trying to build tez-0.7.0, but can not build Tez UI. [ERROR] Failed to execute goal com.github.eirslett:frontend-maven-plugin:0.0.22:install-node-and-npm (install node and npm) on project tez-ui: Could not extract the Node archive: Could not extract archive: '/home/amitkumar/Apache_tez/setUp/tez/tez-ui/src/main/webapp/node_tmp/node.tar.gz': EOFException - [Help 1] what could be the reason for this this failure? Thanks.
Re: hive 1.1.0 on tez0.53 error
Also, if you have access to the name node audit logs, can you search for all accesses of /tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/“ directory and see if/when someone tried to delete it? thanks — Hitesh On Jun 17, 2015, at 7:57 PM, r7raul1...@163.com wrote: Here is hive log: Status: Running (Executing on YARN cluster with App id application_1433219182593_180456) Map 1: -/-Reducer 2: 0/5 Reducer 3: 0/5 Map 1: 0(+1)/1Reducer 2: 0/5 Reducer 3: 0/5 Map 1: 1/1Reducer 2: 0(+1)/5 Reducer 3: 0/5 Map 1: 1/1Reducer 2: 0(+5)/5 Reducer 3: 0/5 Map 1: 1/1Reducer 2: 2(+3)/5 Reducer 3: 0/5 Map 1: 1/1Reducer 2: 4(+1)/5 Reducer 3: 0(+5)/5 Map 1: 1/1Reducer 2: 5/5 Reducer 3: 0(+5)/5 Map 1: 1/1Reducer 2: 5/5 Reducer 3: 5/5 Loading data to table testtmp.tmp_pm_cpttr_hot_srch partition (cur_flg=0, ds=2015-06-16) Partition testtmp.tmp_pm_cpttr_hot_srch{cur_flg=0, ds=2015-06-16} stats: [numFiles=5, numRows=0, totalSize=0, rawDataSize=0] OK Time taken: 3.885 seconds OK Time taken: 0.266 seconds OK Time taken: 0.067 seconds Query ID = lujian_2015061718_f048ad51-d72f-458f-8480-bef366606a68 Total jobs = 1 Launching Job 1 out of 1 Status: Running (Executing on YARN cluster with App id application_1433219182593_180456) Map 1: 0/1Map 2: -/- Map 1: 0/1Map 2: 0/1 Map 1: 0/1Map 2: 0/1 Map 1: 0(+0,-1)/1 Map 2: 0(+0,-1)/1 Map 1: 0(+0,-1)/1 Map 2: 0(+0,-1)/1 Map 1: 0(+0,-2)/1 Map 2: 0(+0,-2)/1 Map 1: 0(+0,-2)/1 Map 2: 0(+0,-2)/1 Map 1: 0(+0,-3)/1 Map 2: 0(+0,-3)/1 Status: Failed Vertex failed, vertexName=Map 2, vertexId=vertex_1433219182593_180456_3_01, diagnostics=[Task failed, taskId=task_1433219182593_180456_3_01_00, diagnostics=[TaskAttempt 0 failed, info=[Container container_1433219182593_180456_01_14 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 1 failed, info=[Container container_1433219182593_180456_01_16 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 2 failed, info=[Container container_1433219182593_180456_01_18 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 3 failed, info=[Container container_1433219182593_180456_01_20 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]]], Vertex failed as one or more tasks failed. failedTasks:1, Vertex vertex_1433219182593_180456_3_01 [Map 2] killed/failed due to:null] Vertex killed, vertexName=Map 1, vertexId=vertex_1433219182593_180456_3_00, diagnostics=[Vertex received Kill while in RUNNING state., Vertex killed as other vertex failed. failedTasks:0, Vertex vertex_1433219182593_180456_3_00 [Map 1] killed/failed due to:null] DAG failed due to vertex failure. failedVertices:1 killedVertices:1 FAILED: Execution Error, return code 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask I think maybe first successfule job delete tez-conf.pb ? r7raul1...@163.com From: Hitesh Shah Date: 2015-06-18 10:46 To: user Subject: Re: hive 1.1.0 on tez0.53 error That particular log is a red herring and not really an issue that is causing the failure. The main problem based on the log is this: 2015-06-17 18:00:43,543 INFO [AsyncDispatcher event handler] history.HistoryEventHandler: [HISTORY][DAG:dag_1433219182593_180456_3][Event:DAG_FINISHED]: dagId=dag_1433219182593_180456_3, startTime=1434535228467, finishTime=1434535243529, timeTaken=15062, status=FAILED, diagnostics=Vertex failed, vertexName=Map 2, vertexId=vertex_1433219182593_180456_3_01, diagnostics=[Task failed, taskId=task_1433219182593_180456_3_01_00, diagnostics=[TaskAttempt 0 failed, info=[Container container_1433219182593_180456_01_14 finished with diagnostics set to [Container failed. File does not exist: hdfs://yhd-jqhadoop2.int.yihaodian.com:8020/tmp/hive/lujian/_tez_session_dir/86bc0010-4816-4251-95aa-bb37b8d029da/.tez/application_1433219182593_180456/tez-conf.pb ]], TaskAttempt 1 failed, info=[Container container_1433219182593_180456_01_16 finished
Re: error on hive insert query
Unless you can pinpoint the problem to something Tez specific, hive-specific questions might be better off being asked on user@hive initially as there is a larger group there that understands Hive as compared to the Tez community. FWIW, TezTask error 1 means “something in the Hive layer using Tez or somewhere within Tez itself” had a failure. Without additional logs/stack trace, this is usually impossible to debug/pinpoint where the problem is. thanks — Hitesh On Jun 16, 2015, at 10:19 AM, Sateesh Karuturi sateesh.karutu...@gmail.com wrote: iam using hive 1.0.0 and tez 0.5.2. when i set hive.execution.engine value in hive-site.xml to tezselect query works well... but in case of insert getting error. the query is : insert into table tablename values(intvalue,'string value'); and the error is : FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.tez.Tez Task
Re: hive 1.1.0+tez0.7+hadoop2.5.0-cdh5.2.0 when use TEZ UI throw exception
There might be a better way to approach this. You can add the following to the top-level pom.xml in the repositories section: repository idcloudera-repo/id nameCloudera Repository/name urlhttps://repository.cloudera.com/artifactory/cloudera-repos/url releases enabledtrue/enabled /releases snapshots enabledfalse/enabled /snapshots /repository Change hadoop.version in the pom.xml to match the CDH 5.2 release. Build tez using the following: “-Phadoop24 -P\!hadoop26” i.e. run with the hadoop-2.4 profile which is compatible with 2.4 and 2.5. The jackson issue might still hold but you should be able to easily modify the jackson version in the top level pom as needed. Feel free to contribute the above back to Tez by creating a patch that put these changes under a maven profile. thanks — Hitesh On Jun 4, 2015, at 6:00 PM, Jianfeng (Jeff) Zhang jzh...@hortonworks.com wrote: Tez use jackson 1.9.13, maybe you could try to use 1.9.13 replace 1.8.8. Please backup the 1.8.8 jars before you replace it :) Best Regard, Jeff Zhang From: r7raul1...@163.com r7raul1...@163.com Reply-To: user user@tez.apache.org Date: Friday, June 5, 2015 at 8:38 AM To: user user@tez.apache.org Subject: Re: Re: hive 1.1.0+tez0.7+hadoop2.5.0-cdh5.2.0 when use TEZ UI throw exception /opt/cloudera/parcels/CDH-5.2.0-1.cdh5.2.0.p0.36/lib/hadoop-yarn/lib/jackson-mapper-asl-1.8.8.jar /opt/cloudera/parcels/CDH-5.2.0-1.cdh5.2.0.p0.36/lib/hadoop-yarn/lib/jackson-jaxrs-1.8.8.jar /opt/cloudera/parcels/CDH-5.2.0-1.cdh5.2.0.p0.36/lib/hadoop-yarn/lib/jackson-core-asl-1.8.8.jar /opt/cloudera/parcels/CDH-5.2.0-1.cdh5.2.0.p0.36/lib/hadoop-yarn/lib/jackson-xc-1.8.8.jar r7raul1...@163.com From: Prakash Ramachandran Date: 2015-06-04 17:21 To: user Subject: Re: hive 1.1.0+tez0.7+hadoop2.5.0-cdh5.2.0 when use TEZ UI throw exception I have faced this issue when there was a mismatch in the jackson version (jackson 1.8 api not compatible with 1.9 api – due to abstract class) Not sure what the exact problem is in this case – can you check the version used by hive against the one in hadoop2.5.0-cdh5.2.0 From: r7raul1...@163.com Reply-To: user Date: Thursday, June 4, 2015 at 2:05 PM To: user Subject: hive 1.1.0+tez0.7+hadoop2.5.0-cdh5.2.0 when use TEZ UI throw exception My tez-site.xml is : property descriptionLog history using the Timeline Server/description nametez.history.logging.service.class/name valueorg.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService/value /property property descriptionPublish configuration information to Timeline server.-XX:+UseParallelGC -XX:+UseG1GC /description nametez.runtime.convert.user-payload.to.history-text/name valuetrue/value /property property nametez.allow.disabled.timeline-domains/name valuetrue/value /property . When run query . I see error in log: 2015-06-04 16:28:15,246 INFO [ServiceThread:org.apache.tez.dag.app.web.WebUIService] mortbay.log: jetty-6.1.26 2015-06-04 16:28:15,265 INFO [ServiceThread:org.apache.tez.dag.app.web.WebUIService] mortbay.log: Extract jar:file:/yarn/nm/filecache/50/tez-0.7.0.tar.gz/lib/hadoop-yarn-common-2.5.0-cdh5.2.5.jar!/webapps/ to /tmp/Jetty_0_0_0_0_62340_webapps.pc2ela/webapp 2015-06-04 16:28:15,320 FATAL [HistoryEventHandlingThread] yarn.YarnUncaughtExceptionHandler: Thread Thread[HistoryEventHandlingThread,5,main] threw an Error. Shutting down now... java.lang.AbstractMethodError: org.codehaus.jackson.map.AnnotationIntrospector.findSerializer(Lorg/codehaus/jackson/map/introspect/Annotated;)Ljava/lang/Object; at org.codehaus.jackson.map.ser.BasicSerializerFactory.findSerializerFromAnnotation(BasicSerializerFactory.java:362) at org.codehaus.jackson.map.ser.BeanSerializerFactory.createSerializer(BeanSerializerFactory.java:252) at org.codehaus.jackson.map.ser.StdSerializerProvider._createUntypedSerializer(StdSerializerProvider.java:782) at org.codehaus.jackson.map.ser.StdSerializerProvider._createAndCacheUntypedSerializer(StdSerializerProvider.java:735) at org.codehaus.jackson.map.ser.StdSerializerProvider.findValueSerializer(StdSerializerProvider.java:344) at org.codehaus.jackson.map.ser.StdSerializerProvider.findTypedValueSerializer(StdSerializerProvider.java:420) at org.codehaus.jackson.map.ser.StdSerializerProvider._serializeValue(StdSerializerProvider.java:601) at org.codehaus.jackson.map.ser.StdSerializerProvider.serializeValue(StdSerializerProvider.java:256) at org.codehaus.jackson.map.ObjectMapper.writeValue(ObjectMapper.java:1604) at org.codehaus.jackson.jaxrs.JacksonJsonProvider.writeTo(JacksonJsonProvider.java:527) at com.sun.jersey.api.client.RequestWriter.writeRequestEntity(RequestWriter.java:300) at
Re: hive 1.1.0 tez0.7 hadoop 2.5.0 run query NoClassDefFoundError
If you compiled Tez against hadoop-2.6.0 and are deploying it on a hadoop-2.5.0 cluster, you should disable tez acls as YARN timeline in 2.5.0 does not support ACLs. Please set tez.am.acls.enabled to false as the Timeline layer is trying to enforce acls for the history data. thanks — Hitesh On Jun 3, 2015, at 10:19 PM, r7raul1...@163.com wrote: hive set hive.execution.engine=tez; hive set hive.tez.exec.print.summary=true; hive select userid,count(*) from u_data group by userid order by userid; Query ID = hdfs_20150604131717_6863a8c8-c3a6-4865-b098-06aff509dbf8 Total jobs = 1 Launching Job 1 out of 1 java.lang.NoClassDefFoundError: org/apache/hadoop/yarn/api/records/timeline/TimelineDomain at org.apache.tez.dag.history.ats.acls.ATSHistoryACLPolicyManager.createTimelineDomain(ATSHistoryACLPolicyManager.java:120) at org.apache.tez.dag.history.ats.acls.ATSHistoryACLPolicyManager.createSessionDomain(ATSHistoryACLPolicyManager.java:167) at org.apache.tez.dag.history.ats.acls.ATSHistoryACLPolicyManager.setupSessionACLs(ATSHistoryACLPolicyManager.java:228) at org.apache.tez.client.TezClientUtils.createApplicationSubmissionContext(TezClientUtils.java:525) at org.apache.tez.client.TezClient.start(TezClient.java:352) at org.apache.hadoop.hive.ql.exec.tez.TezSessionState.open(TezSessionState.java:191) at org.apache.hadoop.hive.ql.exec.tez.TezTask.updateSession(TezTask.java:234) at org.apache.hadoop.hive.ql.exec.tez.TezTask.execute(TezTask.java:136) at org.apache.hadoop.hive.ql.exec.Task.executeTask(Task.java:160) at org.apache.hadoop.hive.ql.exec.TaskRunner.runSequential(TaskRunner.java:88) at org.apache.hadoop.hive.ql.Driver.launchTask(Driver.java:1638) at org.apache.hadoop.hive.ql.Driver.execute(Driver.java:1397) at org.apache.hadoop.hive.ql.Driver.runInternal(Driver.java:1183) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1049) at org.apache.hadoop.hive.ql.Driver.run(Driver.java:1039) at org.apache.hadoop.hive.cli.CliDriver.processLocalCmd(CliDriver.java:207) at org.apache.hadoop.hive.cli.CliDriver.processCmd(CliDriver.java:159) at org.apache.hadoop.hive.cli.CliDriver.processLine(CliDriver.java:370) at org.apache.hadoop.hive.cli.CliDriver.executeDriver(CliDriver.java:754) at org.apache.hadoop.hive.cli.CliDriver.run(CliDriver.java:675) at org.apache.hadoop.hive.cli.CliDriver.main(CliDriver.java:615) 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.util.RunJar.main(RunJar.java:212) Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.yarn.api.records.timeline.TimelineDomain at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) ... 26 more FAILED: Execution Error, return code -101 from org.apache.hadoop.hive.ql.exec.tez.TezTask. org/apache/hadoop/yarn/api/records/timeline/TimelineDomain hive r7raul1...@163.com
[ANNOUNCE] New Apache Tez PMC Member - Jeff Zhang
Hi all, I am very pleased to announce that Jeff Zhang has been voted in as a member of the Apache Tez PMC. We appreciate all the work Jeff has put into the project so far, and are looking forward to his future (greater) contributions. Please join me in welcoming Jeff to the Tez PMC. Congratulations Jeff. thanks, — Hitesh on behalf of the Apache Tez PMC
Re: Tez lauche container error when use UseG1GC
To clarify, given that the error is showing up with container_1432885077153_0004_01_05, that means that the AM launched properly. Use “bin/yarn logs -applicationId application_1432885077153_0004 to get the logs. See if there are any errors for the logs for container_1432885077153_0004_01_05. If there are none, you will need to search for Assigning container to task” for the above container in the AM’s logs. Using this log line, you will see what host the container belongs to and you should then look at the NodeManager logs and search for the container id. The above would be a lot simpler if you have the UI setup to work against 0.5.3 but may still require you to dig through the NodeManager logs. thanks — Hitesh On May 29, 2015, at 3:48 AM, Jianfeng (Jeff) Zhang jzh...@hortonworks.com wrote: Could you check the yarn app logs to see what the error is ? If there’s still no useful info, you may refer the yarn RM/NN logs Best Regard, Jeff Zhang From: r7raul1...@163.com r7raul1...@163.com Reply-To: user user@tez.apache.org Date: Friday, May 29, 2015 at 4:16 PM To: user user@tez.apache.org Subject: Re: Tez lauche container error when use UseG1GC BTW my tez_site.xml content is: configuration property nametez.lib.uris/name valuehdfs:///apps/tez-0.5.3/tez-0.5.3.tar.gz/value /property property nametez.task.generate.counters.per.io/name valuetrue/value /property property descriptionLog history using the Timeline Server/description nametez.history.logging.service.class/name valueorg.apache.tez.dag.history.logging.ats.ATSHistoryLoggingService/value /property property descriptionPublish configuration information to Timeline server /description nametez.runtime.convert.user-payload.to.history-text/name valuetrue/value /property property nametez.am.launch.cmd-opts/name value-XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseG1GC -XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp//value /property /configuration r7raul1...@163.com From: r7raul1...@163.com Date: 2015-05-29 16:15 To: user Subject: Tez lauche container error when use UseG1GC I change my mapreduce.map.java.opts 's value from -Djava.net.preferIPv4Stack=true -Xmx825955249 to -Djava.net.preferIPv4Stack=true -XX:+UseG1GC -Xmx825955249 When I run query by hive 1.1.0+tez0.53 in hadoop 2.5.0. set mapreduce.framework.name=yarn-tez; set hive.execution.engine=tez; select userid,count(*) from u_data group by userid order by userid; The query return error. I found error : 2015-05-29 16:02:39,064 WARN [AsyncDispatcher event handler] container.AMContainerImpl: Container container_1432885077153_0004_01_05 finished with diagnostics set to [Container failed. Exception from container-launch. Container id: container_1432885077153_0004_01_05 Exit code: 1 Stack trace: ExitCodeException exitCode=1: at org.apache.hadoop.util.Shell.runCommand(Shell.java:538) 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.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:196) at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:299) at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:81) 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) But I try hive set hive.execution.engine=mr; hive set mapreduce.framework.name=yarn; hive select userid,count(*) from u_data group by userid order by userid limit 1; Query ID = hdfs_20150529160606_d550bca4-0341-4eb0-aace-a9018bfbb7a9 Total jobs = 2 Launching Job 1 out of 2 Number of reduce tasks not specified. Estimated from input data size: 1 In order to change the average load for a reducer (in bytes): set hive.exec.reducers.bytes.per.reducer=number In order to limit the maximum number of reducers: set hive.exec.reducers.max=number In order to set a constant number of reducers: set mapreduce.job.reduces=number Starting Job = job_1432885077153_0005, Tracking URL = http://localhost:8088/proxy/application_1432885077153_0005/ Kill Command = /opt/cloudera/parcels/CDH-5.2.0-1.cdh5.2.0.p0.36/lib/hadoop/bin/hadoop job -kill job_1432885077153_0005 Hadoop job information for Stage-1: number of mappers: 1; number of reducers: 1 2015-05-29 16:06:34,863 Stage-1 map = 0%, reduce = 0% 2015-05-29 16:06:40,066 Stage-1 map = 100%, reduce = 0%, Cumulative CPU 1.72 sec 2015-05-29 16:06:48,366 Stage-1 map = 100%, reduce = 100%,
Re: How to find query log when I use hiveserver2 on tez?
These are hive-specific questions. Could you please send this mail to the user@hive list instead? thanks — Hitesh On May 26, 2015, at 10:28 PM, r7raul1...@163.com wrote: How to find query log when I use hiveserver2 on tez? My enviroment is hive 1.1.0+tez0.53 use beeline to connect hiveserver2. r7raul1...@163.com
Re: EOFException - TezJob - Cannot submit DAG
Hello Patcharee Could you start with sending a mail to users@pig to see if they have come across this issue first? Also, can you check the application master logs to see if there are any errors ( might be useful to enable DEBUG level logging to get more information )? thanks — Hitesh On May 22, 2015, at 5:50 AM, patcharee patcharee.thong...@uni.no wrote: Hi, I ran a pig script on tez and got the EOFException. Check at http://wiki.apache.org/hadoop/EOFException I have no ideas at all how I can fix it. However I did not get the exception when I executed this pig script on MR. I am using HadoopVersion: 2.6.0.2.2.4.2-2, PigVersion: 0.14.0.2.2.4.2-2, TezVersion: 0.5.2.2.2.4.2-2 I will appreciate any suggestions. Thanks. 2015-05-22 14:44:13,638 [PigTezLauncher-0] ERROR org.apache.pig.backend.hadoop.executionengine.tez.TezJob - Cannot submit DAG - Application id: application_143223768_0133 org.apache.tez.dag.api.TezException: com.google.protobuf.ServiceException: java.io.EOFException: End of File Exception between local host is: compute-10-0.local/10.10.255.241; destination host is: compute-10-3.local:47111; : java.io.EOFException; For more details see: http://wiki.apache.org/hadoop/EOFException at org.apache.tez.client.TezClient.submitDAGSession(TezClient.java:415) at org.apache.tez.client.TezClient.submitDAG(TezClient.java:351) at org.apache.pig.backend.hadoop.executionengine.tez.TezJob.run(TezJob.java:162) at org.apache.pig.backend.hadoop.executionengine.tez.TezLauncher$1.run(TezLauncher.java:167) at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) 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:744) Caused by: com.google.protobuf.ServiceException: java.io.EOFException: End of File Exception between local host is: compute-10-0.local/10.10.255.241; destination host is: compute-10-3.local:47111; : java.io.EOFException; For more details see: http://wiki.apache.org/hadoop/EOFException at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:246) at com.sun.proxy.$Proxy31.submitDAG(Unknown Source) at org.apache.tez.client.TezClient.submitDAGSession(TezClient.java:408) ... 8 more Caused by: java.io.EOFException: End of File Exception between local host is: compute-10-0.local/10.10.255.241; destination host is: compute-10-3.local:47111; : java.io.EOFException; For more details see: http://wiki.apache.org/hadoop/EOFException at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:57) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:526) at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:791) at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:764) at org.apache.hadoop.ipc.Client.call(Client.java:1473) at org.apache.hadoop.ipc.Client.call(Client.java:1400) at org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:232) ... 10 more Caused by: java.io.EOFException at java.io.DataInputStream.readInt(DataInputStream.java:392) at org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1072) at org.apache.hadoop.ipc.Client$Connection.run(Client.java:967)
Re: Tez log location?
There is some history logging done that can be enabled via the SimpleHistoryLogger. This activates by default if ATS logger is not enabled. This is not fully compatible with the ATS data and also as it is mostly experimental, it may not have all the data. To use it, you can configure the “tez.history.logging.service.class” to “” or “org.apache.tez.dag.history.logging.impl.SimpleHistoryLoggingService”. The config property “tez.simple.history.logging.dir” controls the path on HDFS where the history is written to. If the dir path is not configured, it writes the logs as part of the Application Master container logs which can then be pulled via “bin/yarn logs -application” Using the HDFS logger does imply that the UI will no longer be functional. thanks — Hitesh On May 21, 2015, at 4:14 PM, Xiaoyong Zhu xiaoy...@microsoft.com wrote: Hi, I am wondering if I didn’t configure YARN ATS integration, where would be the tez log (I mean the data available in /ws/v1/timeline/TEZ_DAG_ID if there is an integration happening) go to HDFS? Is there a configuration for that? Thanks! Xiaoyong
Re: [DISCUSS] Drop Java 6 support in 0.8
Excellent point @Prakash. I would probably change my vote to a -0 on that point itself. It might be easier to say drop java6 support but harder to say we drop support for all older hadoop versions that need/support java 6. — Hitesh On May 15, 2015, at 10:29 PM, Prakash Ramachandran pramachand...@hortonworks.com wrote: +1 non-binding on dropping 1.6. However given that we compile against hadoop2.4 and hadoop2.2 which has requireJavaVersion 1.6 will it be an issue? On 5/16/15, 3:54 AM, Hitesh Shah hit...@apache.org wrote: +1 on dropping 1.6 support from 0.8.0 onwards. @Mohammad, Hadoop is dropping support for 1.6 from 2.7.0 onwards. I am guessing other ecosystem projects will soon follow suit. — Hitesh On May 15, 2015, at 2:16 PM, Mohammad Islam misla...@yahoo.com wrote: Hi Sid, What are the statuses of other Hadoop projects? Overall, I'm +1 on this. Regards, Mohammad On Friday, May 15, 2015 10:57 AM, Siddharth Seth ss...@apache.org wrote: Java 6 support ended quite a while ago. Trying to support it gets in the way of using libraries which may work with Java 7 only (Netty for example in TEZ-2450). I think we should move Tez 0.8 to work with Java 7 as the minimum version. Thoughts ? Thanks - Sid
Re: is there a way to map Tez UI graphs back to the script?
This will require a change in Hive/Pig and a modification to the Tez UI. For every vertex, there is a ProcessorDescriptor. This supports a setHistoryText API. /** * Provide a human-readable version of the user payload that can be * used in the TEZ UI * @param historyText History text * For better support in the UI, the history text should be a json-encoded string. * The following keys in the json object will be recognized: *desc : A string-value describing the entity *config : A key-value map to represent configuration * @return this object for further chained method calls */ public T setHistoryText(String historyText) The “desc” from above can be used to store the list of operators, query fragment being processed in the particular vertex. To start with, you can file a jira for Hive to add this information as well file a jira for this to be displayed in the Tez UI. Also, if you are looking to learn about either Hive or Tez, these jiras might be a good starting point for your first patch contribution. If you are looking to do this manually, you can run an explain on your query which should set some light on what processing is being done in which vertex. For more details on this, you should ask on the hive user list. thanks — Hitesh On May 15, 2015, at 4:41 AM, Jianfeng (Jeff) Zhang jzh...@hortonworks.com wrote: I believe there’s one jira that try to add context info on the vertex, so that you can see what the operator that vertex is mapped to. Best Regard, Jeff Zhang From: Xiaoyong Zhu xiaoy...@microsoft.com Reply-To: user@tez.apache.org user@tez.apache.org Date: Friday, May 15, 2015 at 5:50 PM To: user@tez.apache.org user@tez.apache.org Subject: is there a way to map Tez UI graphs back to the script? Not sure if there is a way to map the TezUI graph view back to the script? For example, below should be some JOIN operator – is there a way to map it back to the script to see which is the corresponding part? (attached is the script) image001.png Thanks! Xiaoyong image001.png
Re: container fails to start with malloc error
Hi Johannes Not sure if anyone has seen this earlier. Do you know if the machines have enough memory to run the no. of tasks/containers that you are launching? Also, I am assuming that you are compiling and running against the same jdk version? Would you mind sharing the details on what java version are you running? — Hitesh On Apr 14, 2015, at 1:19 AM, Johannes Zillmann jzillm...@googlemail.com wrote: Hey guys, in an customer environment certain Tez jobs fail to start On the client side it looks like: —— INFO [2015-04-08 15:19:30.213] [MrPlanRunnerV2] (YarnClientImpl.java:204) - Submitted application application_1428177121154_0065 INFO [2015-04-08 15:19:30.214] [MrPlanRunnerV2] (TezClient.java:357) - The url to track the Tez Session: http://master:8088/proxy/application_1428177121154_0065/ INFO [2015-04-08 15:19:33.219] [MrPlanRunnerV2] (TezClient.java:556) - App did not succeed. Diagnostics: Application application_1428177121154_0065 failed 2 times due to AM Container for appattempt_1428177121154_0065_02 exited with exitCode: 134 due to: Exception from container-launch: org.apache.hadoop.util.Shell$ExitCodeException: /bin/bash: line 1: 10818 Aborted (core dumped) /opt/teradata/jvm64/jdk7/bin/java -Xmx819m -server -Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN -XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC -Dapple.awt.UIElement=true -Djava.awt.headless=true -Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator -Dlog4j.configuration=tez-container-log4j.properties -Dyarn.app.container.log.dir=/data3/hadoop/yarn/log/application_1428177121154_0065/container_1428177121154_0065_02_01 -Dtez.root.logger=INFO,CLA -Dsun.nio.ch.bugLevel='' org.apache.tez.dag.app.DAGAppMaster --session /data3/hadoop/yarn/log/application_1428177121154_0065/container_1428177121154_0065_02_01/stdout 2 /data3/hadoop/yarn/log/application_1428177121154_0065/container_1428177121154_0065_02_01/stderr org.apache.hadoop.util.Shell$ExitCodeException: /bin/bash: line 1: 10818 Aborted (core dumped) /opt/teradata/jvm64/jdk7/bin/java -Xmx819m -server -Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN -XX:+PrintGCDetails -verbose:gc -XX:+PrintGCTimeStamps -XX:+UseNUMA -XX:+UseParallelGC -Dapple.awt.UIElement=true -Djava.awt.headless=true -Dlog4j.configuratorClass=org.apache.tez.common.TezLog4jConfigurator -Dlog4j.configuration=tez-container-log4j.properties -Dyarn.app.container.log.dir=/data3/hadoop/yarn/log/application_1428177121154_0065/container_1428177121154_0065_02_01 -Dtez.root.logger=INFO,CLA -Dsun.nio.ch.bugLevel='' org.apache.tez.dag.app.DAGAppMaster --session /data3/hadoop/yarn/log/application_1428177121154_0065/container_1428177121154_0065_02_01/stdout 2 /data3/hadoop/yarn/log/application_1428177121154_0065/container_1428177121154_0065_02_01/stderr at org.apache.hadoop.util.Shell.runCommand(Shell.java:505) at org.apache.hadoop.util.Shell.run(Shell.java:418) at org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:650) at org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:195) at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:300) at org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:81) 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) Container exited with a non-zero exit code 134 .Failing this attempt.. Failing the application. —— Then you have that for the task: —— Log Type: stderr Log Length: 429 java: malloc.c:3090: sYSMALLOc: Assertion `(old_top == (((mbinptr) (((char *) ((av)-bins[((1) - 1) * 2])) - __builtin_offsetof (struct malloc_chunk, fd old_size == 0) || ((unsigned long) (old_size) = (unsigned long)__builtin_offsetof (struct malloc_chunk, fd_nextsize))+((2 * (sizeof(size_t))) - 1)) ~((2 * (sizeof(size_t))) - 1))) ((old_top)-size 0x1) ((unsigned long)old_end pagemask) == 0)' failed. Log Type: stdout Log Length: 0 —— Any ideas ? Johannes
Re: Build tez UI failed.
Hi I just downloaded the src-tar for the 0.6.0 release and built it without any issues. At times, I have not seen this but others have faced issues at times ( based on general searches[1] ) where there have been temporal issues downloading node locally at times. [1] https://github.com/eirslett/frontend-maven-plugin/issues/99 Not sure if the above issue is what you have faced. However, you can run the build with debug enabled and can help pinpoint the error, we can try and make the build more robust if possible. thanks — Hitesh On Mar 24, 2015, at 1:47 AM, Azuryy Yu azury...@gmail.com wrote: Hi, I try to build tez-0.6.0, but cannot build Tez UI. [ERROR] Failed to execute goal com.github.eirslett:frontend-maven-plugin:0.0.16:install-node-and-npm (install node and npm) on project tez-ui: Execution install node and npm of goal com.github.eirslett:frontend-maven-plugin:0.0.16:install-node-and-npm failed: A required class was missing while executing com.github.eirslett:frontend-maven-plugin:0.0.16:install-node-and-npm: org/slf4j/helpers/MarkerIgnoringBase what casued this failure? Thanks.
Re: protobuf version clarification
Hi Michael, Yes - that is indeed an error regarding the version of protobuf that needs to be used. Pretty much all of the Hadoop ecosystem components have standardized on protobuf-2.5.0. Newer versions would likely have worked assuming protobuf retained compatibility. In any case, you do need to switch back to protobuf-2.5.0. Modifying the pom would have likely fixed the build issue with Tez but you would have ended up facing errors when trying on a cluster as hadoop needs 2.5.0. Could you file a jira and submit a patch for this issue ( and if you came across any other confusing/unclear items in the documentation )? thanks — Hitesh On Mar 22, 2015, at 8:34 AM, Michael Howard mhow...@podiumdata.com wrote: Installation doc should clarify which version of protobuf is required. Tez installation doc at http://tez.apache.org/install.html says: This assumes that you have already installed ... Protocol Buffers (protoc compiler) 2.5 or later I initially installed protobuf 2.6.1 ... and mvn failed with: [ERROR] Failed to execute goal ... protoc version is 'libprotoc 2.6.1', expected version is '2.5.0' - [Help 1] I observe that pom.xml says: protobuf.version2.5.0/protobuf.version At this point, it wasn't clear to me whether I should roll back to protobuf 2.5.0 or override the protobuf.version to be 2.6.1 For newbies like me, it would be helpful to have more clear instructions regarding protobuf version on the installation page. Thanks! Michael
Re: error after installation - TezSession has already shutdown
You have yarn configured with log aggregation disabled ( yarn.log-aggregation-enable to false ). In this scenario, the logs are on the NodeManager where the AM was launched. For this, find the log-dirs property configured in yarn-site.xml and use that to find the logs for your application. If this is a multi-node cluster, you will need to find logs from all nodes in your cluster. YARN logs aggregation if enabled does this automatically and stores all logs in HDFS. In any case, based on your earlier logs: 2015-03-16 23:39:01,576 [PigTezLauncher-0] INFO org.apache.tez.client.TezClient - Tez Client Version: [ component=tez-api, version=0.5.2, revision=5d728786b681e549a6f33375991dcce97cbf79f3, SCM-URL=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git, buildTIme=20141104-1809 ] and org.apache.tez.client.TezClientUtils - Using tez.lib.uris value from configuration: hdfs://localhost:9000/apps/tez-0.6.0-SNAPSHOT/tez-0.6.0.tar.gz I think the inconsistency of versions might be the reason why things are failing. Please check the ResourceManager UI for the application in question and see if the diagnostics message points to a version issue. thanks — Hitesh On Mar 17, 2015, at 3:51 PM, dagriq dag...@yahoo.ca wrote: hi and thanks for the quick reply. I have tried the hint that you gave me but it was nothing written. Please open the attached screenshot; I issed the command after error was raised... regards On Tue, 3/17/15, Hitesh Shah hit...@apache.org wrote: Subject: Re: error after installation - TezSession has already shutdown To: user@tez.apache.org Received: Tuesday, March 17, 2015, 11:30 AM Hello First issue from the stack trace: org.apache.tez.dag.api.TezUncheckedException: Invalid configuration of tez jars, tez.lib.uris is not defined in the configuration”. It looks like your first run failed due to this. The second run seemed to be configured correctly but failed for a different reason. For this, look for a line along the lines of org.apache.hadoop.yarn.client.api.impl.YarnClientImpl - Submitted application application_1426563447689_0001” to find the application ID. Now, invoke “bin/yarn logs -applicationId appId” to get all the logs and attach it to your email to this list so that we can take a look to see if there are other issues. Actually, my guess is that you have 2 different versions of tez in use. Based on the stack trace, you seem to be using tez-0.5.2 on the client machine but your tarball on HDFS is tez-0.6.0. Both of these need to have matching versions. If you go to the ResourceManager and look at the UI for the applicationId, its diagnostics might likely tell you that the AM was incompatible with the client. If you see a different error, please send across the logs from the above command and we can help further. thanks — HItesh On Mar 17, 2015, at 6:27 AM, dagriq dag...@yahoo.ca wrote: hi I have downloaded and compiled Tez and I received one error when I tried at the bash shell: [ pig -x tez myscript.pig ]. I executed all steps from the [ http://tez.apache.org/install.html ] and I have a new folder in /usr/local/apache-tez. I added one more subfolder called conf where I stored the new file tez-site.xml. My first error message is copied below: 2015-03-16 23:39:00,463 [main] INFO org.apache.pig.backend.hadoop.executionengine.tez.TezJobCompiler - For vertex - scope-34: parallelism=1, memory=1024, java opts=-Djava.net.preferIPv4Stack=true -Dhadoop.metrics.log.level=WARN -Xmx200m -Dlog4j.configuration=tez-container-log4j.properties -Dyarn.app.container.log.dir=LOG_DIR -Dtez.root.logger=INFO,CLA 2015-03-16 23:39:00,862 [PigTezLauncher-0] INFO org.apache.pig.tools.pigstats.tez.TezScriptState - Pig script settings are added to the job 2015-03-16 23:39:01,576 [PigTezLauncher-0] INFO org.apache.tez.client.TezClient - Tez Client Version: [ component=tez-api, version=0.5.2, revision=5d728786b681e549a6f33375991dcce97cbf79f3, SCM-URL=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git, buildTIme=20141104-1809 ] 2015-03-16 23:39:01,757 [PigTezLauncher-0] INFO org.apache.hadoop.yarn.client.RMProxy - Connecting to ResourceManager at /0.0.0.0:8032 2015-03-16 23:39:02,798 [PigTezLauncher-0] INFO org.apache.tez.client.TezClient - Session mode. Starting session. 2015-03-16 23:39:02,800 [PigTezLauncher-0] ERROR org.apache.pig.backend.hadoop.executionengine.tez.TezJob - Cannot submit DAG org.apache.tez.dag.api.TezUncheckedException: Invalid configuration of tez jars, tez.lib.uris is not defined in the configuration at org.apache.tez.client.TezClientUtils.setupTezJarsLocalResources(TezClientUtils.java:162) at org.apache.tez.client.TezClient.getTezJarResources(TezClient.java:711) at org.apache.tez.client.TezClient.start(TezClient.java:294
Re: streamed splitting
Hello Johannes, This is something we have discussed quite often but have not got around to implementing this. There might be an open jira related to “pipelining” of splits. If you cannot find it, please go ahead and create one. The general issues with these are: - how to handle dynamic creation of tasks as splits get created - how to decide how many splits and which splits a single task should handle - involves some facet of grouping to do optimal allocations of newly created splits based on available containers. Size of groups could be different e.g a single group slit consist of either 5 data local splits or 2 rack-local splits or 1 off-rack split when assigning dynamically to a given container. - the single task limit also plays into how you handle fault tolerance and recovery - given that split creation is now dynamic, if the AM crashes in a scenario when not all splits were created but some were already processed, the next attempt when it recovers needs to handle it in a such way to ensure correctness of data processing. thanks — Hitesh On Mar 12, 2015, at 2:38 AM, Johannes Zillmann jzillm...@googlemail.com wrote: Hey guys, dump question. With Tez can i have a input-initializaer which don’t require to create every split before starting the processing of already created splits ? Means if i have a lot of splits and my splitting process takes a long time, can the workers start working already while still doing the splitting ? Johannes
Re: Parallel queries/dags running in same AM?
A clarification for (2), you can share an AM across multiple users by using form of proxy users and passing in the required delegation tokens to talk to various services such as HDFS. Also, HiveServer2 when the doAs mode is set to false, runs all AMs as user hive but can effectively run queries for various different users by doing its security check at the “perimeter”. — Hitesh On Mar 9, 2015, at 10:30 AM, Bikas Saha bi...@hortonworks.com wrote: (1)- For every TEZ AM it is possible to launch just a single query/DAG at a time. So within a given AM several DAGs can be executed only in sequential order (a.k.a. a session), not in parallel. To execute DAGs in parallel we always need several AMs. Correct. Today a single AM will accept new DAGs when the AM is idle and run them. An AM is idle when no DAG is running. (2)- The AM is user-specific, and each user is expected to run queries through its own AM (or on multiple AMs if there is a need for parallelism). Correct in a secure cluster. In a non-secure cluster an AM runs as the yarn user which is common to all AMs. In a secure cluster, any entity that has been given a client token (for that app attempt) by the RM, can communicate with the AM. In a non-secure cluster, any entity that has obtained the AMs connection information from the RM can communicate with the AM. The AM has an additional set of ACL’s that determine who can submit, view, modify DAGs. (3)- Several users can submit their DAGs as the same user (e.g.: through hiveserver2), but in this case we will still have several AM. Correct. However, the number of AMs will be determined by the policy of the mediating server. It may choose to launch a new AM for every new DAG. Or queue up and round robin through a limited set of AMs, etc. Bikas From: Fabio C. [mailto:anyte...@gmail.com] Sent: Monday, March 09, 2015 4:31 AM To: user@tez.apache.org; u...@hive.apache.org Subject: Parallel queries/dags running in same AM? Hi all, I've been using Tez on hive, and I had a chance to hear a conversation that mismatches with my present knowledge, can anyone confirm the following statement? (1)- For every TEZ AM it is possible to launch just a single query/DAG at a time. So within a given AM several DAGs can be executed only in sequential order (a.k.a. a session), not in parallel. To execute DAGs in parallel we always need several AMs. (2)- The AM is user-specific, and each user is expected to run queries through its own AM (or on multiple AMs if there is a need for parallelism). (3)- Several users can submit their DAGs as the same user (e.g.: through hiveserver2), but in this case we will still have several AM. Thanks in advance Fabio