[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15333410#comment-15333410 ] Srikanth Sundarrajan commented on PIG-4903: --- For spark mode, we should support yarn-client, mesos & spark, so that should work based on SPARK_MASTER value. As you rightly called out, setting it to LOCAL for spark_local ought to work. > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch, PIG-4903_2.patch, > PIG-4903_3.patch, PIG-4903_4.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1585#comment-1585 ] Srikanth Sundarrajan commented on PIG-4903: --- Sorry to nitpick [~kellyzly], would this print a 0/1 on the console while the script runs ? {code} + if [[ "$preArg" == "-x" || "$preArg" == "-exectype" ]]; then +execTypeUpperCase=$(echo $execType |tr [a-z] [A-Z]) +if [[ "$execTypeUpperCase" == "$expectExecType" ]]; then +echo 0 +else +echo 1 +fi + else +echo 1 + fi {code} One other thing: What is user choose the exec mode to be spark_local, whereas the env has SPARK_MASTER set to yarn-client. It would force the execution to be yarn-client mode. If mode is spark_local, can the SPARK_MASTER be forced to local mode ? > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch, PIG-4903_2.patch, > PIG-4903_3.patch, PIG-4903_4.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=1532#comment-1532 ] Srikanth Sundarrajan commented on PIG-4903: --- Looks largely ok, except the following: {code} + elif [[ $f == "spark" ]]; then +isSparkMode="true" +remaining[${#remaining[@]}]="$f" + elif [[ $f == "spark_local" || $f == "SPARK_LOCAL" ]]; then +isSparkLocalMode="true" +remaining[${#remaining[@]}]="$f" {code} Any argument with contents "spark", would assume the execution mode to be spark. Can we verify the previous argument and ensure that it is indeed -x? > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch, PIG-4903_2.patch, > PIG-4903_3.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15325646#comment-15325646 ] Srikanth Sundarrajan commented on PIG-4903: --- Makes sense to keep it consistent. Should we then take the following approach: * Make spark_local an explicit execution mode, the other one being just spark. If selected mode is spark_local, any value of env::SPARK_MASTER has to be ignored * If exec mode is spark, then have spark fall back to env::SPARK_MASTER, mandate env::SPARK_HOME > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch, PIG-4903_2.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15324640#comment-15324640 ] Srikanth Sundarrajan commented on PIG-4903: --- Yes [~rohini], By default (particularly if SPARK_MASTER env is not set to yarn-client, yarn-cluster, mesos://... or spark://...), pig -x spark will run the script in local mode. > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch, PIG-4903_2.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15321850#comment-15321850 ] Srikanth Sundarrajan commented on PIG-4903: --- Looked at the latest patch, it still seems like SPARK_HOME (additionally SPARK_JAR) is being checked if they are present. Shouldn't we be doing this only for spark mode ? I think some special handling is necessary for this. By default running pig -x spark will run this in local mode, and doesn't require any spark cluster or hdfs to be present and this allows new users to try and use it quickly. My feeling is that requiring and mandating these exports to be present always seems a bit unfriendly. But will not hold this back for this reason. > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch, PIG-4903_2.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15313486#comment-15313486 ] Srikanth Sundarrajan commented on PIG-4903: --- Have two followup questions on the patch. 1. Will this mandate SPARK_HOME to be set even when the execution is not spark (ex. mapreduce, tez etc) ? 2. Do we want to warn that SPARK_HOME is not set and continue execution using individual jars as opposed to forcing / mandating SPARK_HOME > 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 > Attachments: PIG-4903.patch, PIG-4903_1.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15309216#comment-15309216 ] Srikanth Sundarrajan commented on PIG-4903: --- Two things to consider. 1. Since Spark Assembly (without hadoop) isn't available on maven repo, at build time, you will have to depend on spark core with transitive dependencies. Also new users of pig may find it difficult to install spark and have SPARK_HOME available and it may be less friction if the option to run pig-on-spark without requiring SPARK_HOME 2. At run time, having a single assembly jar greatly helps and we would like to avoid shipping individual jars. For these reasons, the code snippet I had attached, on priority checks for existence of SPARK_HOME, when not present it falls back to including core + transitive dependencies. Makes sense ? > 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 > Attachments: PIG-4903.patch > > > 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)
[jira] [Commented] (PIG-4903) Avoid add all spark dependency jars to SPARK_YARN_DIST_FILES and SPARK_DIST_CLASSPATH
[ https://issues.apache.org/jira/browse/PIG-4903?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=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)
Re: Welcome to our new Pig PMC member Xuefu Zhang
Congratulations Xuefu Regards, Srikanth Sundarrajan On Thu, Feb 25, 2016 at 10:07 AM Xuefu Zhang <xzh...@cloudera.com> wrote: > Thank you, Liyun! You did the hard work. I think you well deserve a > committership once we merge the branch to trunk. > > --Xuefu > > On Wed, Feb 24, 2016 at 5:18 PM, Zhang, Liyun <liyun.zh...@intel.com> > wrote: > > > Congratulations Xuefu! > > > > > > Kelly Zhang/Zhang,Liyun > > Best Regards > > > > > > > > -Original Message- > > From: Jarek Jarcec Cecho [mailto:jar...@gmail.com] On Behalf Of Jarek > > Jarcec Cecho > > Sent: Thursday, February 25, 2016 6:36 AM > > To: dev@pig.apache.org > > Cc: u...@pig.apache.org > > Subject: Re: Welcome to our new Pig PMC member Xuefu Zhang > > > > Congratulations Xuefu! > > > > Jarcec > > > > > On Feb 24, 2016, at 1:29 PM, Rohini Palaniswamy < > rohini.adi...@gmail.com> > > wrote: > > > > > > It is my pleasure to announce that Xuefu Zhang is our newest addition > > > to the Pig PMC. Xuefu is a long time committer of Pig and has been > > > actively involved in driving the Pig on Spark effort for the past year. > > > > > > Please join me in congratulating Xuefu !!! > > > > > > Regards, > > > Rohini > > > > >
[jira] [Assigned] (PIG-4746) Ensure spork can be run as PIG action in Oozie
[ https://issues.apache.org/jira/browse/PIG-4746?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan reassigned PIG-4746: - Assignee: Srikanth Sundarrajan > Ensure spork can be run as PIG action in Oozie > -- > > Key: PIG-4746 > URL: https://issues.apache.org/jira/browse/PIG-4746 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: Pallavi Rao > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > > I was able get PIG on SPARK going with Oozie. But, only in "local" mode. Here > is what I did: > 1. Used workflow schema version uri:oozie:workflow:0.2 and passed exectype as > an argument. > 2. Copied Spark jars + kyro jar into workflow app lib. > To get spork going in yarn-client mode, couple of enhancements will need to > be made: > 1. Right now, spark launcher reads SPARK_MASTER as env. variable. Need to > make this a PIG property. > 2. The spark libraries need to be in classpath of the driver in case of > yarn-clientmode. This will need a fix similar to PIG-4667 -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4741) the value of $SPARK_DIST_CLASSPATH in pig file is invalid
[ https://issues.apache.org/jira/browse/PIG-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15021754#comment-15021754 ] Srikanth Sundarrajan commented on PIG-4741: --- [~kellyzly], We did attempt to support in [yarn-cluster|https://issues.apache.org/jira/browse/PIG-4681] mode, but it seemed a bit involved, so marked it as Wont Fix. Should there be interest, we can re-look at it. > the value of $SPARK_DIST_CLASSPATH in pig file is invalid > - > > Key: PIG-4741 > URL: https://issues.apache.org/jira/browse/PIG-4741 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: liyunzhang_intel >Assignee: liyunzhang_intel > Fix For: spark-branch > > Attachments: PIG-4741.patch > > > the value of > [$SPARK_DIST_CLASSPATH|https://github.com/apache/pig/blob/spark/bin/pig#L380] > in bin/pig is invalid > {code} > SPARK_DIST_CLASSPATH=${SPARK_DIST_CLASSPATH}:\${PWD}/`basename $f` > {code} > there is no need to escape the {{PWD}}. If we add "\", the value of > SPARK_DIST_CLASSPATH will like: > {code} > > ${PWD}/akka-actor_2.10-2.3.4-spark.jar:${PWD}/akka-remote_2.10-2.3.4-spark.jar > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4741) the value of $SPARK_DIST_CLASSPATH in pig file is invalid
[ https://issues.apache.org/jira/browse/PIG-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15021488#comment-15021488 ] Srikanth Sundarrajan commented on PIG-4741: --- SPARK_DIST_CLASSPATH is used by AM launcher via Node Manager::ContainerExecutor. If the environment variable holds un-substituted PWD, the Container Launcher script would substitute this with container home directory which would be something along the lines of {{/usercache//appcache//}}. PWD will be substituted with this home directory location on the AM. On the other hand if it is left directly as {{${PWD}}}, it would get replaced with the local directory from where the pig script is launched and you are most likely to see Container exit with class not found exception. > the value of $SPARK_DIST_CLASSPATH in pig file is invalid > - > > Key: PIG-4741 > URL: https://issues.apache.org/jira/browse/PIG-4741 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: liyunzhang_intel >Assignee: liyunzhang_intel > Fix For: spark-branch > > Attachments: PIG-4741.patch > > > the value of > [$SPARK_DIST_CLASSPATH|https://github.com/apache/pig/blob/spark/bin/pig#L380] > in bin/pig is invalid > {code} > SPARK_DIST_CLASSPATH=${SPARK_DIST_CLASSPATH}:\${PWD}/`basename $f` > {code} > there is no need to escape the {{PWD}}. If we add "\", the value of > SPARK_DIST_CLASSPATH will like: > {code} > > ${PWD}/akka-actor_2.10-2.3.4-spark.jar:${PWD}/akka-remote_2.10-2.3.4-spark.jar > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4741) the value of $SPARK_DIST_CLASSPATH in pig file is invalid
[ https://issues.apache.org/jira/browse/PIG-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15021021#comment-15021021 ] Srikanth Sundarrajan commented on PIG-4741: --- If there is an issue, you can consider replacing the {noformat} ${PWD}/ with ./ {noformat} > the value of $SPARK_DIST_CLASSPATH in pig file is invalid > - > > Key: PIG-4741 > URL: https://issues.apache.org/jira/browse/PIG-4741 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: liyunzhang_intel >Assignee: liyunzhang_intel > Fix For: spark-branch > > Attachments: PIG-4741.patch > > > the value of > [$SPARK_DIST_CLASSPATH|https://github.com/apache/pig/blob/spark/bin/pig#L380] > in bin/pig is invalid > {code} > SPARK_DIST_CLASSPATH=${SPARK_DIST_CLASSPATH}:\${PWD}/`basename $f` > {code} > there is no need to escape the {{PWD}}. If we add "\", the value of > SPARK_DIST_CLASSPATH will like: > {code} > > ${PWD}/akka-actor_2.10-2.3.4-spark.jar:${PWD}/akka-remote_2.10-2.3.4-spark.jar > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4741) the value of $SPARK_DIST_CLASSPATH in pig file is invalid
[ https://issues.apache.org/jira/browse/PIG-4741?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15020964#comment-15020964 ] Srikanth Sundarrajan commented on PIG-4741: --- "\" before ${PWD} is required to escape substitution, else PWD will be substituted by the pig script locally in the launcher instead of getting substituted at the AM host. [~kellyzly]/ [~xuefuz], Is there an issue with the system currently that required this fix ? > the value of $SPARK_DIST_CLASSPATH in pig file is invalid > - > > Key: PIG-4741 > URL: https://issues.apache.org/jira/browse/PIG-4741 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: liyunzhang_intel >Assignee: liyunzhang_intel > Fix For: spark-branch > > Attachments: PIG-4741.patch > > > the value of > [$SPARK_DIST_CLASSPATH|https://github.com/apache/pig/blob/spark/bin/pig#L380] > in bin/pig is invalid > {code} > SPARK_DIST_CLASSPATH=${SPARK_DIST_CLASSPATH}:\${PWD}/`basename $f` > {code} > there is no need to escape the {{PWD}}. If we add "\", the value of > SPARK_DIST_CLASSPATH will like: > {code} > > ${PWD}/akka-actor_2.10-2.3.4-spark.jar:${PWD}/akka-remote_2.10-2.3.4-spark.jar > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4720) Spark related JARs are not included when importing project via IDE
[ https://issues.apache.org/jira/browse/PIG-4720?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14980088#comment-14980088 ] Srikanth Sundarrajan commented on PIG-4720: --- Patch looks good. Verified build as well. +1 > Spark related JARs are not included when importing project via IDE > --- > > Key: PIG-4720 > URL: https://issues.apache.org/jira/browse/PIG-4720 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: Xianda Ke >Assignee: Xianda Ke > Fix For: spark-branch > > Attachments: PIG-4720.patch > > > It is a minior issue. Spark related JARs are not included when importing > project via IDE. > {code} > $ ant -Dhadoopversion=23 eclipse-files > {code} > Open the generated .classpath, the spark related JARs are not in the > classpathentry list. Because the spark JARs were moved to a new > directory(PIG-4667), but eclipse-files target in build.xml are not changed. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14974015#comment-14974015 ] Srikanth Sundarrajan commented on PIG-4698: --- FYKA [~xuefuz] / [~mohitsabharwal] > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > Attachments: PIG-4698.patch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4698: -- Status: Patch Available (was: Open) > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > Attachments: PIG-4698.patch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 39641: PIG-4698 Enable dynamic resource allocation/de-allocation on Yarn backends
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/39641/ --- Review request for pig. Bugs: PIG-4698 https://issues.apache.org/jira/browse/PIG-4698 Repository: pig-git Description --- Resource elasticity needs to be enabled on Yarn backend to allow jobs to scale out better and provide better wall clock execution times, while unused resources should be released back to RM for use. Diffs - src/docs/src/documentation/content/xdocs/start.xml eedd5b7 src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java b542013 Diff: https://reviews.apache.org/r/39641/diff/ Testing --- Verified that the dynamic configuration is hornoured by the yarn system. Requires the auxillary shuffle service need to be enabled at the node manager and application level for this to work correctly. Thanks, Srikanth Sundarrajan
[jira] [Commented] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14966876#comment-14966876 ] Srikanth Sundarrajan commented on PIG-4698: --- Yes. We need to enable auxillary shuffle service in YARN. Here is the list of changes I had to do to get it to work. 1. Copied spark-1.4.1-yarn-shuffle.jar to yarn/lib folder on NodeManagers 2. Had to change the yarn-site on the NodeManager with the following changes {noformat} yarn.nodemanager.aux-services mapreduce_shuffle,spark_shuffle yarn.nodemanager.aux-services.spark_shuffle.class org.apache.spark.network.yarn.YarnShuffleService spark.shuffle.service.enabled true {noformat} 3. Restart the node manager The gist of the change that you are likely to see in the patch is {code} //Copy all spark.* properties to SparkConf for (String key : pigCtxtProperties.stringPropertyNames()) { if (key.startsWith("spark.")) { LOG.debug("Copying key " + key + " with value " + pigCtxtProperties.getProperty(key) + " to SparkConf"); sparkConf.set(key, pigCtxtProperties.getProperty(key)); } } {code} besides a few other changes & documentation. I will finalize the patch and upload it sometime tomorrow. Also will verify some scenarios where executors are removed and things fall back to aux-shuffle service works fine. > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan >Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14966590#comment-14966590 ] Srikanth Sundarrajan commented on PIG-4698: --- Or more tersely {code} set spark.dynamicAllocation.enabled true; A = LOAD '/tmp/in' USING PigStorage('\t') AS (line); STORE A INTO '/tmp/out' USING PigStorage(','); {code} dropping {{set spark.shuffle.service.enabled true;}}, as the same can be automatically set if dynamic is enabled in YARN mode. > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14966587#comment-14966587 ] Srikanth Sundarrajan commented on PIG-4698: --- Am looking at propagating conf set in pig script to spark automatically such that this feature can be enabled without too much translation inside pig for approach #1. For instance the example below should do the trick. {code} set spark.dynamicAllocation.enabled true; set spark.shuffle.service.enabled true; A = LOAD '/tmp/in' USING PigStorage('\t') AS (line); STORE A INTO '/tmp/out' USING PigStorage(','); {code} [~xuefuz], Thoughts ? > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4693: -- Attachment: PIG-4693.patch Verified that OrcStorage works well with kryo-2.21. Used the following simple example to verify. {code} A = LOAD 'student.txt' USING PigStorage(',') as (name:chararray, age:int, gpa:double); store A into 'student.orc' using OrcStorage(''); {code} Uploaded a simple patch downgrading the kryo dependency > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > Attachments: PIG-4693.patch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4693: -- Status: Patch Available (was: Open) > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > Attachments: PIG-4693.patch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4680) Enable pig job graphs to resume from last successful state
[ https://issues.apache.org/jira/browse/PIG-4680?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14951708#comment-14951708 ] Srikanth Sundarrajan commented on PIG-4680: --- [~rohini], [~daijy], Would really appreciate some early feedback on the ask and the approach. > Enable pig job graphs to resume from last successful state > -- > > Key: PIG-4680 > URL: https://issues.apache.org/jira/browse/PIG-4680 > Project: Pig > Issue Type: Improvement > Components: impl >Reporter: Abhishek Agarwal >Assignee: Abhishek Agarwal > Attachments: PIG-4680.patch > > > Pig scripts can have multiple ETL jobs in the DAG which may take hours to > finish. In case of transient errors, the job fails. When the job is rerun, > all the nodes in Job graph will rerun. Some of these nodes may have already > run successfully. Redundant runs lead to wastage of cluster capacity and > pipeline delays. > In case of failure, we can persist the graph state. In next run, only the > failed nodes and their successors will rerun. This is of course subject to > preconditions such as > - Pig script has not changed > - Input locations have not changed > - Output data from previous run is intact > - Configuration has not changed -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14948889#comment-14948889 ] Srikanth Sundarrajan commented on PIG-4693: --- [~xuefuz], From what I can tell, switching the loading order of kryo wont help. My suggestion would be to downgrade the kryo dependency in pig to be consistent. I can verify if Orc wells with this change. Will wait for inputs from [~vanzin] before I attempt that. > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14948991#comment-14948991 ] Srikanth Sundarrajan commented on PIG-4693: --- Thanks [~vanzin]. The trunk version of chill still [refers|https://github.com/twitter/chill/blob/develop/chill-scala/src/main/scala/com/twitter/chill/KryoBase.scala#L24] to org.objenesis.strategy.InstantiatorStrategy, so I guess the option to use newer version of chill to overcome this isn't viable. > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
Srikanth Sundarrajan created PIG-4698: - Summary: Enable dynamic resource allocation/de-allocation on Yarn backends Key: PIG-4698 URL: https://issues.apache.org/jira/browse/PIG-4698 Project: Pig Issue Type: Sub-task Components: spark Affects Versions: spark-branch Reporter: Srikanth Sundarrajan Assignee: Srikanth Sundarrajan -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4698: -- Description: Resource elasticity needs to be enabled on Yarn backend to allow jobs to scale out better and provide better wall clock execution times, while unused resources should be released back to RM for use. > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4698) Enable dynamic resource allocation/de-allocation on Yarn backends
[ https://issues.apache.org/jira/browse/PIG-4698?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14949904#comment-14949904 ] Srikanth Sundarrajan commented on PIG-4698: --- There are a couple of options on how we can go about this 1. Spark supports [dynamic allocation|http://spark.apache.org/docs/latest/job-scheduling.html#dynamic-resource-allocation] and the same can be [configured|http://spark.apache.org/docs/latest/configuration.html#dynamic-allocation] for Yarn backends. As a first step this can be enabled and this simply allows for the executors to expand and shrink between a min & max bound. 2. As a subsequent effort we can attempt to use SparkContext::requestExecutors() and SparkContext::killExecutors appropriately to control this in a fine grained fashion depending on the stage of execution and resources required for that stage. Would prefer that we go with approach #1 for now. [~xuefuz], suggested the same in an offline conversation as well. Thoughts? > Enable dynamic resource allocation/de-allocation on Yarn backends > - > > Key: PIG-4698 > URL: https://issues.apache.org/jira/browse/PIG-4698 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch >Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > > Resource elasticity needs to be enabled on Yarn backend to allow jobs to > scale out better and provide better wall clock execution times, while unused > resources should be released back to RM for use. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14946525#comment-14946525 ] Srikanth Sundarrajan commented on PIG-4693: --- Issue persists even with usage with spark-core. {noformat} [Loaded org.apache.spark.broadcast.TorrentBroadcast$$anonfun$5 from file:/data/d1/home/sriksun/pig/lib/spark/spark-core_2.10-1.4.1.jar] [Loaded org.apache.spark.serializer.KryoSerializationStream from file:/data/d1/home/sriksun/pig/lib/spark/spark-core_2.10-1.4.1.jar] [Loaded org.apache.spark.serializer.KryoDeserializationStream from file:/data/d1/home/sriksun/pig/lib/spark/spark-core_2.10-1.4.1.jar] [Loaded com.twitter.chill.KryoInstantiator from file:/data/d1/home/sriksun/pig/lib/spark/chill-java-0.5.0.jar] [Loaded com.twitter.chill.EmptyScalaKryoInstantiator from file:/data/d1/home/sriksun/pig/lib/spark/chill_2.10-0.5.0.jar] [Loaded com.twitter.chill.KryoInstantiator$3 from file:/data/d1/home/sriksun/pig/lib/spark/chill-java-0.5.0.jar] [Loaded com.twitter.chill.KryoInstantiator$2 from file:/data/d1/home/sriksun/pig/lib/spark/chill-java-0.5.0.jar] [Loaded com.twitter.chill.KryoInstantiator$1 from file:/data/d1/home/sriksun/pig/lib/spark/chill-java-0.5.0.jar] [Loaded com.twitter.chill.KryoInstantiator$4 from file:/data/d1/home/sriksun/pig/lib/spark/chill-java-0.5.0.jar] [Loaded org.objenesis.strategy.InstantiatorStrategy from file:/usr/hdp/2.2.0.0-2041/hadoop-mapreduce/mockito-all-1.8.5.jar] [Loaded com.esotericsoftware.kryo.KryoException from file:/data/d1/home/sriksun/pig/lib/kryo-2.22.jar] {noformat} > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14948068#comment-14948068 ] Srikanth Sundarrajan commented on PIG-4693: --- That might not help. As this is a direct dependency for spark-core core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala {code} ... import com.esotericsoftware.kryo.{Kryo, KryoException} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.roaringbitmap.{ArrayContainer, BitmapContainer, RoaringArray, RoaringBitmap} ... def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator val kryo = instantiator.newKryo() ... {code} /chill_2.10/0.5.0/com/twitter/chill/ScalaKryoInstantiator.scala {code} package com.twitter.chill ... class EmptyScalaKryoInstantiator extends KryoInstantiator { override def newKryo = { val k = new KryoBase k.setRegistrationRequired(false) k.setInstantiatorStrategy(new org.objenesis.strategy.StdInstantiatorStrategy) k } } ... {code} /chill_2.10/0.5.0/com/twitter/chill/KryoBase.scala {code} ... package com.twitter.chill ... import org.objenesis.instantiator.ObjectInstantiator import org.objenesis.strategy.InstantiatorStrategy ... class KryoBase extends Kryo { ... override def setInstantiatorStrategy(st: InstantiatorStrategy) { super.setInstantiatorStrategy(st) strategy = Some(st) } ... {code} > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan >Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14947062#comment-14947062 ] Srikanth Sundarrajan commented on PIG-4693: --- Yes. After the AM is launched, client makes a request to the AM for newHadoopRDD corresponding to the Load Statement, which is shipped through the kryo serializer as I had set this as the default spark serializer. The stack trace in the original comment has more detail. Posting a section of it here for quick reference. {noformat} java.lang.NoSuchMethodError: com.esotericsoftware.kryo.Kryo.setInstantiatorStrategy(Lorg/objenesis/strategy/InstantiatorStrategy;)V at com.twitter.chill.KryoBase.setInstantiatorStrategy(KryoBase.scala:86) at com.twitter.chill.EmptyScalaKryoInstantiator.newKryo(ScalaKryoInstantiator.scala:59) at org.apache.spark.serializer.KryoSerializer.newKryo(KryoSerializer.scala:80) at org.apache.spark.serializer.KryoSerializerInstance.borrowKryo(KryoSerializer.scala:227) at org.apache.spark.serializer.KryoSerializerInstance.(KryoSerializer.scala:212) at org.apache.spark.serializer.KryoSerializer.newInstance(KryoSerializer.scala:128) at org.apache.spark.broadcast.TorrentBroadcast$.blockifyObject(TorrentBroadcast.scala:201) at org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:102) at org.apache.spark.broadcast.TorrentBroadcast.(TorrentBroadcast.scala:85) at org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:34) at org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:62) at org.apache.spark.SparkContext.broadcast(SparkContext.scala:1291) at org.apache.spark.rdd.NewHadoopRDD.(NewHadoopRDD.scala:77) at org.apache.spark.SparkContext$$anonfun$newAPIHadoopRDD$1.apply(SparkContext.scala:1099) at org.apache.spark.SparkContext$$anonfun$newAPIHadoopRDD$1.apply(SparkContext.scala:1094) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:108) at org.apache.spark.SparkContext.withScope(SparkContext.scala:681) at org.apache.spark.SparkContext.newAPIHadoopRDD(SparkContext.scala:1094) at org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter.convert(LoadConverter.java:88) {noformat} > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (PIG-4693) Class conflicts Kryo bundled in spark vs kryo bundled with pig
Srikanth Sundarrajan created PIG-4693: - Summary: Class conflicts Kryo bundled in spark vs kryo bundled with pig Key: PIG-4693 URL: https://issues.apache.org/jira/browse/PIG-4693 Project: Pig Issue Type: Sub-task Components: spark Affects Versions: spark-branch Reporter: Srikanth Sundarrajan Assignee: Srikanth Sundarrajan -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4693: -- Summary: Class conflicts: Kryo bundled in spark vs kryo bundled with pig (was: Class conflicts Kryo bundled in spark vs kryo bundled with pig) > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14944499#comment-14944499 ] Srikanth Sundarrajan commented on PIG-4693: --- [~xuefuz], [~mohitz], [~kexianda], Thoughts welcome. > Class conflicts: Kryo bundled in spark vs kryo bundled with pig > --- > > Key: PIG-4693 > URL: https://issues.apache.org/jira/browse/PIG-4693 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4693) Class conflicts: Kryo bundled in spark vs kryo bundled with pig
[ https://issues.apache.org/jira/browse/PIG-4693?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14944494#comment-14944494 ] Srikanth Sundarrajan commented on PIG-4693: --- Running the following simple pig script {code} IN = LOAD 'test-data' USING PigStorage(''); G = GROUP IN BY $11; R = FOREACH G GENERATE group, SUM(IN.$10); STORE R INTO 'test-out' USING PigStorage(','); {code} results in {noformat} ERROR 2998: Unhandled internal error. com.esotericsoftware.kryo.Kryo.setInstantiatorStrategy(Lorg/objenesis/strategy/InstantiatorStrategy;)V java.lang.NoSuchMethodError: com.esotericsoftware.kryo.Kryo.setInstantiatorStrategy(Lorg/objenesis/strategy/InstantiatorStrategy;)V at com.twitter.chill.KryoBase.setInstantiatorStrategy(KryoBase.scala:86) at com.twitter.chill.EmptyScalaKryoInstantiator.newKryo(ScalaKryoInstantiator.scala:59) at org.apache.spark.serializer.KryoSerializer.newKryo(KryoSerializer.scala:80) at org.apache.spark.serializer.KryoSerializerInstance.borrowKryo(KryoSerializer.scala:227) at org.apache.spark.serializer.KryoSerializerInstance.(KryoSerializer.scala:212) at org.apache.spark.serializer.KryoSerializer.newInstance(KryoSerializer.scala:128) at org.apache.spark.broadcast.TorrentBroadcast$.blockifyObject(TorrentBroadcast.scala:201) at org.apache.spark.broadcast.TorrentBroadcast.writeBlocks(TorrentBroadcast.scala:102) at org.apache.spark.broadcast.TorrentBroadcast.(TorrentBroadcast.scala:85) at org.apache.spark.broadcast.TorrentBroadcastFactory.newBroadcast(TorrentBroadcastFactory.scala:34) at org.apache.spark.broadcast.BroadcastManager.newBroadcast(BroadcastManager.scala:62) at org.apache.spark.SparkContext.broadcast(SparkContext.scala:1291) at org.apache.spark.rdd.NewHadoopRDD.(NewHadoopRDD.scala:77) at org.apache.spark.SparkContext$$anonfun$newAPIHadoopRDD$1.apply(SparkContext.scala:1099) at org.apache.spark.SparkContext$$anonfun$newAPIHadoopRDD$1.apply(SparkContext.scala:1094) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:147) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:108) at org.apache.spark.SparkContext.withScope(SparkContext.scala:681) at org.apache.spark.SparkContext.newAPIHadoopRDD(SparkContext.scala:1094) at org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter.convert(LoadConverter.java:88) at org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter.convert(LoadConverter.java:58) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:636) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:603) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:603) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:603) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:603) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.physicalToRDD(SparkLauncher.java:603) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkOperToRDD(SparkLauncher.java:555) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.sparkPlanToRDD(SparkLauncher.java:504) at org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher.launchPig(SparkLauncher.java:206) at org.apache.pig.backend.hadoop.executionengine.HExecutionEngine.launchPig(HExecutionEngine.java:301) at org.apache.pig.PigServer.launchPlan(PigServer.java:1390) at org.apache.pig.PigServer.executeCompiledLogicalPlan(PigServer.java:1375) at org.apache.pig.PigServer.execute(PigServer.java:1364) at org.apache.pig.PigServer.executeBatch(PigServer.java:415) at org.apache.pig.PigServer.executeBatch(PigServer.java:398) at org.apache.pig.tools.grunt.GruntParser.executeBatch(GruntParser.java:171) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:234) at org.apache.pig.tools.grunt.GruntParser.parseStopOnError(GruntParser.java:205) at org.apache.pig.tools.grunt.Grunt.exec(Grunt.java:81) at org.apache.pig.Main.run(Main.java:624) at org.apache.pig.Main.main(Main.java:170) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606
[jira] [Commented] (PIG-4236) Avoid packaging spark specific jars into pig fat jar
[ https://issues.apache.org/jira/browse/PIG-4236?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14909169#comment-14909169 ] Srikanth Sundarrajan commented on PIG-4236: --- Am assuming PIG-4667 has already addressed this by shipping spark specific jars for execution and avoids bundling this in pig uber jar. [~praveenr019], [~xuefuz], Do you think there is anything left to be done on this ? > Avoid packaging spark specific jars into pig fat jar > > > Key: PIG-4236 > URL: https://issues.apache.org/jira/browse/PIG-4236 > Project: Pig > Issue Type: Sub-task > Components: spark >Reporter: Praveen Rachabattuni > Fix For: spark-branch > > > Spark and its dependencies jars shouldn't be packaged into the pig jar as it > would be redundant to do so. > Pig dependencies should be either added from lib directory or use the legacy > jar which contains pig along with its dependencies. > Please refer to PIG-4047 for more info. -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4681) Enable Pig on Spark to run on Yarn Cluster mode
[ https://issues.apache.org/jira/browse/PIG-4681?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14909168#comment-14909168 ] Srikanth Sundarrajan commented on PIG-4681: --- In the cluster mode the Spark Context will be created on the AM and all the pig execution need to move to the AM. This requires us to initialize pigContext on the AM as well. Given that pig has always had a thick client which controlled the execution from this client, am inclined to mark this as "Wont Fix". I intend to add necessary notes in Docs/FAQs as a patch as part of this JIRA for future reference. [~xuefuz], What are your thoughts? > Enable Pig on Spark to run on Yarn Cluster mode > --- > > Key: PIG-4681 > URL: https://issues.apache.org/jira/browse/PIG-4681 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan >Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4680) Enable pig job graphs to resume from last successful state
[ https://issues.apache.org/jira/browse/PIG-4680?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14802876#comment-14802876 ] Srikanth Sundarrajan commented on PIG-4680: --- This can be quite handy, particularly when pig scripts is launched via oozie and if the launcher were to fail and attempt is retried. > Enable pig job graphs to resume from last successful state > -- > > Key: PIG-4680 > URL: https://issues.apache.org/jira/browse/PIG-4680 > Project: Pig > Issue Type: Improvement > Components: impl >Reporter: Abhishek Agarwal >Assignee: Abhishek Agarwal > > Pig scripts can have multiple ETL jobs in the DAG which may take hours to > finish. In case of transient errors, the job fails. When the job is rerun, > all the nodes in Job graph will rerun. Some of these nodes may have already > run successfully. Redundant runs lead to wastage of cluster capacity and > pipeline delays. > In case of failure, we can persist the graph state. In next run, only the > failed nodes and their successors will rerun. This is of course subject to > preconditions such as > - Pig script has not changed > - Input locations have not changed > - Output data from previous run is intact > - Configuration has not changed -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4680) Enable pig job graphs to resume from last successful state
[ https://issues.apache.org/jira/browse/PIG-4680?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4680: -- Assignee: Abhishek Agarwal > Enable pig job graphs to resume from last successful state > -- > > Key: PIG-4680 > URL: https://issues.apache.org/jira/browse/PIG-4680 > Project: Pig > Issue Type: Improvement > Components: impl >Reporter: Abhishek Agarwal >Assignee: Abhishek Agarwal > > Pig scripts can have multiple ETL jobs in the DAG which may take hours to > finish. In case of transient errors, the job fails. When the job is rerun, > all the nodes in Job graph will rerun. Some of these nodes may have already > run successfully. Redundant runs lead to wastage of cluster capacity and > pipeline delays. > In case of failure, we can persist the graph state. In next run, only the > failed nodes and their successors will rerun. This is of course subject to > preconditions such as > - Pig script has not changed > - Input locations have not changed > - Output data from previous run is intact > - Configuration has not changed -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Created] (PIG-4681) Enable Pig on Spark to run on Yarn Cluster mode
Srikanth Sundarrajan created PIG-4681: - Summary: Enable Pig on Spark to run on Yarn Cluster mode Key: PIG-4681 URL: https://issues.apache.org/jira/browse/PIG-4681 Project: Pig Issue Type: Sub-task Components: spark Affects Versions: spark-branch Reporter: Srikanth Sundarrajan Assignee: Srikanth Sundarrajan Fix For: spark-branch -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4681) Enable Pig on Spark to run on Yarn Cluster mode
[ https://issues.apache.org/jira/browse/PIG-4681?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4681: -- Labels: spork (was: spark) > Enable Pig on Spark to run on Yarn Cluster mode > --- > > Key: PIG-4681 > URL: https://issues.apache.org/jira/browse/PIG-4681 > Project: Pig > Issue Type: Sub-task > Components: spark >Affects Versions: spark-branch > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Labels: spork > Fix For: spark-branch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Re: Review Request 38352: Enable Pig on Spark to run on Yarn Client mode
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/38352/ --- (Updated Sept. 16, 2015, 12:06 p.m.) Review request for pig. Bugs: PIG-4667 https://issues.apache.org/jira/browse/PIG-4667 Repository: pig-git Description --- Enable Pig on Spark to run on Yarn Client mode Diffs (updated) - bin/pig 15341d1 build.xml b17b0e1 ivy.xml 2ebebdc ivy/libraries.properties 4d1f61e src/docs/src/documentation/content/xdocs/start.xml 97d3a4d Diff: https://reviews.apache.org/r/38352/diff/ Testing --- Script used for testing A = LOAD '/tmp/x' USING PigStorage('\t') AS (line); STORE A INTO '/tmp/y' USING PigStorage(','); Used the following environment setting before launching the script. declare -x HADOOP_CONF_DIR="/opt/hadoop-2.6.0.2.2.0.0-2041/etc/hadoop/" declare -x HADOOP_HOME="/opt/hadoop-2.6.0.2.2.0.0-2041/" declare -x SPARK_MASTER="yarn-client" Thanks, Srikanth Sundarrajan
[jira] [Updated] (PIG-4667) Enable Pig on Spark to run on Yarn Client mode
[ https://issues.apache.org/jira/browse/PIG-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4667: -- Attachment: PIG-4667-v1.patch Hi [~xuefuz], Have attached a revised patch which removes changes to Kryo and Guava lib versions. Have verified that yarn-client works with these changes. > Enable Pig on Spark to run on Yarn Client mode > -- > > Key: PIG-4667 > URL: https://issues.apache.org/jira/browse/PIG-4667 > Project: Pig > Issue Type: Sub-task > Components: spark > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > Attachments: PIG-4667-logs.tgz, PIG-4667-v1.patch, PIG-4667.patch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
Review Request 38352: Enable Pig on Spark to run on Yarn Client mode
--- This is an automatically generated e-mail. To reply, visit: https://reviews.apache.org/r/38352/ --- Review request for pig. Bugs: PIG-4667 https://issues.apache.org/jira/browse/PIG-4667 Repository: pig-git Description --- Enable Pig on Spark to run on Yarn Client mode Diffs - bin/pig 15341d1 build.xml b17b0e1 ivy.xml 2ebebdc ivy/libraries.properties 4d1f61e src/docs/src/documentation/content/xdocs/start.xml 97d3a4d Diff: https://reviews.apache.org/r/38352/diff/ Testing --- Script used for testing A = LOAD '/tmp/x' USING PigStorage('\t') AS (line); STORE A INTO '/tmp/y' USING PigStorage(','); Used the following environment setting before launching the script. declare -x HADOOP_CONF_DIR="/opt/hadoop-2.6.0.2.2.0.0-2041/etc/hadoop/" declare -x HADOOP_HOME="/opt/hadoop-2.6.0.2.2.0.0-2041/" declare -x SPARK_MASTER="yarn-client" Thanks, Srikanth Sundarrajan
[jira] [Updated] (PIG-4667) Enable Pig on Spark to run on Yarn Client mode
[ https://issues.apache.org/jira/browse/PIG-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4667: -- Summary: Enable Pig on Spark to run on Yarn Client mode (was: Enable Pig on Spark to run on Yarn Client/Cluster mode) > Enable Pig on Spark to run on Yarn Client mode > -- > > Key: PIG-4667 > URL: https://issues.apache.org/jira/browse/PIG-4667 > Project: Pig > Issue Type: Sub-task > Components: spark > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > Attachments: PIG-4667-logs.tgz > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4667) Enable Pig on Spark to run on Yarn Client mode
[ https://issues.apache.org/jira/browse/PIG-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4667: -- Status: Patch Available (was: Open) > Enable Pig on Spark to run on Yarn Client mode > -- > > Key: PIG-4667 > URL: https://issues.apache.org/jira/browse/PIG-4667 > Project: Pig > Issue Type: Sub-task > Components: spark > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > Attachments: PIG-4667-logs.tgz, PIG-4667.patch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Updated] (PIG-4667) Enable Pig on Spark to run on Yarn Client mode
[ https://issues.apache.org/jira/browse/PIG-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Srikanth Sundarrajan updated PIG-4667: -- Attachment: PIG-4667.patch Was able to use SPARK_DIST_CLASSPATH env variable to set the required classpath elements. Patch includes changes to the build scripts, startup scripts and docs. > Enable Pig on Spark to run on Yarn Client mode > -- > > Key: PIG-4667 > URL: https://issues.apache.org/jira/browse/PIG-4667 > Project: Pig > Issue Type: Sub-task > Components: spark > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > Attachments: PIG-4667-logs.tgz, PIG-4667.patch > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4667) Enable Pig on Spark to run on Yarn Client/Cluster mode
[ https://issues.apache.org/jira/browse/PIG-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14734656#comment-14734656 ] Srikanth Sundarrajan commented on PIG-4667: --- [~xuefuz], Jars that are added to the spark context are available to the executors and in yarn-client mode the driver is running within the same JVM as pig and we have issues with neither of this. The issue really is in making all the spark-libs available for the AM (which is invoking ExecutorLauncher in yarn-client mode). If we dont have the assembly, spark code simply ships spark-yarn jar (which is the jarOf(Client)), while spark-core and other dependent libs such as scala, akka etc doesn't get shipped. ClientArguments() class in spark-yarn module allows for additional jars to be added to dist-cache and the AM classpath, however when we create a SparkContext, there doesn't seem to be any way to pass these jars. Tried adding them to --files, though they are added to dist-cache and are localized, they are not part of the classpath. Here are the options that I am currently considering. 1. Create a maven pom to create a shaded assembly jar and then use them 2. Try using ant tasks to re-create shaded assembly similar to what spark-assembly module 3. Allow users to specify SPARK_HOME and the wire up bin/pig to use the artifacts from the SPARK_HOME, without which spark version will work with local mode. I am inclined to go with option #3, as it is clean and allows for us to keep in line with changes that might happen in spark dependencies/packaging. Would like to hear your thoughts. > Enable Pig on Spark to run on Yarn Client/Cluster mode > -- > > Key: PIG-4667 > URL: https://issues.apache.org/jira/browse/PIG-4667 > Project: Pig > Issue Type: Sub-task > Components: spark > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > Attachments: PIG-4667-logs.tgz > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)
[jira] [Commented] (PIG-4667) Enable Pig on Spark to run on Yarn Client/Cluster mode
[ https://issues.apache.org/jira/browse/PIG-4667?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=14733564#comment-14733564 ] Srikanth Sundarrajan commented on PIG-4667: --- Here is a quick update on this It seems a bit tricky to get multiple jars passed to ExecutorLancher (Yarn-AM) except for spark jar and an app jar. Am trying to seek help from the spark community to see if there is a way to ship additional jars that needs to be on AM class path. (There is a way to add files to dist cache, but they dont seem to be added automatically to the classpath) Easy workaround for this is to work with spark-assembly.jar, however the same isn't available on maven repo for the pig code base to work with by default. Am concurrently trying to see if it is possible to download the spark dependencies and create a shaded jar to work with. > Enable Pig on Spark to run on Yarn Client/Cluster mode > -- > > Key: PIG-4667 > URL: https://issues.apache.org/jira/browse/PIG-4667 > Project: Pig > Issue Type: Sub-task > Components: spark > Reporter: Srikanth Sundarrajan > Assignee: Srikanth Sundarrajan > Fix For: spark-branch > > Attachments: PIG-4667-logs.tgz > > -- This message was sent by Atlassian JIRA (v6.3.4#6332)