hive-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Gopal V <gop...@apache.org>
Subject Re: Getting Tez working against cdh 5.3
Date Fri, 23 Jan 2015 18:09:29 GMT
On 1/20/15, 4:11 PM, Edward Capriolo wrote:
> I see. That helped a lot.
>
> java.lang.UnsatisfiedLinkError:
> org.apache.hadoop.util.NativeCodeLoader.buildSupportsSnappy()Z
>      at org.apache.hadoop.util.NativeCodeLoader.buildSupportsSnappy(Native
...
> I am not sure what this fix is doing exactly. Does this mean the JVM
> getting called is not being sent the path to the hadoop-native? Am I better
> off just compiling tez against the cdh jars like you did?

You need to compile it against CDH jars anyway.

That particular case is the LD_LIBRARY_PATH missing for 
tez.task.launch.env variable.

With rolling upgrades, Tez picks the specific libhadoop.so on each node 
instead of picking a systems default (like /usr/lib/hadoop/native/).

Since you are seeing this error, I'm assuming you're running a very old 
incubating build instead of the Apache stable - this should just work on 
0.5.0.

Not sure whether CDH puts the native libs somewhere standard, but you 
can try cut-pasting the 0.5.0 defaults into the 0.4.1 and make it work.

> No I am not trying for a private install. I want the default hive-server to
> be able to submit tez jobs.
>
> My goal is to have a quick recipe for getting tez to work with cdh 5.3 with
> minimal hacking of the install.

Yeah, the private build scripts I mentioned get you there with the 
exception of the metastore.

It should build something where you can just run "./dist/hive/bin/hive 
--service hiveserver2" and get the JDBC connectivity up easily without 
being an administrator or overwriting any of the system installed JARs.

HTH.

Cheers,
Gopal

> On Tue, Jan 20, 2015 at 6:39 PM, Gopal V <gopalv@apache.org> wrote:
>
>> On 1/20/15, 12:34 PM, Edward Capriolo wrote:
>>
>>> Actually more likely something like this:
>>>
>>> https://issues.apache.org/jira/browse/TEZ-1621
>>>
>>
>> I have a working Hive-13 + Tez install on CDH-5.2.0-1.cdh5.2.0.p0.36.
>>
>> Most of the work needed to get that to work was to build all of Hive+Tez
>> against the CDH jars instead of the Apache 2.4.0.
>>
>> You need to do
>>
>> "yarn logs -applicationId <app-id>"
>>
>> which should give you the stderr as well, because like the JIRA
>> referenced, this looks like an ABI compatibility issue.
>>
>> Tez hasn't got any diagnostic for those cases since it never saw the
>> container come up and send a heartbeat.
>>
>> If you're trying to install this for a private install, I keep updating
>> this github which installs all of this under your user instead of as "hive".
>>
>> https://github.com/t3rmin4t0r/tez-autobuild
>>
>> Cheers,
>> Gopal
>>
>>  On Tue, Jan 20, 2015 at 2:02 PM, Prasanth Jayachandran <
>>> pjayachandran@hortonworks.com> wrote:
>>>
>>>  My guess is..
>>>>  "java" binary is not in PATH of the shell script that launches the
>>>> container.. try creating a symbolic link in /bin/ to point to java..
>>>>
>>>> On Tue, Jan 20, 2015 at 7:22 AM, Edward Capriolo <edlinuxguru@gmail.com>
>>>> wrote:
>>>>
>>>>  It seems that CDH does not ship with enough jars to run tez out of the
>>>>> box.
>>>>>
>>>>> I have found the related cloudera forked pom.
>>>>>
>>>>> In this pom hive is built against tez 0.4.1-incubating-tez2.0-SHAPSHOT
>>>>>
>>>>> Thus I followed the instructions here:
>>>>>
>>>>> http://tez.apache.org/install_pre_0_5_0.html
>>>>>
>>>>> hive> dfs -lsr /apps
>>>>>     > ;
>>>>> lsr: DEPRECATED: Please use 'ls -R' instead.
>>>>> drwxr-xr-x   - ecapriolo supergroup          0 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating
>>>>> drwxr-xr-x   - ecapriolo supergroup          0 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib
>>>>> -rw-r--r--   3 ecapriolo supergroup     303139 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/avro-1.7.4.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup      41123 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/commons-cli-1.2.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     610259 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/commons-collections4-4.0.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup    1648200 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/guava-11.0.2.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     710492 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/guice-3.0.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     656365 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/hadoop-mapreduce-client-common-2.2.0.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup    1455001 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/hadoop-mapreduce-client-core-2.2.0.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup      21537 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/hadoop-mapreduce-client-
>>>>> shuffle-2.2.0.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup      81743 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/jettison-1.3.4.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     533455 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/protobuf-java-2.5.0.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     995968 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/lib/snappy-java-1.0.4.1.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     752332 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-api-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup      34089 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-common-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     980132 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-dag-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     246395 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-mapreduce-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     199984 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-mapreduce-examples-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     114676 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-runtime-internals-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup     352835 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-runtime-library-0.4.1-incubating.jar
>>>>> -rw-r--r--   3 ecapriolo supergroup       6832 2015-01-16 23:00
>>>>> /apps/tez-0.4.1-incubating/tez-tests-0.4.1-incubating.jar
>>>>>
>>>>> This is my tez-site.xml
>>>>>
>>>>> <configuration>
>>>>>   <property>
>>>>>     <name>tez.lib.uris</name>
>>>>>     <value>${fs.default.name}/apps/tez-0.4.1-incubating,${fs
>>>>> .default.name
>>>>> }/apps/tez-0.4.1-incubating/lib/</value>
>>>>>   </property>
>>>>> </configuration>
>>>>>
>>>>> [ecapriolo@production-hadoop-cdh-69-7 ~]$ ls -lahR
>>>>> /home/ecapriolo/tez-0.4.1-incubating/
>>>>> /home/ecapriolo/tez-0.4.1-incubating/:
>>>>> total 2.7M
>>>>> drwxrwxr-x 3 ecapriolo ecapriolo 4.0K Jan 16 22:54 .
>>>>> drwx------ 7 ecapriolo ecapriolo  20K Jan 20 15:20 ..
>>>>> drwxrwxr-x 2 ecapriolo ecapriolo 4.0K Jan 16 22:54 lib
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 735K Jan 16 22:54
>>>>> tez-api-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo  34K Jan 16 22:54
>>>>> tez-common-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 958K Jan 16 22:54
>>>>> tez-dag-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 241K Jan 16 22:54
>>>>> tez-mapreduce-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 196K Jan 16 22:54
>>>>> tez-mapreduce-examples-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 112K Jan 16 22:54
>>>>> tez-runtime-internals-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 345K Jan 16 22:54
>>>>> tez-runtime-library-0.4.1-incubating.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 6.7K Jan 16 22:54
>>>>> tez-tests-0.4.1-incubating.jar
>>>>>
>>>>> /home/ecapriolo/tez-0.4.1-incubating/lib:
>>>>> total 6.8M
>>>>> drwxrwxr-x 2 ecapriolo ecapriolo 4.0K Jan 16 22:54 .
>>>>> drwxrwxr-x 3 ecapriolo ecapriolo 4.0K Jan 16 22:54 ..
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 297K Jan 16 22:54 avro-1.7.4.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo  41K Jan 16 22:54 commons-cli-1.2.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 596K Jan 16 22:54
>>>>> commons-collections4-4.0.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 1.6M Jan 16 22:54 guava-11.0.2.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 694K Jan 16 22:54 guice-3.0.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 641K Jan 16 22:54
>>>>> hadoop-mapreduce-client-common-2.2.0.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 1.4M Jan 16 22:54
>>>>> hadoop-mapreduce-client-core-2.2.0.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo  22K Jan 16 22:54
>>>>> hadoop-mapreduce-client-shuffle-2.2.0.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo  80K Jan 16 22:54 jettison-1.3.4.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 521K Jan 16 22:54
>>>>> protobuf-java-2.5.0.jar
>>>>> -rw-rw-r-- 1 ecapriolo ecapriolo 973K Jan 16 22:54
>>>>> snappy-java-1.0.4.1.jar
>>>>>
>>>>> tez.sh
>>>>> TEZ_CONF_DIR=/home/ecapriolo
>>>>> TEZ_JARS=/home/ecapriolo/tez-0.4.1-incubating
>>>>> export HADOOP_CLASSPATH=${TEZ_CONF_DIR}:${TEZ_JARS}/*:${TEZ_JARS}/lib/*
>>>>> #hive -hiveconf mapreduce.framework.name=yarn-tez
>>>>> #hive -hiveconf hive.root.logger=DEGUG,console
>>>>> hive
>>>>>
>>>>> hive> set
>>>>> hive.execution.engine=tez;
>>>>> hive> select sum(viral_count) from author_article_hourly where dt=
>>>>> 2015011622;
>>>>> Total jobs = 1
>>>>> Launching Job 1 out of 1
>>>>>
>>>>>
>>>>> Status: Running (application id: application_1420748315294_70716)
>>>>>
>>>>> Map 1: -/-    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Map 1: 0/1    Reducer 2: 0/1
>>>>> Status: Failed
>>>>> Vertex failed, vertexName=Map 1,
>>>>> vertexId=vertex_1420748315294_70716_1_01, diagnostics=[Task failed,
>>>>> taskId=task_1420748315294_70716_1_01_000000,
>>>>> diagnostics=[AttemptID:attempt_1420748315294_70716_1_01_000000_0
>>>>> Info:Container container_1420748315294_70716_01_000002 COMPLETED with
>>>>> diagnostics set to [Exception from container-launch.
>>>>> Container id: container_1420748315294_70716_01_000002
>>>>> Exit code: 255
>>>>> Stack trace: ExitCodeException exitCode=255:
>>>>>     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:197)
>>>>>     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)
>>>>>
>>>>> Other logs are here (from the containers that I can extract from yawn.
>>>>>
>>>>> Exit code: 255
>>>>> Stack trace: ExitCodeException exitCode=255:
>>>>>     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:197)
>>>>>     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:744)
>>>>>
>>>>> application_1420748315294_70716
>>>>>
>>>>> digraph _1 {
>>>>> graph [ label="_1", fontsize=24, fontname=Helvetica];
>>>>> node [fontsize=12, fontname=Helvetica];
>>>>> edge [fontsize=9, fontcolor=blue, fontname=Arial];
>>>>> "_1.Map_1_author_article_hourly" [ label =
>>>>> "Map_1[author_article_hourly]", shape = "box" ];
>>>>> "_1.Map_1_author_article_hourly" -> "_1.Map_1" [ label = "Input
>>>>> [inputClass=MRInputLegacy,\n initializer=MRInputAMSplitGenerator]" ];
>>>>> "_1.Reducer_2" [ label = "Reducer_2[ReduceTezProcessor]" ];
>>>>> "_1.Reducer_2" -> "_1.Reducer_2_out_Reducer_2" [ label = "Output
>>>>> [outputClass=MROutput,\n initializer=]" ];
>>>>> "_1.Map_1" [ label = "Map_1[MapTezProcessor]" ];
>>>>> "_1.Map_1" -> "_1.Reducer_2" [ label = "[input=OnFileSortedOutput,\n
>>>>> output=ShuffledMergedInputLegacy,\n dataMovement=SCATTER_GATHER,\n
>>>>> schedulingType=SEQUENTIAL]" ];
>>>>> "_1.Reducer_2_out_Reducer_2" [ label = "Reducer_2[out_Reducer_2]", shape
>>>>> = "box" ];
>>>>> }
>>>>>
>>>>>
>>>>> Container exited with a non-zero exit code 255
>>>>> ]], Vertex failed as one or more tasks failed. failedTasks:1]
>>>>> Vertex killed, vertexName=Reducer 2,
>>>>> vertexId=vertex_1420748315294_70716_1_00, diagnostics=[Vertex received
>>>>> Kill
>>>>> while in RUNNING state., Vertex killed as other vertex failed.
>>>>> failedTasks:0]
>>>>> DAG failed due to vertex failure. failedVertices:1 killedVertices:1,
>>>>> counters=Counters: 2, org.apache.tez.common.counters.DAGCounter,
>>>>> NUM_FAILED_TASKS=4, TOTAL_LAUNCHED_TASKS=4
>>>>> 2015-01-20 15:02:21,934 INFO [AsyncDispatcher event handler]
>>>>> org.apache.tez.dag.app.dag.impl.DAGImpl: DAG: dag_1420748315294_70716_1
>>>>> finished with state: FAILED
>>>>> 2015-01-20 15:02:21,934 INFO [AsyncDispatcher event handler]
>>>>> org.apache.tez.dag.app.dag.impl.DAGImpl: dag_1420748315294_70716_1
>>>>> transitioned from TERMINATING to FAILED
>>>>> 2015-01-20 15:02:21,935 INFO [AsyncDispatcher event handler]
>>>>> org.apache.tez.dag.app.DAGAppMaster: DAG completed,
>>>>> dagId=dag_1420748315294_70716_1, dagState=FAILED
>>>>> 2015-01-20 15:02:21,935 INFO [AsyncDispatcher event handler]
>>>>> org.apache.tez.common.TezUtils: Redirecting log files based on addend:
>>>>> dag_1420748315294_70716_1_post
>>>>>
>>>>> Has anyone got this working or have and ideas as to what is up here?
>>>>>
>>>>> Thanks,
>>>>> Edward
>>>>>
>>>>>
>>>>>
>>>>>
>>>> CONFIDENTIALITY NOTICE
>>>> NOTICE: This message is intended for the use of the individual or entity
>>>> to which it is addressed and may contain information that is
>>>> confidential,
>>>> privileged and exempt from disclosure under applicable law. If the reader
>>>> of this message is not the intended recipient, you are hereby notified
>>>> that
>>>> any printing, copying, dissemination, distribution, disclosure or
>>>> forwarding of this communication is strictly prohibited. If you have
>>>> received this communication in error, please contact the sender
>>>> immediately
>>>> and delete it from your system. Thank You.
>>>>
>>>
>>>
>>>
>>
>
>


Mime
View raw message