[ 
https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15303400#comment-15303400
 ] 

Srikanth Sundarrajan commented on PIG-4903:
-------------------------------------------

If spark-assembly is present, then nothing else is needed, however if 
spark-assembly isn't available and you need to materialize the dependency 
through all direct & transitive dependency of spark-core & spark-yarn, I think 
you will need them all the dependencies in the Yarn container classpath. The 
SPARK_YARN_DIST_FILES & SPARK_DIST_CLASSPATH, will help achieve this. 
(org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java uses 
the env var SPARK_JARS to figure the SPARK_JARS while launching).

I dont recall the exact reason for excluding the spark-yarn* explicitly, but I 
vaguely remember it causing duplicate spark-yarn*.jar in the dist cache and 
that causing issues. I can dig that up and revert.

> Avoid add all spark dependency jars to  SPARK_YARN_DIST_FILES and 
> SPARK_DIST_CLASSPATH
> --------------------------------------------------------------------------------------
>
>                 Key: PIG-4903
>                 URL: https://issues.apache.org/jira/browse/PIG-4903
>             Project: Pig
>          Issue Type: Sub-task
>          Components: spark
>            Reporter: liyunzhang_intel
>
> There are some comments about bin/pig on 
> https://reviews.apache.org/r/45667/#comment198955.
> {code}
> ################# ADDING SPARK DEPENDENCIES ##################
> # Spark typically works with a single assembly file. However this
> # assembly isn't available as a artifact to pull in via ivy.
> # To work around this short coming, we add all the jars barring
> # spark-yarn to DIST through dist-files and then add them to classpath
> # of the executors through an independent env variable. The reason
> # for excluding spark-yarn is because spark-yarn is already being added
> # by the spark-yarn-client via jarOf(Client.Class)
> for f in $PIG_HOME/lib/*.jar; do
>     if [[ $f == $PIG_HOME/lib/spark-assembly* ]]; then
>         # Exclude spark-assembly.jar from shipped jars, but retain in 
> classpath
>         SPARK_JARS=${SPARK_JARS}:$f;
>     else
>         SPARK_JARS=${SPARK_JARS}:$f;
>         SPARK_YARN_DIST_FILES=${SPARK_YARN_DIST_FILES},file://$f;
>         SPARK_DIST_CLASSPATH=${SPARK_DIST_CLASSPATH}:\${PWD}/`basename $f`
>     fi
> done
> CLASSPATH=${CLASSPATH}:${SPARK_JARS}
> export SPARK_YARN_DIST_FILES=`echo ${SPARK_YARN_DIST_FILES} | sed 's/^,//g'`
> export SPARK_JARS=${SPARK_YARN_DIST_FILES}
> export SPARK_DIST_CLASSPATH
> {code}
> Here we first copy all spark dependency jar like 
> spark-network-shuffle_2.10-1.6.1 jar to distcache(SPARK_YARN_DIST_FILES) then 
> add them to the classpath of executor(SPARK_DIST_CLASSPATH). Actually we need 
> not copy all these depency jar to SPARK_DIST_CLASSPATH because all these 
> dependency jars are included in spark-assembly.jar and spark-assembly.jar is 
> uploaded with the spark job.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to