You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@tez.apache.org by Jiří Šimša <ji...@gmail.com> on 2015/11/12 03:54:09 UTC

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

Posted by Jiří Šimša <ji...@gmail.com>.
Thank you Hitesh. I will give that a try.

Best,

--
Jiří Šimša

On Mon, Nov 16, 2015 at 11:36 AM, Hitesh Shah <hi...@apache.org> wrote:

> I am not sure whether the fat-jar is creating an issue. Can you try the
> following:
>
>    - convert the fat-jar ( jar with dependencies ) into a directory of
> separate jars i.e. a dir with all jars within it ( no nested dirs - just
> one top level dir with all jars in it )
>    - upload this dir to HDFS
>    - add this dir to tez.aux.uris ( and remove the fat-jar )
>
> Beyond this, if you are familiar with YARN, a few things you can do to
> capture more info which can help us help you:
>    - set "yarn.nodemanager.delete.debug-delay-sec” to say 600 or 900 in
> yarn-site.xml ( requires node manager restart ). This retains the container
> data after the application completes
>    - find a launch_container.sh file for any container ( of the
> application that is failing ) within local-dirs configured for YARN in your
> yarn-site.xml
>
> The launch_container.sh will tell you what files are being added to the
> container’s env and how the classpath is being setup.
>
> thanks
> — Hitesh
>
>
> On Nov 16, 2015, at 8:41 AM, Jiří Šimša <ji...@gmail.com> wrote:
>
> > Hi Bikas and Hitesh,
> >
> > Any suggestion as to how to get tachyon.hadoop.TFS recognized by Tez's
> cluster? Thanks.
> >
> > Best,
> >
> > --
> > Jiří Šimša
> >
> > On Thu, Nov 12, 2015 at 10:42 AM, Jiří Šimša <ji...@gmail.com>
> wrote:
> > Thank you Bikas and Hitesh for your responses.
> >
> > I believe the problem is in the cluster. Here is the relevant
> information:
> >
> > 1) My HADOOP_CLASSPATH:
> >
> > $ hadoop classpath
> >
> /usr/local/Cellar/hadoop/2.7.1/libexec/etc/hadoop:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/*:/Users/jsimsa/Projects/tez:/Users/jsimsa/Projects/tez/jars/*:/Users/jsimsa/Projects/tez/jars/lib/*:/contrib/capacity-scheduler/*.jar
> >
> > 2) The contents of /Users/jsimsa/Projects/tez/tez-site.xml:
> >
> > <?xml version="1.0"?>
> > <!--
> >   Licensed under the Apache License, Version 2.0 (the "License");
> >   you may not use this file except in compliance with the License.
> >   You may obtain a copy of the License at
> >
> >     http://www.apache.org/licenses/LICENSE-2.0
> >
> >   Unless required by applicable law or agreed to in writing, software
> >   distributed under the License is distributed on an "AS IS" BASIS,
> >   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
> implied.
> >   See the License for the specific language governing permissions and
> >   limitations under the License. See accompanying LICENSE file.
> > -->
> >
> > <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
> >
> > <!-- Put site-specific property overrides in this file. -->
> >
> > <configuration>
> >
> > <property>
> >   <name>tez.lib.uris</name>
> >
>  <value>${fs.defaultFS}/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz</value>
> > </property>
> > <property>
> >   <name>tez.aux.uris</name>
> >
>  <value>${fs.defaultFS}/apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar</value>
> > </property>
> >
> > </configuration>
> >
> > 3) The contents of the /apps HDFS folder:
> >
> > $ ./bin/hdfs dfs -lsr /apps
> > lsr: DEPRECATED: Please use 'ls -R' instead.
> > SLF4J: Class path contains multiple SLF4J bindings.
> > SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
> > 15/11/12 10:39:52 WARN util.NativeCodeLoader: Unable to load
> native-hadoop library for your platform... using builtin-java classes where
> applicable
> > drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:43
> /apps/tachyon-0.8.2-SNAPSHOT
> > -rw-r--r--   1 jsimsa supergroup   43809325 2015-11-11 18:43
> /apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
> > drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:44
> /apps/tez-0.8.2-SNAPSHOT
> > -rw-r--r--   1 jsimsa supergroup   43884378 2015-11-11 18:44
> /apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
> >
> >
> > 4) Finally, the command I am running and its output:
> >
> > $
> HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
> hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar
> orderedwordcount tachyon://localhost:19998/input.txt
> tachyon://localhost:19998/output.txt
> > SLF4J: Class path contains multiple SLF4J bindings.
> > SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > SLF4J: Found binding in
> [jar:file:/Users/jsimsa/Projects/tez/jars/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> > SLF4J: Found binding in
> [jar:file:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
> > 15/11/12 10:37:29 WARN util.NativeCodeLoader: Unable to load
> native-hadoop library for your platform... using builtin-java classes where
> applicable
> > 15/11/12 10:37:29 INFO client.TezClient: Tez Client Version: [
> component=tez-api, version=0.8.2-SNAPSHOT,
> revision=6562a9d882fc455f511dd9d93af1d159d3e3e71b, SCM-URL=scm:git:
> https://git-wip-us.apache.org/repos/asf/tez.git,
> buildTime=2015-11-11T19:44:28Z ]
> > 15/11/12 10:37:29 INFO client.RMProxy: Connecting to ResourceManager at /
> 0.0.0.0:8032
> > 15/11/12 10:37:30 INFO : initialize(tachyon://localhost:19998/input.txt,
> Configuration: core-default.xml, core-site.xml, mapred-default.xml,
> mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml,
> hdfs-site.xml, tez-site.xml). Connecting to Tachyon:
> tachyon://localhost:19998/input.txt
> > 15/11/12 10:37:30 INFO : Loading Tachyon properties from Hadoop
> configuration: {}
> > 15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is
> trying to connect with BlockMaster master @ localhost/127.0.0.1:19998
> > 15/11/12 10:37:30 INFO : Client registered with BlockMaster master @
> localhost/127.0.0.1:19998
> > 15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is
> trying to connect with FileSystemMaster master @ localhost/127.0.0.1:19998
> > 15/11/12 10:37:30 INFO : Client registered with FileSystemMaster master
> @ localhost/127.0.0.1:19998
> > 15/11/12 10:37:30 INFO : tachyon://localhost:19998
> tachyon://localhost:19998 hdfs://localhost:9000
> > 15/11/12 10:37:30 INFO : getWorkingDirectory: /
> > 15/11/12 10:37:30 INFO : getWorkingDirectory: /
> > 15/11/12 10:37:30 INFO examples.OrderedWordCount: Running
> OrderedWordCount
> > 15/11/12 10:37:30 INFO client.TezClient: Submitting DAG application with
> id: application_1447296197811_0003
> > 15/11/12 10:37:30 INFO client.TezClientUtils: Using tez.lib.uris value
> from configuration:
> hdfs://localhost:9000/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
> > 15/11/12 10:37:30 INFO client.TezClient: Tez system stage directory
> hdfs://localhost:9000/tmp/jsimsa/tez/staging/.tez/application_1447296197811_0003
> doesn't exist and is created
> > 15/11/12 10:37:30 INFO client.TezClient: Submitting DAG to YARN,
> applicationId=application_1447296197811_0003, dagName=OrderedWordCount,
> callerContext={ context=TezExamples, callerType=null, callerId=null }
> > 15/11/12 10:37:30 INFO impl.YarnClientImpl: Submitted application
> application_1447296197811_0003
> > 15/11/12 10:37:30 INFO client.TezClient: The url to track the Tez AM:
> http://Jiris-MacBook-Pro.local:8088/proxy/application_1447296197811_0003/
> > 15/11/12 10:37:34 INFO client.RMProxy: Connecting to ResourceManager at /
> 0.0.0.0:8032
> > 15/11/12 10:37:34 INFO client.DAGClientImpl: DAG: State: FAILED
> Progress: 0% TotalTasks: 1 Succeeded: 0 Running: 0 Failed: 0 Killed: 0
> > 15/11/12 10:37:34 INFO client.DAGClientImpl: DAG completed.
> FinalState=FAILED
> > 15/11/12 10:37:34 INFO examples.OrderedWordCount: DAG diagnostics:
> [Vertex failed, vertexName=Tokenizer,
> vertexId=vertex_1447296197811_0003_1_00, diagnostics=[Vertex
> vertex_1447296197811_0003_1_00 [Tokenizer] killed/failed due
> to:ROOT_INPUT_INIT_FAILURE, Vertex Input: Input initializer failed,
> vertex=vertex_1447296197811_0003_1_00 [Tokenizer],
> java.lang.RuntimeException: java.lang.ClassNotFoundException: Class
> tachyon.hadoop.TFS not found
> >       at
> org.apache.hadoop.conf.Configuration.getClass(Configuration.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.input.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:293)
> >       at
> org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:265)
> >       at
> org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits(FileInputFormat.java:387)
> >       at
> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat.getSplits(TezGroupedSplitsInputFormat.java:97)
> >       at
> org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateNewSplits(MRInputHelpers.java:413)
> >       at
> org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateInputSplitsToMem(MRInputHelpers.java:292)
> >       at
> org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.initialize(MRInputAMSplitGenerator.java:120)
> >       at
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:246)
> >       at
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:240)
> >       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.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:240)
> >       at
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:227)
> >       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.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)
> >       ... 25 more
> > ], Vertex failed, vertexName=Sorter,
> vertexId=vertex_1447296197811_0003_1_02, diagnostics=[Vertex init failed :
> org.apache.tez.dag.api.TezUncheckedException: java.lang.RuntimeException:
> java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not found
> >       at
> org.apache.tez.mapreduce.committer.MROutputCommitter.getOutputCommitter(MROutputCommitter.java:139)
> >       at
> org.apache.tez.mapreduce.committer.MROutputCommitter.initialize(MROutputCommitter.java:81)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2334)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2316)
> >       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.tez.dag.app.dag.impl.VertexImpl.initializeCommitters(VertexImpl.java:2316)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl.initializeVertex(VertexImpl.java:2350)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl.access$4700(VertexImpl.java:204)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.handleInitEvent(VertexImpl.java:3497)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3394)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3375)
> >       at
> org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
> >       at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
> >       at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
> >       at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
> >       at
> org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1975)
> >       at
> org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:203)
> >       at
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2096)
> >       at
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2082)
> >       at
> org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:183)
> >       at
> org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:114)
> >       at java.lang.Thread.run(Thread.java:745)
> > Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException:
> Class tachyon.hadoop.TFS not found
> >       at
> org.apache.hadoop.conf.Configuration.getClass(Configuration.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.<init>(FileOutputCommitter.java:105)
> >       at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.<init>(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 <hi...@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 "<dir>/*” instead of just "<dir>”
> > 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.simsa@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

Posted by Hitesh Shah <hi...@apache.org>.
I am not sure whether the fat-jar is creating an issue. Can you try the following: 

   - convert the fat-jar ( jar with dependencies ) into a directory of separate jars i.e. a dir with all jars within it ( no nested dirs - just one top level dir with all jars in it )
   - upload this dir to HDFS
   - add this dir to tez.aux.uris ( and remove the fat-jar )

Beyond this, if you are familiar with YARN, a few things you can do to capture more info which can help us help you: 
   - set "yarn.nodemanager.delete.debug-delay-sec” to say 600 or 900 in yarn-site.xml ( requires node manager restart ). This retains the container data after the application completes
   - find a launch_container.sh file for any container ( of the application that is failing ) within local-dirs configured for YARN in your yarn-site.xml 

The launch_container.sh will tell you what files are being added to the container’s env and how the classpath is being setup.

thanks
— Hitesh 


On Nov 16, 2015, at 8:41 AM, Jiří Šimša <ji...@gmail.com> wrote:

> Hi Bikas and Hitesh,
> 
> Any suggestion as to how to get tachyon.hadoop.TFS recognized by Tez's cluster? Thanks.
> 
> Best,
> 
> --
> Jiří Šimša
> 
> On Thu, Nov 12, 2015 at 10:42 AM, Jiří Šimša <ji...@gmail.com> wrote:
> Thank you Bikas and Hitesh for your responses.
> 
> I believe the problem is in the cluster. Here is the relevant information:
> 
> 1) My HADOOP_CLASSPATH:
> 
> $ hadoop classpath
> /usr/local/Cellar/hadoop/2.7.1/libexec/etc/hadoop:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/*:/Users/jsimsa/Projects/tez:/Users/jsimsa/Projects/tez/jars/*:/Users/jsimsa/Projects/tez/jars/lib/*:/contrib/capacity-scheduler/*.jar
> 
> 2) The contents of /Users/jsimsa/Projects/tez/tez-site.xml:
> 
> <?xml version="1.0"?>
> <!--
>   Licensed under the Apache License, Version 2.0 (the "License");
>   you may not use this file except in compliance with the License.
>   You may obtain a copy of the License at
> 
>     http://www.apache.org/licenses/LICENSE-2.0
> 
>   Unless required by applicable law or agreed to in writing, software
>   distributed under the License is distributed on an "AS IS" BASIS,
>   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>   See the License for the specific language governing permissions and
>   limitations under the License. See accompanying LICENSE file.
> -->
> 
> <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
> 
> <!-- Put site-specific property overrides in this file. -->
> 
> <configuration>
> 
> <property>
>   <name>tez.lib.uris</name>
>   <value>${fs.defaultFS}/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz</value> 
> </property>
> <property>
>   <name>tez.aux.uris</name>
>   <value>${fs.defaultFS}/apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar</value> 
> </property>
> 
> </configuration>
> 
> 3) The contents of the /apps HDFS folder:
> 
> $ ./bin/hdfs dfs -lsr /apps
> lsr: DEPRECATED: Please use 'ls -R' instead.
> SLF4J: Class path contains multiple SLF4J bindings.
> SLF4J: Found binding in [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
> 15/11/12 10:39:52 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
> drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:43 /apps/tachyon-0.8.2-SNAPSHOT
> -rw-r--r--   1 jsimsa supergroup   43809325 2015-11-11 18:43 /apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
> drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:44 /apps/tez-0.8.2-SNAPSHOT
> -rw-r--r--   1 jsimsa supergroup   43884378 2015-11-11 18:44 /apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
> 
> 
> 4) Finally, the command I am running and its output:
> 
> $ HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar orderedwordcount tachyon://localhost:19998/input.txt tachyon://localhost:19998/output.txt
> SLF4J: Class path contains multiple SLF4J bindings.
> SLF4J: Found binding in [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in [jar:file:/Users/jsimsa/Projects/tez/jars/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in [jar:file:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
> 15/11/12 10:37:29 WARN util.NativeCodeLoader: Unable to load native-hadoop library for your platform... using builtin-java classes where applicable
> 15/11/12 10:37:29 INFO client.TezClient: Tez Client Version: [ component=tez-api, version=0.8.2-SNAPSHOT, revision=6562a9d882fc455f511dd9d93af1d159d3e3e71b, SCM-URL=scm:git:https://git-wip-us.apache.org/repos/asf/tez.git, buildTime=2015-11-11T19:44:28Z ]
> 15/11/12 10:37:29 INFO client.RMProxy: Connecting to ResourceManager at /0.0.0.0:8032
> 15/11/12 10:37:30 INFO : initialize(tachyon://localhost:19998/input.txt, Configuration: core-default.xml, core-site.xml, mapred-default.xml, mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml, hdfs-site.xml, tez-site.xml). Connecting to Tachyon: tachyon://localhost:19998/input.txt
> 15/11/12 10:37:30 INFO : Loading Tachyon properties from Hadoop configuration: {}
> 15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is trying to connect with BlockMaster master @ localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : Client registered with BlockMaster master @ localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is trying to connect with FileSystemMaster master @ localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : Client registered with FileSystemMaster master @ localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : tachyon://localhost:19998 tachyon://localhost:19998 hdfs://localhost:9000
> 15/11/12 10:37:30 INFO : getWorkingDirectory: /
> 15/11/12 10:37:30 INFO : getWorkingDirectory: /
> 15/11/12 10:37:30 INFO examples.OrderedWordCount: Running OrderedWordCount
> 15/11/12 10:37:30 INFO client.TezClient: Submitting DAG application with id: application_1447296197811_0003
> 15/11/12 10:37:30 INFO client.TezClientUtils: Using tez.lib.uris value from configuration: hdfs://localhost:9000/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
> 15/11/12 10:37:30 INFO client.TezClient: Tez system stage directory hdfs://localhost:9000/tmp/jsimsa/tez/staging/.tez/application_1447296197811_0003 doesn't exist and is created
> 15/11/12 10:37:30 INFO client.TezClient: Submitting DAG to YARN, applicationId=application_1447296197811_0003, dagName=OrderedWordCount, callerContext={ context=TezExamples, callerType=null, callerId=null }
> 15/11/12 10:37:30 INFO impl.YarnClientImpl: Submitted application application_1447296197811_0003
> 15/11/12 10:37:30 INFO client.TezClient: The url to track the Tez AM: http://Jiris-MacBook-Pro.local:8088/proxy/application_1447296197811_0003/
> 15/11/12 10:37:34 INFO client.RMProxy: Connecting to ResourceManager at /0.0.0.0:8032
> 15/11/12 10:37:34 INFO client.DAGClientImpl: DAG: State: FAILED Progress: 0% TotalTasks: 1 Succeeded: 0 Running: 0 Failed: 0 Killed: 0
> 15/11/12 10:37:34 INFO client.DAGClientImpl: DAG completed. FinalState=FAILED
> 15/11/12 10:37:34 INFO examples.OrderedWordCount: DAG diagnostics: [Vertex failed, vertexName=Tokenizer, vertexId=vertex_1447296197811_0003_1_00, diagnostics=[Vertex vertex_1447296197811_0003_1_00 [Tokenizer] killed/failed due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: Input initializer failed, vertex=vertex_1447296197811_0003_1_00 [Tokenizer], java.lang.RuntimeException: java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not found
> 	at org.apache.hadoop.conf.Configuration.getClass(Configuration.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.input.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:293)
> 	at org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:265)
> 	at org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits(FileInputFormat.java:387)
> 	at org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat.getSplits(TezGroupedSplitsInputFormat.java:97)
> 	at org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateNewSplits(MRInputHelpers.java:413)
> 	at org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateInputSplitsToMem(MRInputHelpers.java:292)
> 	at org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.initialize(MRInputAMSplitGenerator.java:120)
> 	at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:246)
> 	at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:240)
> 	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.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:240)
> 	at org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:227)
> 	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.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)
> 	... 25 more
> ], Vertex failed, vertexName=Sorter, vertexId=vertex_1447296197811_0003_1_02, diagnostics=[Vertex init failed : org.apache.tez.dag.api.TezUncheckedException: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not found
> 	at org.apache.tez.mapreduce.committer.MROutputCommitter.getOutputCommitter(MROutputCommitter.java:139)
> 	at org.apache.tez.mapreduce.committer.MROutputCommitter.initialize(MROutputCommitter.java:81)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2334)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2316)
> 	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.tez.dag.app.dag.impl.VertexImpl.initializeCommitters(VertexImpl.java:2316)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl.initializeVertex(VertexImpl.java:2350)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl.access$4700(VertexImpl.java:204)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.handleInitEvent(VertexImpl.java:3497)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3394)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3375)
> 	at org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
> 	at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
> 	at org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
> 	at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
> 	at org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1975)
> 	at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:203)
> 	at org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2096)
> 	at org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2082)
> 	at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:183)
> 	at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:114)
> 	at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not found
> 	at org.apache.hadoop.conf.Configuration.getClass(Configuration.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.<init>(FileOutputCommitter.java:105)
> 	at org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.<init>(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 <hi...@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 "<dir>/*” instead of just "<dir>”
> 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.simsa@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

Posted by Jiří Šimša <ji...@gmail.com>.
Hi Bikas and Hitesh,

Any suggestion as to how to get tachyon.hadoop.TFS recognized by Tez's
cluster? Thanks.

Best,

--
Jiří Šimša

On Thu, Nov 12, 2015 at 10:42 AM, Jiří Šimša <ji...@gmail.com> wrote:

> Thank you Bikas and Hitesh for your responses.
>
> I believe the problem is in the cluster. Here is the relevant information:
>
> *1) My HADOOP_CLASSPATH:*
>
> $ hadoop classpath
>
> /usr/local/Cellar/hadoop/2.7.1/libexec/etc/hadoop:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/*:/Users/jsimsa/Projects/tez:/Users/jsimsa/Projects/tez/jars/*:/Users/jsimsa/Projects/tez/jars/lib/*:/contrib/capacity-scheduler/*.jar
>
> *2) The contents of /Users/jsimsa/Projects/tez/tez-site.xml:*
>
> <?xml version="1.0"?>
> <!--
>   Licensed under the Apache License, Version 2.0 (the "License");
>   you may not use this file except in compliance with the License.
>   You may obtain a copy of the License at
>
>     http://www.apache.org/licenses/LICENSE-2.0
>
>   Unless required by applicable law or agreed to in writing, software
>   distributed under the License is distributed on an "AS IS" BASIS,
>   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>   See the License for the specific language governing permissions and
>   limitations under the License. See accompanying LICENSE file.
> -->
>
> <?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
>
> <!-- Put site-specific property overrides in this file. -->
>
> <configuration>
>
> <property>
>   <name>tez.lib.uris</name>
>
> <value>${fs.defaultFS}/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz</value>
> </property>
> <property>
>   <name>tez.aux.uris</name>
>
> <value>${fs.defaultFS}/apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar</value>
> </property>
>
> </configuration>
>
> *3) The contents of the /apps HDFS folder:*
>
> $ ./bin/hdfs dfs -lsr /apps
> lsr: DEPRECATED: Please use 'ls -R' instead.
> SLF4J: Class path contains multiple SLF4J bindings.
> SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
> 15/11/12 10:39:52 WARN util.NativeCodeLoader: Unable to load native-hadoop
> library for your platform... using builtin-java classes where applicable
> drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:43
> /apps/tachyon-0.8.2-SNAPSHOT
> -rw-r--r--   1 jsimsa supergroup   43809325 2015-11-11 18:43
> /apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
> drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:44
> /apps/tez-0.8.2-SNAPSHOT
> -rw-r--r--   1 jsimsa supergroup   43884378 2015-11-11 18:44
> /apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
>
>
> *4) Finally, the command I am running and its output:*
>
> $
> HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
> hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar
> orderedwordcount tachyon://localhost:19998/input.txt
> tachyon://localhost:19998/output.txt
> SLF4J: Class path contains multiple SLF4J bindings.
> SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in
> [jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in
> [jar:file:/Users/jsimsa/Projects/tez/jars/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
> SLF4J: Found binding in
> [jar:file:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
> 15/11/12 10:37:29 WARN util.NativeCodeLoader: Unable to load native-hadoop
> library for your platform... using builtin-java classes where applicable
> 15/11/12 10:37:29 INFO client.TezClient: Tez Client Version: [
> component=tez-api, version=0.8.2-SNAPSHOT,
> revision=6562a9d882fc455f511dd9d93af1d159d3e3e71b, SCM-URL=scm:git:
> https://git-wip-us.apache.org/repos/asf/tez.git,
> buildTime=2015-11-11T19:44:28Z ]
> 15/11/12 10:37:29 INFO client.RMProxy: Connecting to ResourceManager at /
> 0.0.0.0:8032
> 15/11/12 10:37:30 INFO : initialize(tachyon://localhost:19998/input.txt,
> Configuration: core-default.xml, core-site.xml, mapred-default.xml,
> mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml,
> hdfs-site.xml, tez-site.xml). Connecting to Tachyon:
> tachyon://localhost:19998/input.txt
> 15/11/12 10:37:30 INFO : Loading Tachyon properties from Hadoop
> configuration: {}
> 15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is trying
> to connect with BlockMaster master @ localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : Client registered with BlockMaster master @
> localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is trying
> to connect with FileSystemMaster master @ localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : Client registered with FileSystemMaster master @
> localhost/127.0.0.1:19998
> 15/11/12 10:37:30 INFO : tachyon://localhost:19998
> tachyon://localhost:19998 hdfs://localhost:9000
> 15/11/12 10:37:30 INFO : getWorkingDirectory: /
> 15/11/12 10:37:30 INFO : getWorkingDirectory: /
> 15/11/12 10:37:30 INFO examples.OrderedWordCount: Running OrderedWordCount
> 15/11/12 10:37:30 INFO client.TezClient: Submitting DAG application with
> id: application_1447296197811_0003
> 15/11/12 10:37:30 INFO client.TezClientUtils: Using tez.lib.uris value
> from configuration:
> hdfs://localhost:9000/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
> 15/11/12 10:37:30 INFO client.TezClient: Tez system stage directory
> hdfs://localhost:9000/tmp/jsimsa/tez/staging/.tez/application_1447296197811_0003
> doesn't exist and is created
> 15/11/12 10:37:30 INFO client.TezClient: Submitting DAG to YARN,
> applicationId=application_1447296197811_0003, dagName=OrderedWordCount,
> callerContext={ context=TezExamples, callerType=null, callerId=null }
> 15/11/12 10:37:30 INFO impl.YarnClientImpl: Submitted application
> application_1447296197811_0003
> 15/11/12 10:37:30 INFO client.TezClient: The url to track the Tez AM:
> http://Jiris-MacBook-Pro.local:8088/proxy/application_1447296197811_0003/
> 15/11/12 10:37:34 INFO client.RMProxy: Connecting to ResourceManager at /
> 0.0.0.0:8032
> 15/11/12 10:37:34 INFO client.DAGClientImpl: DAG: State: FAILED Progress:
> 0% TotalTasks: 1 Succeeded: 0 Running: 0 Failed: 0 Killed: 0
> 15/11/12 10:37:34 INFO client.DAGClientImpl: DAG completed.
> FinalState=FAILED
> 15/11/12 10:37:34 INFO examples.OrderedWordCount: DAG diagnostics: [Vertex
> failed, vertexName=Tokenizer, vertexId=vertex_1447296197811_0003_1_00,
> diagnostics=[Vertex vertex_1447296197811_0003_1_00 [Tokenizer]
> killed/failed due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: Input
> initializer failed, vertex=vertex_1447296197811_0003_1_00 [Tokenizer],
> java.lang.RuntimeException: java.lang.ClassNotFoundException: Class
> tachyon.hadoop.TFS not found
> at org.apache.hadoop.conf.Configuration.getClass(Configuration.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.input.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:293)
> at
> org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:265)
> at
> org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits(FileInputFormat.java:387)
> at
> org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat.getSplits(TezGroupedSplitsInputFormat.java:97)
> at
> org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateNewSplits(MRInputHelpers.java:413)
> at
> org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateInputSplitsToMem(MRInputHelpers.java:292)
> at
> org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.initialize(MRInputAMSplitGenerator.java:120)
> at
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:246)
> at
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:240)
> 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.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:240)
> at
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:227)
> 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.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)
> ... 25 more
> ], Vertex failed, vertexName=Sorter,
> vertexId=vertex_1447296197811_0003_1_02, diagnostics=[Vertex init failed :
> org.apache.tez.dag.api.TezUncheckedException: java.lang.RuntimeException:
> java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not found
> at
> org.apache.tez.mapreduce.committer.MROutputCommitter.getOutputCommitter(MROutputCommitter.java:139)
> at
> org.apache.tez.mapreduce.committer.MROutputCommitter.initialize(MROutputCommitter.java:81)
> at org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2334)
> at org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2316)
> 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.tez.dag.app.dag.impl.VertexImpl.initializeCommitters(VertexImpl.java:2316)
> at
> org.apache.tez.dag.app.dag.impl.VertexImpl.initializeVertex(VertexImpl.java:2350)
> at
> org.apache.tez.dag.app.dag.impl.VertexImpl.access$4700(VertexImpl.java:204)
> at
> org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.handleInitEvent(VertexImpl.java:3497)
> at
> org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3394)
> at
> org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3375)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
> at
> org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
> at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1975)
> at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:203)
> at
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2096)
> at
> org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2082)
> at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:183)
> at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:114)
> at java.lang.Thread.run(Thread.java:745)
> Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException:
> Class tachyon.hadoop.TFS not found
> at org.apache.hadoop.conf.Configuration.getClass(Configuration.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.<init>(FileOutputCommitter.java:105)
> at
> org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.<init>(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 <hi...@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 "<dir>/*” instead of just "<dir>”
>> 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.simsa@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

Posted by Jiří Šimša <ji...@gmail.com>.
Thank you Bikas and Hitesh for your responses.

I believe the problem is in the cluster. Here is the relevant information:

*1) My HADOOP_CLASSPATH:*

$ hadoop classpath
/usr/local/Cellar/hadoop/2.7.1/libexec/etc/hadoop:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/hdfs/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/yarn/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/lib/*:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/mapreduce/*:/Users/jsimsa/Projects/tez:/Users/jsimsa/Projects/tez/jars/*:/Users/jsimsa/Projects/tez/jars/lib/*:/contrib/capacity-scheduler/*.jar

*2) The contents of /Users/jsimsa/Projects/tez/tez-site.xml:*

<?xml version="1.0"?>
<!--
  Licensed under the Apache License, Version 2.0 (the "License");
  you may not use this file except in compliance with the License.
  You may obtain a copy of the License at

    http://www.apache.org/licenses/LICENSE-2.0

  Unless required by applicable law or agreed to in writing, software
  distributed under the License is distributed on an "AS IS" BASIS,
  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
  See the License for the specific language governing permissions and
  limitations under the License. See accompanying LICENSE file.
-->

<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>

<!-- Put site-specific property overrides in this file. -->

<configuration>

<property>
  <name>tez.lib.uris</name>

<value>${fs.defaultFS}/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz</value>
</property>
<property>
  <name>tez.aux.uris</name>

<value>${fs.defaultFS}/apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar</value>
</property>

</configuration>

*3) The contents of the /apps HDFS folder:*

$ ./bin/hdfs dfs -lsr /apps
lsr: DEPRECATED: Please use 'ls -R' instead.
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in
[jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in
[jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
15/11/12 10:39:52 WARN util.NativeCodeLoader: Unable to load native-hadoop
library for your platform... using builtin-java classes where applicable
drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:43
/apps/tachyon-0.8.2-SNAPSHOT
-rw-r--r--   1 jsimsa supergroup   43809325 2015-11-11 18:43
/apps/tachyon-0.8.2-SNAPSHOT/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
drwxr-xr-x   - jsimsa supergroup          0 2015-11-11 18:44
/apps/tez-0.8.2-SNAPSHOT
-rw-r--r--   1 jsimsa supergroup   43884378 2015-11-11 18:44
/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz


*4) Finally, the command I am running and its output:*

$
HADOOP_CLASSPATH=${HADOOP_CLASSPATH}:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar
hadoop jar ./tez-examples/target/tez-examples-0.8.2-SNAPSHOT.jar
orderedwordcount tachyon://localhost:19998/input.txt
tachyon://localhost:19998/output.txt
SLF4J: Class path contains multiple SLF4J bindings.
SLF4J: Found binding in
[jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in
[jar:file:/usr/local/Cellar/hadoop/2.7.1/libexec/share/hadoop/common/lib/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in
[jar:file:/Users/jsimsa/Projects/tez/jars/lib/slf4j-log4j12-1.7.5.jar!/org/slf4j/impl/StaticLoggerBinder.class]
SLF4J: Found binding in
[jar:file:/Users/jsimsa/Projects/tachyon-amplab/clients/client/target/tachyon-client-0.8.2-SNAPSHOT-jar-with-dependencies.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]
15/11/12 10:37:29 WARN util.NativeCodeLoader: Unable to load native-hadoop
library for your platform... using builtin-java classes where applicable
15/11/12 10:37:29 INFO client.TezClient: Tez Client Version: [
component=tez-api, version=0.8.2-SNAPSHOT,
revision=6562a9d882fc455f511dd9d93af1d159d3e3e71b, SCM-URL=scm:git:
https://git-wip-us.apache.org/repos/asf/tez.git,
buildTime=2015-11-11T19:44:28Z ]
15/11/12 10:37:29 INFO client.RMProxy: Connecting to ResourceManager at /
0.0.0.0:8032
15/11/12 10:37:30 INFO : initialize(tachyon://localhost:19998/input.txt,
Configuration: core-default.xml, core-site.xml, mapred-default.xml,
mapred-site.xml, yarn-default.xml, yarn-site.xml, hdfs-default.xml,
hdfs-site.xml, tez-site.xml). Connecting to Tachyon:
tachyon://localhost:19998/input.txt
15/11/12 10:37:30 INFO : Loading Tachyon properties from Hadoop
configuration: {}
15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is trying
to connect with BlockMaster master @ localhost/127.0.0.1:19998
15/11/12 10:37:30 INFO : Client registered with BlockMaster master @
localhost/127.0.0.1:19998
15/11/12 10:37:30 INFO : Tachyon client (version 0.8.2-SNAPSHOT) is trying
to connect with FileSystemMaster master @ localhost/127.0.0.1:19998
15/11/12 10:37:30 INFO : Client registered with FileSystemMaster master @
localhost/127.0.0.1:19998
15/11/12 10:37:30 INFO : tachyon://localhost:19998
tachyon://localhost:19998 hdfs://localhost:9000
15/11/12 10:37:30 INFO : getWorkingDirectory: /
15/11/12 10:37:30 INFO : getWorkingDirectory: /
15/11/12 10:37:30 INFO examples.OrderedWordCount: Running OrderedWordCount
15/11/12 10:37:30 INFO client.TezClient: Submitting DAG application with
id: application_1447296197811_0003
15/11/12 10:37:30 INFO client.TezClientUtils: Using tez.lib.uris value from
configuration:
hdfs://localhost:9000/apps/tez-0.8.2-SNAPSHOT/tez-0.8.2-SNAPSHOT.tar.gz
15/11/12 10:37:30 INFO client.TezClient: Tez system stage directory
hdfs://localhost:9000/tmp/jsimsa/tez/staging/.tez/application_1447296197811_0003
doesn't exist and is created
15/11/12 10:37:30 INFO client.TezClient: Submitting DAG to YARN,
applicationId=application_1447296197811_0003, dagName=OrderedWordCount,
callerContext={ context=TezExamples, callerType=null, callerId=null }
15/11/12 10:37:30 INFO impl.YarnClientImpl: Submitted application
application_1447296197811_0003
15/11/12 10:37:30 INFO client.TezClient: The url to track the Tez AM:
http://Jiris-MacBook-Pro.local:8088/proxy/application_1447296197811_0003/
15/11/12 10:37:34 INFO client.RMProxy: Connecting to ResourceManager at /
0.0.0.0:8032
15/11/12 10:37:34 INFO client.DAGClientImpl: DAG: State: FAILED Progress:
0% TotalTasks: 1 Succeeded: 0 Running: 0 Failed: 0 Killed: 0
15/11/12 10:37:34 INFO client.DAGClientImpl: DAG completed.
FinalState=FAILED
15/11/12 10:37:34 INFO examples.OrderedWordCount: DAG diagnostics: [Vertex
failed, vertexName=Tokenizer, vertexId=vertex_1447296197811_0003_1_00,
diagnostics=[Vertex vertex_1447296197811_0003_1_00 [Tokenizer]
killed/failed due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: Input
initializer failed, vertex=vertex_1447296197811_0003_1_00 [Tokenizer],
java.lang.RuntimeException: java.lang.ClassNotFoundException: Class
tachyon.hadoop.TFS not found
at org.apache.hadoop.conf.Configuration.getClass(Configuration.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.input.FileInputFormat.singleThreadedListStatus(FileInputFormat.java:293)
at
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.listStatus(FileInputFormat.java:265)
at
org.apache.hadoop.mapreduce.lib.input.FileInputFormat.getSplits(FileInputFormat.java:387)
at
org.apache.hadoop.mapreduce.split.TezGroupedSplitsInputFormat.getSplits(TezGroupedSplitsInputFormat.java:97)
at
org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateNewSplits(MRInputHelpers.java:413)
at
org.apache.tez.mapreduce.hadoop.MRInputHelpers.generateInputSplitsToMem(MRInputHelpers.java:292)
at
org.apache.tez.mapreduce.common.MRInputAMSplitGenerator.initialize(MRInputAMSplitGenerator.java:120)
at
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:246)
at
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:240)
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.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:240)
at
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:227)
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.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)
... 25 more
], Vertex failed, vertexName=Sorter,
vertexId=vertex_1447296197811_0003_1_02, diagnostics=[Vertex init failed :
org.apache.tez.dag.api.TezUncheckedException: java.lang.RuntimeException:
java.lang.ClassNotFoundException: Class tachyon.hadoop.TFS not found
at
org.apache.tez.mapreduce.committer.MROutputCommitter.getOutputCommitter(MROutputCommitter.java:139)
at
org.apache.tez.mapreduce.committer.MROutputCommitter.initialize(MROutputCommitter.java:81)
at org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2334)
at org.apache.tez.dag.app.dag.impl.VertexImpl$2.run(VertexImpl.java:2316)
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.tez.dag.app.dag.impl.VertexImpl.initializeCommitters(VertexImpl.java:2316)
at
org.apache.tez.dag.app.dag.impl.VertexImpl.initializeVertex(VertexImpl.java:2350)
at
org.apache.tez.dag.app.dag.impl.VertexImpl.access$4700(VertexImpl.java:204)
at
org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.handleInitEvent(VertexImpl.java:3497)
at
org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3394)
at
org.apache.tez.dag.app.dag.impl.VertexImpl$InitTransition.transition(VertexImpl.java:3375)
at
org.apache.hadoop.yarn.state.StateMachineFactory$MultipleInternalArc.doTransition(StateMachineFactory.java:385)
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.tez.state.StateMachineTez.doTransition(StateMachineTez.java:59)
at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:1975)
at org.apache.tez.dag.app.dag.impl.VertexImpl.handle(VertexImpl.java:203)
at
org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2096)
at
org.apache.tez.dag.app.DAGAppMaster$VertexEventDispatcher.handle(DAGAppMaster.java:2082)
at org.apache.tez.common.AsyncDispatcher.dispatch(AsyncDispatcher.java:183)
at org.apache.tez.common.AsyncDispatcher$1.run(AsyncDispatcher.java:114)
at java.lang.Thread.run(Thread.java:745)
Caused by: java.lang.RuntimeException: java.lang.ClassNotFoundException:
Class tachyon.hadoop.TFS not found
at org.apache.hadoop.conf.Configuration.getClass(Configuration.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.<init>(FileOutputCommitter.java:105)
at
org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter.<init>(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 <hi...@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
> "<dir>/*” instead of just "<dir>”
> 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.simsa@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

Posted by Hitesh Shah <hi...@apache.org>.
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 "<dir>/*” instead of just "<dir>”
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.simsa@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

Posted by Bikas Saha <bi...@apache.org>.
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.simsa@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