PIG-4059: Pig On Spark

git-svn-id: https://svn.apache.org/repos/asf/pig/trunk@1796639 13f79535-47bb-0310-9956-ffa450edef68
diff --git a/CHANGES.txt b/CHANGES.txt
index 9ce0648..2b6c257 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -36,6 +36,8 @@
  
 IMPROVEMENTS
 
+PIG-4059: Pig On Spark
+
 PIG-5188: Review pig-index.xml (szita)
 
 PIG-4924: Translate failures.maxpercent MR setting to Tez Tez (rohini)
diff --git a/bin/pig b/bin/pig
index e1212fa..bfb0998 100755
--- a/bin/pig
+++ b/bin/pig
@@ -57,6 +57,21 @@
 includeHCatalog="";
 addJarString=-Dpig.additional.jars.uris\=;
 additionalJars="";
+prevArgExecType=false;
+isSparkMode=false;
+isSparkLocalMode=false;
+
+#verify the execType is SPARK or SPARK_LOCAL or not
+function processExecType(){
+    execType=$1
+    execTypeUpperCase=$(echo $execType |tr [a-z] [A-Z])
+    if [[ "$execTypeUpperCase" == "SPARK" ]]; then
+       isSparkMode=true
+    elif [[ "$execTypeUpperCase" == "SPARK_LOCAL" ]]; then
+       isSparkLocalMode=true
+    fi
+}
+
 # filter command line parameter
 for f in "$@"; do
      if [[ $f == "-secretDebugCmd" || $f == "-printCmdDebug" ]]; then
@@ -70,6 +85,13 @@
         includeHCatalog=true;
       elif [[ "$includeHCatalog" == "true" && $f == $addJarString* ]]; then
         additionalJars=`echo $f | sed s/$addJarString//`
+      elif [[ "$f" == "-x" || "$f" == "-exectype" ]]; then
+        prevArgExecType=true;
+        remaining[${#remaining[@]}]="$f"
+      elif [[ "$prevArgExecType" == "true" ]]; then
+        prevArgExecType=false;
+        processExecType $f
+        remaining[${#remaining[@]}]="$f"
       else
         remaining[${#remaining[@]}]="$f"
      fi
@@ -362,6 +384,44 @@
   PIG_OPTS="$PIG_OPTS -Dpig.additional.jars.uris=$ADDITIONAL_CLASSPATHS"
 fi
 
+################# ADDING SPARK DEPENDENCIES ##################
+# For spark_local mode:
+if [ "$isSparkLocalMode" == "true" ]; then
+#SPARK_MASTER is forced to be "local" in spark_local mode
+        SPARK_MASTER="local"
+    for f in $PIG_HOME/lib/spark/*.jar; do
+            CLASSPATH=${CLASSPATH}:$f;
+    done
+fi
+
+# For spark mode:
+# Please specify SPARK_HOME first so that we can locate $SPARK_HOME/lib/spark-assembly*.jar,
+# we will add spark-assembly*.jar to the classpath.
+if [ "$isSparkMode"  == "true" ]; then
+    if [ -z "$SPARK_HOME" ]; then
+       echo "Error: SPARK_HOME is not set!"
+       exit 1
+    fi
+
+    # Please specify SPARK_JAR which is the hdfs path of spark-assembly*.jar to allow YARN to cache spark-assembly*.jar on nodes so that it doesn't need to be distributed each time an application runs.
+    if [ -z "$SPARK_JAR" ]; then
+       echo "Error: SPARK_JAR is not set, SPARK_JAR stands for the hdfs location of spark-assembly*.jar. This allows YARN to cache spark-assembly*.jar on nodes so that it doesn't need to be distributed each time an application runs."
+       exit 1
+    fi
+
+    if [ -n "$SPARK_HOME" ]; then
+        echo "Using Spark Home: " ${SPARK_HOME}
+        SPARK_ASSEMBLY_JAR=`ls ${SPARK_HOME}/lib/spark-assembly*`
+        CLASSPATH=${CLASSPATH}:$SPARK_ASSEMBLY_JAR
+    fi
+fi
+
+#spark-assembly.jar contains jcl-over-slf4j which would create a LogFactory implementation that is incompatible
+if [ "$isSparkMode"  == "true" ]; then
+    PIG_OPTS="$PIG_OPTS -Dorg.apache.commons.logging.LogFactory=org.apache.commons.logging.impl.LogFactoryImpl"
+fi
+################# ADDING SPARK DEPENDENCIES ##################
+
 # run it
 if [ -n "$HADOOP_BIN" ]; then
     if [ "$debug" == "true" ]; then
diff --git a/build.xml b/build.xml
index 307b863..9255ad0 100644
--- a/build.xml
+++ b/build.xml
@@ -46,6 +46,7 @@
 
     <!-- source properties -->
     <property name="lib.dir" value="${basedir}/lib" />
+    <property name="spark.lib.dir" value="${basedir}/lib/spark" />
     <property name="src.dir" value="${basedir}/src" />
     <property name="python.src.dir" value="${src.dir}/python" />
     <property name="src.lib.dir" value="${basedir}/lib-src" />
@@ -106,9 +107,12 @@
     <property name="test.unit.file" value="${test.src.dir}/unit-tests"/>
     <property name="test.smoke.file" value="${test.src.dir}/smoke-tests"/>
     <property name="test.all.file" value="${test.src.dir}/all-tests"/>
+    <property name="test.spark.file" value="${test.src.dir}/spark-tests"/>
+    <property name="test.spark_local.file" value="${test.src.dir}/spark-local-tests"/>
     <property name="test.exclude.file" value="${test.src.dir}/excluded-tests"/>
     <property name="test.exclude.file.mr" value="${test.src.dir}/excluded-tests-mr"/>
     <property name="test.exclude.file.tez" value="${test.src.dir}/excluded-tests-tez"/>
+    <property name="test.exclude.file.spark" value="${test.src.dir}/excluded-tests-spark"/>
     <property name="pigunit.jarfile" value="pigunit.jar" />
     <property name="piggybank.jarfile" value="${basedir}/contrib/piggybank/java/piggybank.jar" />
     <property name="smoke.tests.jarfile" value="${build.dir}/${final.name}-smoketests.jar" />
@@ -160,6 +164,10 @@
         <propertyreset name="test.exec.type" value="tez" />
     </target>
 
+    <target name="setSparkEnv">
+      <propertyreset name="test.exec.type" value="spark" />
+    </target>
+
     <target name="setWindowsPath" if="${isWindows}">
       <property name="build.path" value="${env.Path};${hadoop.root}\bin" />
     </target>
@@ -253,6 +261,7 @@
     <property name="build.ivy.dir" location="${build.dir}/ivy" />
     <property name="build.ivy.lib.dir" location="${build.ivy.dir}/lib" />
     <property name="ivy.lib.dir" location="${build.ivy.lib.dir}/${ant.project.name}"/>
+    <property name="ivy.lib.dir.spark" location="${ivy.lib.dir}/spark" />
     <property name="build.ivy.report.dir" location="${build.ivy.dir}/report" />
     <property name="build.ivy.maven.dir" location="${build.ivy.dir}/maven" />
     <property name="pom.xml" location="${build.ivy.maven.dir}/pom.xml"/>
@@ -322,6 +331,9 @@
             <fileset dir="${ivy.lib.dir}">
                 <include name="**.*jar"/>
             </fileset>
+            <fileset dir="${ivy.lib.dir.spark}">
+                <include name="**.*jar"/>
+            </fileset>
         </path>
         <taskdef name="eclipse"
                  classname="prantl.ant.eclipse.EclipseTask"
@@ -352,6 +364,7 @@
     <path id="classpath">
         <fileset file="${ivy.lib.dir}/${zookeeper.jarfile}"/>
         <fileset dir="${ivy.lib.dir}" includes="*.jar"/>
+        <fileset dir="${ivy.lib.dir.spark}" includes="*.jar"/>
     </path>
 
     <!-- javadoc-classpath -->
@@ -370,6 +383,7 @@
 
     <fileset dir="${ivy.lib.dir}" id="core.dependencies.jar">
         <exclude name="**.*jar"/>
+        <exclude name="spark/**.*jar"/>
     </fileset>
 
     <fileset dir="${ivy.lib.dir}" id="runtime.dependencies-withouthadoop.jar">
@@ -678,6 +692,7 @@
         <buildJar svnString="${svn.revision}" outputFile="${output.jarfile.core}" includedJars="core.dependencies.jar"/>
         <buildJar svnString="${svn.revision}" outputFile="${output.jarfile.withouthadoop}" includedJars="runtime.dependencies-withouthadoop.jar"/>
         <antcall target="copyCommonDependencies"/>
+        <antcall target="copySparkDependencies"/>
         <antcall target="copyh2Dependencies"/>
         <antcall target="copyHadoop2LocalRuntimeDependencies" />
     </target>
@@ -715,9 +730,17 @@
             <fileset dir="${ivy.lib.dir}" includes="httpdlog-*-${basjes-httpdlog-pigloader.version}.jar"/>
             <fileset dir="${ivy.lib.dir}" includes="parser-core-${basjes-httpdlog-pigloader.version}.jar"/>
             <fileset dir="${ivy.lib.dir}" includes="ivy-*.jar"/>
+            <fileset dir="${ivy.lib.dir}" includes="commons-logging-*.jar"/>
         </copy>
     </target>
 
+    <target name="copySparkDependencies">
+        <mkdir dir="${spark.lib.dir}" />
+        <copy todir="${spark.lib.dir}">
+            <fileset dir="${ivy.lib.dir.spark}" includes="*.jar"/>
+        </copy>
+    </target>
+    
     <target name="copyh2Dependencies" if="isHadoop2">
         <mkdir dir="${lib.dir}/h2" />
         <copy todir="${lib.dir}/h2">
@@ -856,7 +879,12 @@
         <macro-test-runner test.file="${test.all.file}" tests.failed="test-tez.failed"/>
         <fail if="test-tez.failed">Tests failed!</fail>
     </target>
-	
+
+    <target name="test-spark" depends="setSparkEnv,setWindowsPath,setLinuxPath,compile-test,jar,debugger.check,jackson-pig-3039-test-download" description="Run Spark unit tests in Spark cluster-local mode">
+        <macro-test-runner test.file="${test.all.file}" tests.failed="test-spark.failed"/>
+        <fail if="test-spark.failed">Tests failed!</fail>
+    </target>
+
     <target name="debugger.check" depends="debugger.set,debugger.unset"/>
     <target name="debugger.set" if="debugPort">
         <property name="debugArgs" value="-Xdebug -Xrunjdwp:transport=dt_socket,server=y,suspend=y,address=${debugPort}"/>
@@ -881,9 +909,6 @@
             <sysproperty key="test.exec.type" value="${test.exec.type}" />
             <sysproperty key="ssh.gateway" value="${ssh.gateway}" />
             <sysproperty key="hod.server" value="${hod.server}" />
-            <sysproperty key="build.classes" value="${build.classes}" />
-            <sysproperty key="test.build.classes" value="${test.build.classes}" />
-            <sysproperty key="ivy.lib.dir" value="${ivy.lib.dir}" />
             <sysproperty key="java.io.tmpdir" value="${junit.tmp.dir}" />
             <sysproperty key="hadoop.log.dir" value="${test.log.dir}"/>
             <jvmarg line="-XX:+CMSClassUnloadingEnabled -XX:MaxPermSize=128M ${debugArgs} -Djava.library.path=${hadoop.root}\bin"/>
@@ -978,6 +1003,10 @@
         <ant dir="${test.e2e.dir}" target="test-tez"/>
     </target>
 
+    <target name="test-e2e-spark" depends="jar, piggybank" description="run end-to-end tests in spark mode">
+            <ant dir="${test.e2e.dir}" target="test-spark"/>
+    </target>
+
     <target name="test-e2e-deploy" depends="jar" description="deploy end-to-end tests to existing cluster">
         <ant dir="${test.e2e.dir}" target="deploy"/>
     </target>
@@ -1624,6 +1653,8 @@
      <target name="ivy-compile" depends="ivy-resolve" description="Retrieve Ivy-managed artifacts for compile configuration">
        <ivy:retrieve settingsRef="${ant.project.name}.ivy.settings" log="${loglevel}"
                  pattern="${build.ivy.lib.dir}/${ivy.artifact.retrieve.pattern}" conf="compile"/>
+       <ivy:retrieve settingsRef="${ant.project.name}.ivy.settings" log="${loglevel}"
+                 pattern="${ivy.lib.dir.spark}/[artifact]-[revision](-[classifier]).[ext]" conf="spark"/>
        <ivy:cachepath pathid="compile.classpath" conf="compile"/>
      </target>
 
diff --git a/ivy.xml b/ivy.xml
index 1dc8122..3f2c943 100644
--- a/ivy.xml
+++ b/ivy.xml
@@ -40,6 +40,7 @@
     <conf name="buildJar" extends="compile,test" visibility="private"/>
     <conf name="hadoop2" visibility="private"/>
     <conf name="hbase1" visibility="private"/>
+    <conf name="spark" visibility="private" />
   </configurations>
   <publications>
     <artifact name="pig" conf="master"/>
@@ -406,6 +407,24 @@
 
     <dependency org="com.twitter" name="parquet-pig-bundle" rev="${parquet-pig-bundle.version}" conf="compile->master"/>
 
+    <!-- for Spark integration -->
+    <dependency org="org.apache.spark" name="spark-core_2.11" rev="${spark.version}" conf="spark->default">
+        <exclude org="org.eclipse.jetty.orbit" module="javax.servlet"/>
+        <exclude org="org.eclipse.jetty.orbit" module="javax.transaction"/>
+        <exclude org="org.eclipse.jetty.orbit" module="javax.mail.glassfish"/>
+        <exclude org="org.eclipse.jetty.orbit" module="javax.activation"/>
+        <exclude org="org.apache.hadoop" />
+        <exclude org="com.esotericsoftware.kryo" />
+        <exclude org="jline" module="jline"/>
+        <exclude org="com.google.guava" />
+    </dependency>
+    <dependency org="org.apache.spark" name="spark-yarn_2.11" rev="${spark.version}" conf="spark->default">
+        <exclude org="org.apache.hadoop" />
+    </dependency>
+    <dependency org="asm" name="asm" rev="${asm.version}" conf="compile->master"/>
+    <dependency org="javax.servlet" name="javax.servlet-api" rev="3.0.1" conf="spark->default"/>
+    <dependency org="org.scala-lang.modules" name="scala-xml_2.11" rev="${scala-xml.version}" conf="spark->default"/>
+
     <!-- for Tez integration -->
     <dependency org="org.apache.tez" name="tez" rev="${tez.version}"
        conf="hadoop2->master"/>
diff --git a/ivy/libraries.properties b/ivy/libraries.properties
index 46b71ba..a0eb00a 100644
--- a/ivy/libraries.properties
+++ b/ivy/libraries.properties
@@ -73,6 +73,7 @@
 rats-lib.version=0.5.1
 slf4j-api.version=1.6.1
 slf4j-log4j12.version=1.6.1
+spark.version=1.6.1
 xerces.version=2.10.0
 xalan.version=2.7.1
 wagon-http.version=1.0-beta-2
@@ -88,7 +89,7 @@
 mockito.version=1.8.4
 jansi.version=1.9
 asm.version=3.3.1
-snappy-java.version=1.1.0.1
+snappy-java.version=1.1.1.3
 tez.version=0.7.0
 parquet-pig-bundle.version=1.2.3
 snappy.version=0.2
@@ -96,3 +97,4 @@
 curator.version=2.6.0
 htrace.version=3.1.0-incubating
 commons-lang3.version=3.1
+scala-xml.version=1.0.5
diff --git a/src/META-INF/services/org.apache.pig.ExecType b/src/META-INF/services/org.apache.pig.ExecType
index 5c034c8..1ad2569 100644
--- a/src/META-INF/services/org.apache.pig.ExecType
+++ b/src/META-INF/services/org.apache.pig.ExecType
@@ -15,4 +15,5 @@
 org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRExecType
 org.apache.pig.backend.hadoop.executionengine.tez.TezLocalExecType
 org.apache.pig.backend.hadoop.executionengine.tez.TezExecType
-
+org.apache.pig.backend.hadoop.executionengine.spark.SparkExecType
+org.apache.pig.backend.hadoop.executionengine.spark.SparkLocalExecType
diff --git a/src/docs/src/documentation/content/xdocs/start.xml b/src/docs/src/documentation/content/xdocs/start.xml
index c9a1491..aee6e2d 100644
--- a/src/docs/src/documentation/content/xdocs/start.xml
+++ b/src/docs/src/documentation/content/xdocs/start.xml
@@ -26,45 +26,45 @@
 
 <!-- SET UP PIG -->
  <section>
-		<title>Pig Setup</title>
-	
+        <title>Pig Setup</title>
+    
 <!-- ++++++++++++++++++++++++++++++++++ -->
  <section id="req">
  <title>Requirements</title>
  <p><strong>Mandatory</strong></p>
       <p>Unix and Windows users need the following:</p>
-		<ul>
-		  <li> <strong>Hadoop 2.X</strong> - <a href="http://hadoop.apache.org/common/releases.html">http://hadoop.apache.org/common/releases.html</a> (You can run Pig with different versions of Hadoop by setting HADOOP_HOME to point to the directory where you have installed Hadoop. If you do not set HADOOP_HOME, by default Pig will run with the embedded version, currently Hadoop 2.7.3.)</li>
-		  <li> <strong>Java 1.7</strong> - <a href="http://java.sun.com/javase/downloads/index.jsp">http://java.sun.com/javase/downloads/index.jsp</a> (set JAVA_HOME to the root of your Java installation)</li>	
-		</ul>
-		<p></p>
+        <ul>
+          <li> <strong>Hadoop 2.X</strong> - <a href="http://hadoop.apache.org/common/releases.html">http://hadoop.apache.org/common/releases.html</a> (You can run Pig with different versions of Hadoop by setting HADOOP_HOME to point to the directory where you have installed Hadoop. If you do not set HADOOP_HOME, by default Pig will run with the embedded version, currently Hadoop 2.7.3.)</li>
+          <li> <strong>Java 1.7</strong> - <a href="http://java.sun.com/javase/downloads/index.jsp">http://java.sun.com/javase/downloads/index.jsp</a> (set JAVA_HOME to the root of your Java installation)</li>    
+        </ul>
+        <p></p>
  <p><strong>Optional</strong></p>
- 		<ul>
+         <ul>
           <li> <strong>Python 2.7</strong> - <a href="http://jython.org/downloads.html">https://www.python.org</a> (when using Streaming Python UDFs) </li>
           <li> <strong>Ant 1.8</strong> - <a href="http://ant.apache.org/">http://ant.apache.org/</a> (for builds) </li>
-		</ul>
+        </ul>
  
   </section>         
    
 <!-- ++++++++++++++++++++++++++++++++++ -->        
  <section id="download">
  <title>Download Pig</title>
-	<p>To get a Pig distribution, do the following:</p>
-	
-	<ol>
-	<li>Download a recent stable release from one of the Apache Download Mirrors 
-	(see <a href="http://hadoop.apache.org/pig/releases.html"> Pig Releases</a>).</li>
-	
+    <p>To get a Pig distribution, do the following:</p>
+    
+    <ol>
+    <li>Download a recent stable release from one of the Apache Download Mirrors 
+    (see <a href="http://hadoop.apache.org/pig/releases.html"> Pig Releases</a>).</li>
+    
     <li>Unpack the downloaded Pig distribution, and then note the following:
-	    <ul>
-	    <li>The Pig script file, pig, is located in the bin directory (/pig-n.n.n/bin/pig). 
-	    The Pig environment variables are described in the Pig script file.</li>
-	    <li>The Pig properties file, pig.properties, is located in the conf directory (/pig-n.n.n/conf/pig.properties). 
-	    You can specify an alternate location using the PIG_CONF_DIR environment variable.</li>
-	</ul>	
-	</li>
-	<li>Add /pig-n.n.n/bin to your path. Use export (bash,sh,ksh) or setenv (tcsh,csh). For example: <br></br>
-	<code>$ export PATH=/&lt;my-path-to-pig&gt;/pig-n.n.n/bin:$PATH</code>
+        <ul>
+        <li>The Pig script file, pig, is located in the bin directory (/pig-n.n.n/bin/pig). 
+        The Pig environment variables are described in the Pig script file.</li>
+        <li>The Pig properties file, pig.properties, is located in the conf directory (/pig-n.n.n/conf/pig.properties). 
+        You can specify an alternate location using the PIG_CONF_DIR environment variable.</li>
+    </ul>    
+    </li>
+    <li>Add /pig-n.n.n/bin to your path. Use export (bash,sh,ksh) or setenv (tcsh,csh). For example: <br></br>
+    <code>$ export PATH=/&lt;my-path-to-pig&gt;/pig-n.n.n/bin:$PATH</code>
 </li>
 <li>
 Test the Pig installation with this simple command: <code>$ pig -help</code>
@@ -78,10 +78,10 @@
 <title>Build Pig</title>
       <p>To build pig, do the following:</p>
      <ol>
-	  <li> Check out the Pig code from SVN: <code>svn co http://svn.apache.org/repos/asf/pig/trunk</code> </li>
-	  <li> Build the code from the top directory: <code>ant</code> <br></br>
-	  If the build is successful, you should see the pig.jar file created in that directory. </li>	
-	  <li> Validate the pig.jar  by running a unit test: <code>ant test</code></li>
+      <li> Check out the Pig code from SVN: <code>svn co http://svn.apache.org/repos/asf/pig/trunk</code> </li>
+      <li> Build the code from the top directory: <code>ant</code> <br></br>
+      If the build is successful, you should see the pig.jar file created in that directory. </li>    
+      <li> Validate the pig.jar  by running a unit test: <code>ant test</code></li>
      </ol>
  </section>
 </section>
@@ -90,46 +90,53 @@
     
    <!-- RUNNING PIG  -->
    <section id="run">
-	<title>Running Pig </title> 
-	<p>You can run Pig (execute Pig Latin statements and Pig commands) using various modes.</p>
-	<table>
-	<tr>
-	<td></td>
+    <title>Running Pig </title>
+    <p>You can run Pig (execute Pig Latin statements and Pig commands) using various modes.</p>
+    <table>
+    <tr>
+    <td></td>
     <td><strong>Local Mode</strong></td>
     <td><strong>Tez Local Mode</strong></td>
+    <td><strong>Spark Local Mode</strong></td>
     <td><strong>Mapreduce Mode</strong></td>
     <td><strong>Tez Mode</strong></td>
-	</tr>
-	<tr>
-	<td><strong>Interactive Mode </strong></td>
+    <td><strong>Spark Mode</strong></td>
+    </tr>
+    <tr>
+    <td><strong>Interactive Mode </strong></td>
     <td>yes</td>
     <td>experimental</td>
     <td>yes</td>
     <td>yes</td>
-	</tr>
-	<tr>
-	<td><strong>Batch Mode</strong> </td>
+    </tr>
+    <tr>
+    <td><strong>Batch Mode</strong> </td>
     <td>yes</td>
     <td>experimental</td>
     <td>yes</td>
     <td>yes</td>
-	</tr>
-	</table>
-	
-	<!-- ++++++++++++++++++++++++++++++++++ -->
-	   <section id="execution-modes">
-	<title>Execution Modes</title> 
-<p>Pig has two execution modes or exectypes: </p>
+    </tr>
+    </table>
+
+    <!-- ++++++++++++++++++++++++++++++++++ -->
+       <section id="execution-modes">
+    <title>Execution Modes</title>
+<p>Pig has six execution modes or exectypes: </p>
 <ul>
 <li><strong>Local Mode</strong> - To run Pig in local mode, you need access to a single machine; all files are installed and run using your local host and file system. Specify local mode using the -x flag (pig -x local).
 </li>
 <li><strong>Tez Local Mode</strong> - To run Pig in tez local mode. It is similar to local mode, except internally Pig will invoke tez runtime engine. Specify Tez local mode using the -x flag (pig -x tez_local).
 <p><strong>Note:</strong> Tez local mode is experimental. There are some queries which just error out on bigger data in local mode.</p>
 </li>
+<li><strong>Spark Local Mode</strong> - To run Pig in spark local mode. It is similar to local mode, except internally Pig will invoke spark runtime engine. Specify Spark local mode using the -x flag (pig -x spark_local).
+<p><strong>Note:</strong> Spark local mode is experimental. There are some queries which just error out on bigger data in local mode.</p>
+</li>
 <li><strong>Mapreduce Mode</strong> - To run Pig in mapreduce mode, you need access to a Hadoop cluster and HDFS installation. Mapreduce mode is the default mode; you can, <em>but don't need to</em>, specify it using the -x flag (pig OR pig -x mapreduce).
 </li>
 <li><strong>Tez Mode</strong> - To run Pig in Tez mode, you need access to a Hadoop cluster and HDFS installation. Specify Tez mode using the -x flag (-x tez).
 </li>
+<li><strong>Spark Mode</strong> - To run Pig in Spark mode, you need access to a Spark, Yarn or Mesos cluster and HDFS installation. Specify Spark mode using the -x flag (-x spark). In Spark execution mode, it is necessary to set env::SPARK_MASTER to an appropriate value (local - local mode, yarn-client - yarn-client mode, mesos://host:port - spark on mesos or spark://host:port - spark cluster. For more information refer to spark documentation on Master URLs, <em>yarn-cluster mode is currently not supported</em>). Pig scripts run on Spark can take advantage of the <a href="http://spark.apache.org/docs/latest/job-scheduling.html#dynamic-resource-allocation">dynamic allocation</a> feature. The feature can be enabled by simply enabling <em>spark.dynamicAllocation.enabled</em>. Refer to spark <a href="http://spark.apache.org/docs/latest/configuration.html#dynamic-allocation">configuration</a> for additional configuration details. In general all properties in the pig script prefixed with <em>spark.</em> are copied to the Spark Application Configuration. Please note that Yarn auxillary service need to be enabled on Spark for this to work. See Spark documentation for additional details.
+</li>
 </ul>
 <p></p>
 
@@ -148,6 +155,9 @@
 /* Tez local mode */
 $ pig -x tez_local ...
  
+/* Spark local mode */
+$ pig -x spark_local ...
+
 /* mapreduce mode */
 $ pig ...
 or
@@ -155,6 +165,9 @@
 
 /* Tez mode */
 $ pig -x tez ...
+
+/* Spark mode */
+$ pig -x spark ...
 </source>
 
 </section>
@@ -179,7 +192,7 @@
 <source>
 $ pig -x local
 ... - Connecting to ...
-grunt> 
+grunt>
 </source>
 
 <p><strong>Tez Local Mode</strong></p>
@@ -189,6 +202,13 @@
 grunt> 
 </source>
 
+<p><strong>Spark Local Mode</strong></p>
+<source>
+$ pig -x spark_local
+... - Connecting to ...
+grunt> 
+</source>
+
 <p><strong>Mapreduce Mode</strong> </p>
 <source>
 $ pig -x mapreduce
@@ -208,6 +228,14 @@
 ... - Connecting to ...
 grunt> 
 </source>
+
+<p><strong>Spark Mode</strong> </p>
+<source>
+$ pig -x spark
+... - Connecting to ...
+grunt>
+</source>
+
 </section>
 </section>
 
@@ -237,6 +265,10 @@
 <source>
 $ pig -x tez_local id.pig
 </source>
+<p><strong>Spark Local Mode</strong></p>
+<source>
+$ pig -x spark_local id.pig
+</source>
 <p><strong>Mapreduce Mode</strong> </p>
 <source>
 $ pig id.pig
@@ -247,22 +279,26 @@
 <source>
 $ pig -x tez id.pig
 </source>
+<p><strong>Spark Mode</strong> </p>
+<source>
+$ pig -x spark id.pig
+</source>
 </section>
 
   <!-- ==================================================================== -->
-    
+
    <!-- PIG SCRIPTS -->
    <section id="pig-scripts">
-	<title>Pig Scripts</title>
-	
-<p>Use Pig scripts to place Pig Latin statements and Pig commands in a single file. While not required, it is good practice to identify the file using the *.pig extension.</p>	
-	
+    <title>Pig Scripts</title>
+
+<p>Use Pig scripts to place Pig Latin statements and Pig commands in a single file. While not required, it is good practice to identify the file using the *.pig extension.</p>
+
 <p>You can run Pig scripts from the command line and from the Grunt shell
 (see the <a href="cmds.html#run">run</a> and <a href="cmds.html#exec">exec</a> commands). </p>
-	
+
 <p>Pig scripts allow you to pass values to parameters using <a href="cont.html#Parameter-Sub">parameter substitution</a>. </p>
 
-<!-- +++++++++++++++++++++++++++++++++++++++++++ -->	
+<!-- +++++++++++++++++++++++++++++++++++++++++++ -->    
    <p id="comments"><strong>Comments in Scripts</strong></p>
    
    <p>You can include comments in Pig scripts:</p>
@@ -284,8 +320,8 @@
 B = FOREACH A GENERATE name;  -- transforming data
 DUMP B;  -- retrieving results
 </source>   
-	
-<!-- +++++++++++++++++++++++++++++++++++++++++++ -->		
+    
+<!-- +++++++++++++++++++++++++++++++++++++++++++ -->        
 
 <p id="dfs"><strong>Scripts and Distributed File Systems</strong></p>
 
@@ -293,7 +329,7 @@
 <source>
 $ pig hdfs://nn.mydomain.com:9020/myscripts/script.pig
 </source> 
-</section>	
+</section>    
 </section>
 </section>
 
@@ -355,7 +391,7 @@
     
    <!-- PIG LATIN STATEMENTS -->
    <section id="pl-statements">
-	<title>Pig Latin Statements</title>	
+    <title>Pig Latin Statements</title>    
    <p>Pig Latin statements are the basic constructs you use to process data using Pig. 
    A Pig Latin statement is an operator that takes a <a href="basic.html#relations">relation</a> as input and produces another relation as output. 
    (This definition applies to all Pig Latin operators except LOAD and STORE which read data from and write data to the file system.) 
@@ -496,20 +532,20 @@
 <p></p>
 <p id="pig-properties">To specify Pig properties use one of these mechanisms:</p>
 <ul>
-	<li>The pig.properties file (add the directory that contains the pig.properties file to the classpath)</li>
-	<li>The -D and a Pig property in PIG_OPTS environment variable (export PIG_OPTS=-Dpig.tmpfilecompression=true)</li>
-	<li>The -P command line option and a properties file (pig -P mypig.properties)</li>
-	<li>The <a href="cmds.html#set">set</a> command (set pig.exec.nocombiner true)</li>
+    <li>The pig.properties file (add the directory that contains the pig.properties file to the classpath)</li>
+    <li>The -D and a Pig property in PIG_OPTS environment variable (export PIG_OPTS=-Dpig.tmpfilecompression=true)</li>
+    <li>The -P command line option and a properties file (pig -P mypig.properties)</li>
+    <li>The <a href="cmds.html#set">set</a> command (set pig.exec.nocombiner true)</li>
 </ul>
 <p><strong>Note:</strong> The properties file uses standard Java property file format.</p>
 <p>The following precedence order is supported: pig.properties &lt; -D Pig property &lt; -P properties file &lt; set command. This means that if the same property is provided using the –D command line option as well as the –P command line option (properties file), the value of the property in the properties file will take precedence.</p>
 
 <p id="hadoop-properties">To specify Hadoop properties you can use the same mechanisms:</p>
 <ul>
-	<li>Hadoop configuration files (include pig-cluster-hadoop-site.xml)</li>
-	<li>The -D and a Hadoop property in PIG_OPTS environment variable (export PIG_OPTS=–Dmapreduce.task.profile=true) </li>
-	<li>The -P command line option and a property file (pig -P property_file)</li>
-	<li>The <a href="cmds.html#set">set</a> command (set mapred.map.tasks.speculative.execution false)</li>
+    <li>Hadoop configuration files (include pig-cluster-hadoop-site.xml)</li>
+    <li>The -D and a Hadoop property in PIG_OPTS environment variable (export PIG_OPTS=–Dmapreduce.task.profile=true) </li>
+    <li>The -P command line option and a property file (pig -P property_file)</li>
+    <li>The <a href="cmds.html#set">set</a> command (set mapred.map.tasks.speculative.execution false)</li>
 </ul>
 <p></p>
 <p>The same precedence holds: Hadoop configuration files &lt; -D Hadoop property &lt; -P properties_file &lt; set command.</p>
@@ -523,7 +559,7 @@
   <section id="tutorial">
 <title>Pig Tutorial </title>
 
-<p>The Pig tutorial shows you how to run Pig scripts using Pig's local mode, mapreduce mode and Tez mode (see <a href="#execution-modes">Execution Modes</a>).</p>
+<p>The Pig tutorial shows you how to run Pig scripts using Pig's local mode, mapreduce mode, Tez mode and Spark mode (see <a href="#execution-modes">Execution Modes</a>).</p>
 
 <p>To get started, do the following preliminary tasks:</p>
 
@@ -541,8 +577,8 @@
 <li>Create the pigtutorial.tar.gz file:
 <ul>
     <li>Move to the Pig tutorial directory (.../pig-0.16.0/tutorial).</li>
-	<li>Run the "ant" command from the tutorial directory. This will create the pigtutorial.tar.gz file.
-	</li>
+    <li>Run the "ant" command from the tutorial directory. This will create the pigtutorial.tar.gz file.
+    </li>
 </ul>
 
 </li>
@@ -574,6 +610,10 @@
 <source>
 $ pig -x tez_local script1-local.pig
 </source>
+Or if you are using Spark local mode:
+<source>
+$ pig -x spark_local script1-local.pig
+</source>
 </li>
 <li>Review the result files, located in the script1-local-results.txt directory.
 <p>The output may contain a few Hadoop warnings which can be ignored:</p>
@@ -587,7 +627,7 @@
 
  <!-- ++++++++++++++++++++++++++++++++++ --> 
 <section>
-<title> Running the Pig Scripts in Mapreduce Mode or Tez Mode</title>
+<title> Running the Pig Scripts in Mapreduce Mode, Tez Mode or Spark Mode</title>
 
 <p>To run the Pig scripts in mapreduce mode, do the following: </p>
 <ol>
@@ -606,6 +646,8 @@
 <source>
 export PIG_CLASSPATH=/mycluster/conf:/tez/conf
 </source>
+<p>If you are using Spark, you will also need to specify SPARK_HOME and specify SPARK_JAR which is the hdfs location where you uploaded $SPARK_HOME/lib/spark-assembly*.jar:</p>
+<source>export SPARK_HOME=/mysparkhome/; export SPARK_JAR=hdfs://example.com:8020/spark-assembly*.jar</source>
 <p><strong>Note:</strong> The PIG_CLASSPATH can also be used to add any other 3rd party dependencies or resource files a pig script may require. If there is also a need to make the added entries take the highest precedence in the Pig JVM's classpath order, one may also set the env-var PIG_USER_CLASSPATH_FIRST to any value, such as 'true' (and unset the env-var to disable).</p></li>
 <li>Set the HADOOP_CONF_DIR environment variable to the location of the cluster configuration directory:
 <source>
@@ -620,6 +662,10 @@
 <source>
 $ pig -x tez script1-hadoop.pig
 </source>
+Or if you are using Spark:
+<source>
+$ pig -x spark script1-hadoop.pig
+</source>
 </li>
 
 <li>Review the result files, located in the script1-hadoop-results or script2-hadoop-results HDFS directory:
diff --git a/src/org/apache/pig/PigConfiguration.java b/src/org/apache/pig/PigConfiguration.java
index 44e437c..c9a91b6 100644
--- a/src/org/apache/pig/PigConfiguration.java
+++ b/src/org/apache/pig/PigConfiguration.java
@@ -482,6 +482,11 @@
     public static final String PIG_LOG_TRACE_ID = "pig.log.trace.id";
 
     /**
+     * Use Netty file server for Pig on Spark, true or false, default value is false
+     */
+    public static final String PIG_SPARK_USE_NETTY_FILESERVER = "pig.spark.rpc.useNettyFileServer";
+
+    /**
      * @deprecated use {@link #PIG_LOG_TRACE_ID} instead. Will be removed in Pig 0.18
      */
     public static final String CALLER_ID = PIG_LOG_TRACE_ID;
diff --git a/src/org/apache/pig/PigWarning.java b/src/org/apache/pig/PigWarning.java
index fcda114..6e93e95 100644
--- a/src/org/apache/pig/PigWarning.java
+++ b/src/org/apache/pig/PigWarning.java
@@ -68,6 +68,8 @@
     DELETE_FAILED,
     PROJECTION_INVALID_RANGE,
     NO_LOAD_FUNCTION_FOR_CASTING_BYTEARRAY,
-    SKIP_UDF_CALL_FOR_NULL
+    SKIP_UDF_CALL_FOR_NULL,
+    SPARK_WARN, //bulk collection of warnings under Spark exec engine
+    SPARK_CUSTOM_WARN // same as above but for custom UDF warnings only, see PIG-2207
     ;
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/AccumulatorOptimizer.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/AccumulatorOptimizer.java
index ac03d40..d9536bb 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/AccumulatorOptimizer.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/AccumulatorOptimizer.java
@@ -35,6 +35,6 @@
     }
 
     public void visitMROp(MapReduceOper mr) throws VisitorException {
-        AccumulatorOptimizerUtil.addAccumulator(mr.reducePlan);
+        AccumulatorOptimizerUtil.addAccumulator(mr.reducePlan, mr.reducePlan.getRoots());
     }
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemover.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemover.java
index 4d91556..5325b82 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemover.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemover.java
@@ -28,7 +28,6 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
-import org.apache.pig.impl.plan.PlanException;
 import org.apache.pig.impl.plan.DependencyOrderWalker;
 import org.apache.pig.impl.plan.VisitorException;
 import org.apache.pig.impl.util.Pair;
@@ -69,7 +68,7 @@
         public void visit() throws VisitorException {
             super.visit();
             for (Pair<POFilter, PhysicalPlan> pair: removalQ) {
-                removeFilter(pair.first, pair.second);
+                NoopFilterRemoverUtil.removeFilter(pair.first, pair.second);
             }
             removalQ.clear();
         }
@@ -91,23 +90,5 @@
                 }
             }
         }
-        
-        private void removeFilter(POFilter filter, PhysicalPlan plan) {
-            if (plan.size() > 1) {
-                try {
-                    List<PhysicalOperator> fInputs = filter.getInputs();
-                    List<PhysicalOperator> sucs = plan.getSuccessors(filter);
-
-                    plan.removeAndReconnect(filter);
-                    if(sucs!=null && sucs.size()!=0){
-                        for (PhysicalOperator suc : sucs) {
-                            suc.setInputs(fInputs);
-                        }
-                    }
-                } catch (PlanException pe) {
-                    log.info("Couldn't remove a filter in optimizer: "+pe.getMessage());
-                }
-            }
-        }
     }
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemoverUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemoverUtil.java
new file mode 100644
index 0000000..982009a
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/NoopFilterRemoverUtil.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.mapReduceLayer;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
+import org.apache.pig.impl.plan.PlanException;
+
+import java.util.List;
+
+public class NoopFilterRemoverUtil {
+    private static Log log = LogFactory.getLog(NoopFilterRemoverUtil.class);
+
+    public static void removeFilter(POFilter filter, PhysicalPlan plan) {
+        if (plan.size() > 1) {
+            try {
+                List<PhysicalOperator> fInputs = filter.getInputs();
+                List<PhysicalOperator> sucs = plan.getSuccessors(filter);
+
+                plan.removeAndReconnect(filter);
+                if(sucs!=null && sucs.size()!=0){
+                    for (PhysicalOperator suc : sucs) {
+                        suc.setInputs(fInputs);
+                    }
+                }
+            } catch (PlanException pe) {
+                log.info("Couldn't remove a filter in optimizer: "+pe.getMessage());
+            }
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigHadoopLogger.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigHadoopLogger.java
index 255650e..e85e73a 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigHadoopLogger.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigHadoopLogger.java
@@ -23,7 +23,7 @@
 import org.apache.pig.LoadFunc;
 import org.apache.pig.StoreFunc;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PigLogger;
-import org.apache.pig.tools.pigstats.PigStatusReporter;
+import org.apache.pig.tools.pigstats.PigWarnCounter;
 
 /**
  *
@@ -36,7 +36,7 @@
     private static Log log = LogFactory.getLog(PigHadoopLogger.class);
     private static PigHadoopLogger logger = null;
 
-    private PigStatusReporter reporter = null;
+    private PigWarnCounter reporter = null;
     private boolean aggregate = false;
 
     private PigHadoopLogger() {
@@ -52,7 +52,7 @@
         return logger;
     }
 
-    public void setReporter(PigStatusReporter reporter) {
+    public void setReporter(PigWarnCounter reporter) {
         this.reporter = reporter;
     }
 
@@ -65,10 +65,10 @@
         if (getAggregate()) {
             if (reporter != null) {
                 if (o instanceof EvalFunc || o instanceof LoadFunc || o instanceof StoreFunc) {
-                    reporter.incrCounter(className, warningEnum.name(), 1);
+                    reporter.incrWarnCounter(className, warningEnum.name(), 1L);
                 }
                 // For backwards compatibility, always report with warningEnum, see PIG-3739
-                reporter.incrCounter(warningEnum, 1);
+                reporter.incrWarnCounter(warningEnum, 1L);
             } else {
                 //TODO:
                 //in local mode of execution if the PigHadoopLogger is used initially,
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java
index 6fe8ff3..aad4926 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigInputFormat.java
@@ -61,14 +61,6 @@
     public static final String PIG_INPUT_SIGNATURES = "pig.inpSignatures";
     public static final String PIG_INPUT_LIMITS = "pig.inpLimits";
 
-    /**
-     * @deprecated Use {@link UDFContext} instead in the following way to get
-     * the job's {@link Configuration}:
-     * <pre>UdfContext.getUdfContext().getJobConf()</pre>
-     */
-    @Deprecated
-    public static Configuration sJob;
-
     /* (non-Javadoc)
      * @see org.apache.hadoop.mapreduce.InputFormat#createRecordReader(org.apache.hadoop.mapreduce.InputSplit, org.apache.hadoop.mapreduce.TaskAttemptContext)
      */
@@ -78,43 +70,66 @@
             org.apache.hadoop.mapreduce.InputSplit split,
             TaskAttemptContext context) throws IOException,
             InterruptedException {
-        // We need to create a TaskAttemptContext based on the Configuration which
-        // was used in the getSplits() to produce the split supplied here. For
-        // this, let's find out the input of the script which produced the split
-        // supplied here and then get the corresponding Configuration and setup
-        // TaskAttemptContext based on it and then call the real InputFormat's
-        // createRecordReader() method
+        RecordReaderFactory factory = new RecordReaderFactory(split, context);
+        return factory.createRecordReader();
+    }
 
-        PigSplit pigSplit = (PigSplit)split;
-        activeSplit = pigSplit;
-        // XXX hadoop 20 new API integration: get around a hadoop 20 bug by
-        // passing total # of splits to each split so it can be retrieved
-        // here and set it to the configuration object. This number is needed
-        // by PoissonSampleLoader to compute the number of samples
-        int n = pigSplit.getTotalSplits();
-        context.getConfiguration().setInt("pig.mapsplits.count", n);
-        Configuration conf = context.getConfiguration();
-        PigContext.setPackageImportList((ArrayList<String>) ObjectSerializer
-                .deserialize(conf.get("udf.import.list")));
-        MapRedUtil.setupUDFContext(conf);
-        LoadFunc loadFunc = getLoadFunc(pigSplit.getInputIndex(), conf);
-        // Pass loader signature to LoadFunc and to InputFormat through
-        // the conf
-        passLoadSignature(loadFunc, pigSplit.getInputIndex(), conf);
 
-        // merge entries from split specific conf into the conf we got
-        PigInputFormat.mergeSplitSpecificConf(loadFunc, pigSplit, conf);
+    /**
+     * Helper class to create record reader
+     */
+    protected static class RecordReaderFactory {
+        protected InputFormat inputFormat;
+        protected PigSplit pigSplit;
+        protected LoadFunc loadFunc;
+        protected TaskAttemptContext context;
+        protected long limit;
 
-        // for backward compatibility
-        PigInputFormat.sJob = conf;
+        public RecordReaderFactory(org.apache.hadoop.mapreduce.InputSplit split,
+                                   TaskAttemptContext context) throws IOException {
 
-        InputFormat inputFormat = loadFunc.getInputFormat();
+            // We need to create a TaskAttemptContext based on the Configuration which
+            // was used in the getSplits() to produce the split supplied here. For
+            // this, let's find out the input of the script which produced the split
+            // supplied here and then get the corresponding Configuration and setup
+            // TaskAttemptContext based on it and then call the real InputFormat's
+            // createRecordReader() method
 
-        List<Long> inpLimitLists =
-                (ArrayList<Long>)ObjectSerializer.deserialize(
-                        conf.get(PIG_INPUT_LIMITS));
+            PigSplit pigSplit = (PigSplit)split;
+            // XXX hadoop 20 new API integration: get around a hadoop 20 bug by
+            // passing total # of splits to each split so it can be retrieved
+            // here and set it to the configuration object. This number is needed
+            // by PoissonSampleLoader to compute the number of samples
+            int n = pigSplit.getTotalSplits();
+            context.getConfiguration().setInt("pig.mapsplits.count", n);
+            Configuration conf = context.getConfiguration();
+            PigContext.setPackageImportList((ArrayList<String>) ObjectSerializer
+                    .deserialize(conf.get("udf.import.list")));
+            MapRedUtil.setupUDFContext(conf);
+            LoadFunc loadFunc = getLoadFunc(pigSplit.getInputIndex(), conf);
+            // Pass loader signature to LoadFunc and to InputFormat through
+            // the conf
+            passLoadSignature(loadFunc, pigSplit.getInputIndex(), conf);
 
-        return new PigRecordReader(inputFormat, pigSplit, loadFunc, context, inpLimitLists.get(pigSplit.getInputIndex()));
+            // merge entries from split specific conf into the conf we got
+            PigInputFormat.mergeSplitSpecificConf(loadFunc, pigSplit, conf);
+
+            InputFormat inputFormat = loadFunc.getInputFormat();
+
+            List<Long> inpLimitLists =
+                    (ArrayList<Long>)ObjectSerializer.deserialize(
+                            conf.get(PIG_INPUT_LIMITS));
+
+            this.inputFormat = inputFormat;
+            this.pigSplit = pigSplit;
+            this.loadFunc = loadFunc;
+            this.context = context;
+            this.limit = inpLimitLists.get(pigSplit.getInputIndex());
+        }
+
+        public org.apache.hadoop.mapreduce.RecordReader<Text, Tuple> createRecordReader() throws IOException, InterruptedException {
+            return new PigRecordReader(inputFormat, pigSplit, loadFunc, context, limit);
+        }
     }
 
 
@@ -339,10 +354,4 @@
         return pigSplit;
     }
 
-    public static PigSplit getActiveSplit() {
-        return activeSplit;
-    }
-
-    private static PigSplit activeSplit;
-
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigSplit.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigSplit.java
index e866b28..63a3ce3 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigSplit.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/PigSplit.java
@@ -48,6 +48,7 @@
 import org.apache.hadoop.io.serializer.Deserializer;
 import org.apache.hadoop.io.serializer.SerializationFactory;
 import org.apache.hadoop.io.serializer.Serializer;
+import org.apache.hadoop.mapred.SplitLocationInfo;
 import org.apache.hadoop.mapreduce.InputFormat;
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
@@ -125,6 +126,12 @@
      */
     String[] locations = null;
 
+
+    /**
+     * overall splitLocationInfos
+     */
+    SplitLocationInfo[] splitLocationInfos = null;
+
     // this seems necessary for Hadoop to instatiate this split on the
     // backend
     public PigSplit() {}
@@ -201,6 +208,51 @@
         return locations;
     }
 
+
+    @Override
+    public SplitLocationInfo[] getLocationInfo() throws IOException {
+        if (splitLocationInfos == null) {
+            HashMap<SplitLocationInfo, Long> locMap = new HashMap<SplitLocationInfo, Long>();
+            Long lenInMap;
+            for (InputSplit split : wrappedSplits) {
+                SplitLocationInfo[] locs = split.getLocationInfo();
+                if( locs != null) {
+                    for (SplitLocationInfo loc : locs) {
+                        try {
+                            if ((lenInMap = locMap.get(loc)) == null)
+                                locMap.put(loc, split.getLength());
+                            else
+                                locMap.put(loc, lenInMap + split.getLength());
+                        } catch (InterruptedException e) {
+                            throw new IOException("InputSplit.getLength throws exception: ", e);
+                        }
+                    }
+                }
+            }
+            Set<Map.Entry<SplitLocationInfo, Long>> entrySet = locMap.entrySet();
+            Map.Entry<SplitLocationInfo, Long>[] hostSize =
+                    entrySet.toArray(new Map.Entry[entrySet.size()]);
+            Arrays.sort(hostSize, new Comparator<Map.Entry<SplitLocationInfo, Long>>() {
+
+                @Override
+                public int compare(Entry<SplitLocationInfo, Long> o1, Entry<SplitLocationInfo, Long> o2) {
+                    long diff = o1.getValue() - o2.getValue();
+                    if (diff < 0) return 1;
+                    if (diff > 0) return -1;
+                    return 0;
+                }
+            });
+            // maximum 5 locations are in list: refer to PIG-1648 for more details
+            int nHost = Math.min(hostSize.length, 5);
+            splitLocationInfos = new SplitLocationInfo[nHost];
+            for (int i = 0; i < nHost; ++i) {
+                splitLocationInfos[i] = hostSize[i].getKey();
+            }
+        }
+        return splitLocationInfos;
+    }
+
+
     @Override
     public long getLength() throws IOException, InterruptedException {
         if (length == -1) {
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SecondaryKeyOptimizerMR.java b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SecondaryKeyOptimizerMR.java
index 8170f02..c2bf711 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SecondaryKeyOptimizerMR.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/mapReduceLayer/SecondaryKeyOptimizerMR.java
@@ -52,7 +52,8 @@
         if (mr.getCustomPartitioner()!=null)
             return;
 
-        info = SecondaryKeyOptimizerUtil.applySecondaryKeySort(mr.mapPlan, mr.reducePlan);
+        SecondaryKeyOptimizerUtil secondaryKeyOptUtil = new SecondaryKeyOptimizerUtil();
+        info = secondaryKeyOptUtil.applySecondaryKeySort(mr.mapPlan, mr.reducePlan);
         if (info != null && info.isUseSecondaryKey()) {
             mr.setUseSecondaryKey(true);
             mr.setSecondarySortOrder(info.getSecondarySortOrder());
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java
index 0e35273..8f8b968 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/PhysicalOperator.java
@@ -497,6 +497,10 @@
        parentPlan = physicalPlan;
     }
 
+    public PhysicalPlan getParentPlan() {
+        return parentPlan;
+    }
+
     public Log getLogger() {
         return log;
     }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
index ecf780c..2c8dea6 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/expressionOperators/POUserFunc.java
@@ -120,6 +120,10 @@
         instantiateFunc(funcSpec);
     }
 
+    public void setFuncInputSchema(){
+        setFuncInputSchema(signature);
+    }
+
     private void instantiateFunc(FuncSpec fSpec) {
         this.func = (EvalFunc) PigContext.instantiateFuncFromSpec(fSpec);
         this.setSignature(signature);
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java
index 3bad98b..5242273 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhyPlanVisitor.java
@@ -73,6 +73,7 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
 import org.apache.pig.impl.plan.PlanVisitor;
 import org.apache.pig.impl.plan.PlanWalker;
 import org.apache.pig.impl.plan.VisitorException;
@@ -363,4 +364,8 @@
 
     public void visitPoissonSample(POPoissonSample poissonSample) throws VisitorException {
     }
+
+    public void visitBroadcastSpark(POBroadcastSpark poBroadcastSpark) {
+    }
+
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java
index 2376d03..4ee4591 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/plans/PhysicalPlan.java
@@ -143,13 +143,6 @@
         to.setInputs(getPredecessors(to));
     }
 
-    /*public void connect(List<PhysicalOperator> from, PhysicalOperator to) throws IOException{
-        if(!to.supportsMultipleInputs()){
-            throw new IOException("Invalid Operation on " + to.name() + ". It doesn't support multiple inputs.");
-        }
-
-    }*/
-
     @Override
     public void remove(PhysicalOperator op) {
         op.setInputs(null);
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java
new file mode 100644
index 0000000..fa13fd9
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POBroadcastSpark.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class POBroadcastSpark extends PhysicalOperator {
+    private static final long serialVersionUID = 1L;
+
+    protected String broadcastedVariableName;
+
+    public POBroadcastSpark(OperatorKey k) {
+        super(k);
+    }
+
+    public POBroadcastSpark(POBroadcastSpark copy)
+            throws ExecException {
+        super(copy);
+    }
+
+    /**
+     * Set your broadcast variable name so that
+     * BroadcastConverter can put this broadcasted variable in a map
+     * which can be referenced by other functions / closures in Converters
+     *
+     * @param varName
+     */
+    public void setBroadcastedVariableName(String varName) {
+        broadcastedVariableName = varName;
+    }
+
+    public String getBroadcastedVariableName() {
+        return broadcastedVariableName;
+    }
+
+    @Override
+    public Tuple illustratorMarkup(Object in, Object out, int eqClassIndex) {
+        return null;
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return false;
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "BroadcastSpark - " + mKey.toString();
+    }
+
+    @Override
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visitBroadcastSpark(this);
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCollectedGroup.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCollectedGroup.java
index bcbfe2b..5be0fe2 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCollectedGroup.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POCollectedGroup.java
@@ -70,6 +70,15 @@
 
     private transient boolean useDefaultBag;
 
+    //For Spark
+    private transient boolean endOfInput = false;
+    public boolean isEndOfInput() {
+        return endOfInput;
+    }
+    public void setEndOfInput (boolean isEndOfInput) {
+        endOfInput = isEndOfInput;
+    }
+
     public POCollectedGroup(OperatorKey k) {
         this(k, -1, null);
     }
@@ -132,7 +141,7 @@
             if (inp.returnStatus == POStatus.STATUS_EOP) {
                 // Since the output is buffered, we need to flush the last
                 // set of records when the close method is called by mapper.
-                if (this.parentPlan.endOfAllInput) {
+                if (this.parentPlan.endOfAllInput || isEndOfInput()) {
                     return getStreamCloseResult();
                 } else {
                     break;
@@ -257,13 +266,13 @@
             leafOps.add(leaf);
         }
    }
-    
+
     private void setIllustratorEquivalenceClasses(Tuple tin) {
         if (illustrator != null) {
           illustrator.getEquivalenceClasses().get(0).add(tin);
         }
     }
-    
+
     @Override
     public Tuple illustratorMarkup(Object in, Object out, int eqClassIndex) {
         return null;
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java
index d80951a..b158756 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoin.java
@@ -519,6 +519,10 @@
         return LRs;
     }
 
+    public boolean isLeftOuterJoin() {
+        return isLeftOuterJoin;
+    }
+
     public int getFragment() {
         return fragment;
     }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoinSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoinSpark.java
new file mode 100644
index 0000000..0d46b21
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POFRJoinSpark.java
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.SchemaTupleBackend;
+import org.apache.pig.data.SchemaTupleClassGenerator;
+import org.apache.pig.data.SchemaTupleFactory;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+
+import java.util.List;
+import java.util.Map;
+
+public class POFRJoinSpark extends POFRJoin {
+    private static final Log log = LogFactory.getLog(POFRJoinSpark.class);
+
+    private Map<String, List<Tuple>> broadcasts;
+
+    public POFRJoinSpark(POFRJoin copy) throws ExecException {
+        super(copy);
+    }
+
+    @Override
+    protected void setUpHashMap() throws ExecException {
+        log.info("Building replication hash table");
+
+        SchemaTupleFactory[] inputSchemaTupleFactories = new SchemaTupleFactory[inputSchemas.length];
+        SchemaTupleFactory[] keySchemaTupleFactories = new SchemaTupleFactory[inputSchemas.length];
+        for (int i = 0; i < inputSchemas.length; i++) {
+            addSchemaToFactories(inputSchemas[i], inputSchemaTupleFactories, i);
+            addSchemaToFactories(keySchemas[i], keySchemaTupleFactories, i);
+        }
+
+        replicates.set(fragment, null);
+        int i = -1;
+        long start = System.currentTimeMillis();
+        for (int k = 0; k < inputSchemas.length; ++k) {
+            ++i;
+
+            SchemaTupleFactory inputSchemaTupleFactory = inputSchemaTupleFactories[i];
+            SchemaTupleFactory keySchemaTupleFactory = keySchemaTupleFactories[i];
+
+            if (i == fragment) {
+                replicates.set(i, null);
+                continue;
+            }
+
+            TupleToMapKey replicate = new TupleToMapKey(1000, keySchemaTupleFactory);
+
+            log.debug("Completed setup. Trying to build replication hash table");
+            List<Tuple> tuples = broadcasts.get(parentPlan.getPredecessors(this).get(i).getOperatorKey().toString());
+
+            POLocalRearrange localRearrange = LRs[i];
+
+            for (Tuple t : tuples) {
+                localRearrange.attachInput(t);
+                Result res = localRearrange.getNextTuple();
+                if (getReporter() != null) {
+                    getReporter().progress();
+                }
+                Tuple tuple = (Tuple) res.result;
+                if (isKeyNull(tuple.get(1))) continue;
+                Object key = tuple.get(1);
+                Tuple value = getValueTuple(localRearrange, tuple);
+
+                if (replicate.get(key) == null) {
+                    replicate.put(key, new POMergeJoin.TuplesToSchemaTupleList(1, inputSchemaTupleFactory));
+                }
+
+                replicate.get(key).add(value);
+
+            }
+            replicates.set(i, replicate);
+        }
+        long end = System.currentTimeMillis();
+        log.debug("Hash Table built. Time taken: " + (end - start));
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "FRJoinSpark[" + DataType.findTypeName(resultType)
+                + "]" + " - " + mKey.toString();
+    }
+
+    private void addSchemaToFactories(Schema schema, SchemaTupleFactory[] schemaTupleFactories, int index) {
+        if (schema != null) {
+            log.debug("Using SchemaTuple for FR Join Schema: " + schema);
+            schemaTupleFactories[index] = SchemaTupleBackend.newSchemaTupleFactory(schema, false, SchemaTupleClassGenerator.GenContext.FR_JOIN);
+        }
+    }
+
+    public void attachInputs(Map<String, List<Tuple>> broadcasts) {
+        this.broadcasts = broadcasts;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java
index 4873e9e..aba5131 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POForEach.java
@@ -828,6 +828,10 @@
         }
     }
 
+    public PhysicalOperator[] getPlanLeafOps() {
+        return planLeafOps;
+    }
+
     public void setMapSideOnly(boolean mapSideOnly) {
         this.mapSideOnly = mapSideOnly;
     }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POGlobalRearrange.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POGlobalRearrange.java
index 52cfb73..92003fc 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POGlobalRearrange.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POGlobalRearrange.java
@@ -74,7 +74,13 @@
     public POGlobalRearrange(OperatorKey k, int rp, List inp) {
         super(k, rp, inp);
     }
-    
+
+    public POGlobalRearrange(POGlobalRearrange copy) throws ExecException {
+        super(copy);
+        this.cross = copy.cross;
+        this.customPartitioner = copy.customPartitioner;
+    }
+
     @Override
     public void visit(PhyPlanVisitor v) throws VisitorException {
         v.visitGlobalRearrange(this);
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeCogroup.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeCogroup.java
index 4923d3f..f18d47a 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeCogroup.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeCogroup.java
@@ -122,6 +122,15 @@
         this.endOfRecordMark = endOfRecordMark;
     }
 
+    //For Spark
+    private transient boolean endOfInput = false;
+    public boolean isEndOfInput() {
+        return endOfInput;
+    }
+    public void setEndOfInput (boolean isEndOfInput) {
+        endOfInput = isEndOfInput;
+    }
+
     @Override
     public Result getNextTuple() throws ExecException {
 
@@ -145,7 +154,7 @@
                 break;
 
             case POStatus.STATUS_EOP:
-                if(!this.parentPlan.endOfAllInput)
+                if(!(this.parentPlan.endOfAllInput || isEndOfInput()))
                     return baseInp;
 
                 if(lastTime)
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java
index 13f70c0..815a325 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POMergeJoin.java
@@ -131,6 +131,24 @@
 
     private byte endOfRecordMark = POStatus.STATUS_NULL;
 
+    // Only for Spark
+    // If current operator reaches at its end, flag endOfInput is set as true.
+    // The old flag parentPlan.endOfAllInput doesn't work in spark mode, because it is shared
+    // between operators in the same plan, so it could be set by preceding operators even
+    // current operator does not reach at its end. (see PIG-4876)
+    private transient boolean endOfInput = false;
+    public boolean isEndOfInput() {
+        return endOfInput;
+    }
+    public void setEndOfInput (boolean isEndOfInput) {
+        endOfInput = isEndOfInput;
+    }
+
+    // Only for spark.
+    // it means that current operator reaches at its end and the last left input was
+    // added into 'leftTuples', ready for join.
+    private boolean leftInputConsumedInSpark = false;
+
     // This serves as the default TupleFactory
     private transient TupleFactory mTupleFactory;
 
@@ -352,7 +370,7 @@
 
                     }
                     else if(cmpval > 0){    // We got ahead on right side. Store currently read right tuple.
-                        if(!this.parentPlan.endOfAllInput){
+                        if(!(this.parentPlan.endOfAllInput|| leftInputConsumedInSpark)){
                             prevRightKey = rightKey;
                             prevRightInp = rightInp;
                             // There cant be any more join on this key.
@@ -413,11 +431,14 @@
             }
 
         case POStatus.STATUS_EOP:
-            if(this.parentPlan.endOfAllInput){
+            if(this.parentPlan.endOfAllInput || isEndOfInput()){
                 // We hit the end on left input.
                 // Tuples in bag may still possibly join with right side.
                 curJoinKey = prevLeftKey;
                 curLeftKey = null;
+                if (isEndOfInput()) {
+                    leftInputConsumedInSpark = true;
+                }
                 break;
             }
             else    // Fetch next left input.
@@ -427,7 +448,9 @@
             return curLeftInp;
         }
 
-        if((null != prevRightKey) && !this.parentPlan.endOfAllInput && ((Comparable)prevRightKey).compareTo(curLeftKey) >= 0){
+        if((null != prevRightKey)
+                && !(this.parentPlan.endOfAllInput || leftInputConsumedInSpark)
+                && ((Comparable)prevRightKey).compareTo(curLeftKey) >= 0){
 
             // This will happen when we accumulated inputs on left side and moved on, but are still behind the right side
             // In that case, throw away the tuples accumulated till now and add the one we read in this function call.
@@ -509,7 +532,7 @@
                 leftTuples.add((Tuple)curLeftInp.result);
                 prevLeftInp = curLeftInp;
                 prevLeftKey = curLeftKey;
-                if(this.parentPlan.endOfAllInput){  // This is end of all input and this is last time we will read right input.
+                if(this.parentPlan.endOfAllInput || leftInputConsumedInSpark){  // This is end of all input and this is last time we will read right input.
                     // Right loader in this case wouldn't get a chance to close input stream. So, we close it ourself.
                     try {
                         ((IndexableLoadFunc)rightLoader).close();
@@ -719,4 +742,8 @@
     public LOJoin.JOINTYPE getJoinType() {
         return joinType;
     }
+
+    public POLocalRearrange[] getLRs() {
+        return LRs;
+    }
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPoissonSample.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPoissonSample.java
index f2830c2..8921cb1 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPoissonSample.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POPoissonSample.java
@@ -29,45 +29,45 @@
 
 public class POPoissonSample extends PhysicalOperator {
 
-    private static final long serialVersionUID = 1L;
+    protected static final long serialVersionUID = 1L;
 
     // 17 is not a magic number. It can be obtained by using a poisson
     // cumulative distribution function with the mean set to 10 (empirically,
     // minimum number of samples) and the confidence set to 95%
     public static final int DEFAULT_SAMPLE_RATE = 17;
 
-    private int sampleRate = 0;
+    protected int sampleRate = 0;
 
-    private float heapPerc = 0f;
+    protected float heapPerc = 0f;
 
-    private Long totalMemory;
+    protected Long totalMemory;
 
-    private transient boolean initialized;
+    protected transient boolean initialized;
 
     // num of rows skipped so far
-    private transient int numSkipped;
+    protected transient int numSkipped;
 
     // num of rows sampled so far
-    private transient int numRowsSampled;
+    protected transient int numRowsSampled;
 
     // average size of tuple in memory, for tuples sampled
-    private transient long avgTupleMemSz;
+    protected transient long avgTupleMemSz;
 
     // current row number
-    private transient long rowNum;
+    protected transient long rowNum;
 
     // number of tuples to skip after each sample
-    private transient long skipInterval;
+    protected transient long skipInterval;
 
     // bytes in input to skip after every sample.
     // divide this by avgTupleMemSize to get skipInterval
-    private transient long memToSkipPerSample;
+    protected transient long memToSkipPerSample;
 
     // has the special row with row number information been returned
-    private transient boolean numRowSplTupleReturned;
+    protected transient boolean numRowSplTupleReturned;
 
     // new Sample result
-    private transient Result newSample;
+    protected transient Result newSample;
 
     public POPoissonSample(OperatorKey k, int rp, int sr, float hp, long tm) {
         super(k, rp, null);
@@ -204,7 +204,7 @@
      * and recalculate skipInterval
      * @param t - tuple
      */
-    private void updateSkipInterval(Tuple t) {
+    protected void updateSkipInterval(Tuple t) {
         avgTupleMemSz =
             ((avgTupleMemSz*numRowsSampled) + t.getMemorySize())/(numRowsSampled + 1);
         skipInterval = memToSkipPerSample/avgTupleMemSz;
@@ -224,7 +224,7 @@
      * @return - Tuple appended with special marker string column, num-rows column
      * @throws ExecException
      */
-    private Result createNumRowTuple(Tuple sample) throws ExecException {
+    protected Result createNumRowTuple(Tuple sample) throws ExecException {
         int sz = (sample == null) ? 0 : sample.size();
         Tuple t = mTupleFactory.newTuple(sz + 2);
 
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java
index 3f877ad..a24a3be 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/physicalLayer/relationalOperators/POSort.java
@@ -347,6 +347,10 @@
         mSortFunc = sortFunc;
     }
 
+    public Comparator<Tuple> getMComparator() {
+        return mComparator;
+    }
+
     public List<Boolean> getMAscCols() {
         return mAscCols;
     }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java
new file mode 100644
index 0000000..5eac045
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/JobGraphBuilder.java
@@ -0,0 +1,443 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang3.ArrayUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.pig.PigException;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.JobCreationException;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PhyPlanSetter;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.UDFFinishVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPoissonSample;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSkewedJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.RDDConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.SkewedJoinConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POJoinGroupSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POPoissonSampleSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.tools.pigstats.spark.SparkPigStats;
+import org.apache.pig.tools.pigstats.spark.SparkStatsUtil;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.rdd.RDD;
+
+import com.google.common.collect.Lists;
+
+public class JobGraphBuilder extends SparkOpPlanVisitor {
+
+    private static final Log LOG = LogFactory.getLog(JobGraphBuilder.class);
+    public static final int NULLPART_JOB_ID = -1;
+
+    private Map<Class<? extends PhysicalOperator>, RDDConverter> convertMap = null;
+    private SparkPigStats sparkStats = null;
+    private JavaSparkContext sparkContext = null;
+    private JobMetricsListener jobMetricsListener = null;
+    private String jobGroupID = null;
+    private Set<Integer> seenJobIDs = new HashSet<Integer>();
+    private SparkOperPlan sparkPlan = null;
+    private Map<OperatorKey, RDD<Tuple>> sparkOpRdds = new HashMap<OperatorKey, RDD<Tuple>>();
+    private Map<OperatorKey, RDD<Tuple>> physicalOpRdds = new HashMap<OperatorKey, RDD<Tuple>>();
+    private JobConf jobConf = null;
+    private PigContext pc;
+
+    public JobGraphBuilder(SparkOperPlan plan, Map<Class<? extends PhysicalOperator>, RDDConverter> convertMap,
+                           SparkPigStats sparkStats, JavaSparkContext sparkContext, JobMetricsListener
+            jobMetricsListener, String jobGroupID, JobConf jobConf, PigContext pc) {
+        super(plan, new DependencyOrderWalker<SparkOperator, SparkOperPlan>(plan, true));
+        this.sparkPlan = plan;
+        this.convertMap = convertMap;
+        this.sparkStats = sparkStats;
+        this.sparkContext = sparkContext;
+        this.jobMetricsListener = jobMetricsListener;
+        this.jobGroupID = jobGroupID;
+        this.jobConf = jobConf;
+        this.pc = pc;
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        new PhyPlanSetter(sparkOp.physicalPlan).visit();
+        try {
+            setReplicationForMergeJoin(sparkOp.physicalPlan);
+            sparkOperToRDD(sparkOp);
+            finishUDFs(sparkOp.physicalPlan);
+        } catch (Exception e) {
+            throw new VisitorException("fail to get the rdds of this spark operator: ", e);
+        }
+    }
+
+    private void setReplicationForMergeJoin(PhysicalPlan plan) throws IOException {
+        List<Path> filesForMoreReplication = new ArrayList<>();
+        List<POMergeJoin> poMergeJoins = PlanHelper.getPhysicalOperators(plan, POMergeJoin.class);
+        if (poMergeJoins.size() > 0) {
+            for (POMergeJoin poMergeJoin : poMergeJoins) {
+                String idxFileName = poMergeJoin.getIndexFile();
+                if (idxFileName != null) {
+                    filesForMoreReplication.add(new Path(idxFileName));
+                }
+                // in spark mode, set as null so that PoMergeJoin won't use hadoop distributed cache
+                // see POMergeJoin.seekInRightStream()
+                poMergeJoin.setIndexFile(null);
+            }
+        }
+
+        setReplicationForFiles(filesForMoreReplication);
+    }
+
+    private void setReplicationForFiles(List<Path> files) throws IOException {
+        FileSystem fs = FileSystem.get(this.jobConf);
+        short replication = (short) jobConf.getInt(MRConfiguration.SUMIT_REPLICATION, 10);
+        for (int i = 0; i < files.size(); i++) {
+            fs.setReplication(files.get(i), replication);
+        }
+    }
+
+    // Calling EvalFunc.finish()
+    private void finishUDFs(PhysicalPlan physicalPlan) throws VisitorException {
+        UDFFinishVisitor finisher = new UDFFinishVisitor(physicalPlan,
+                new DependencyOrderWalker<PhysicalOperator, PhysicalPlan>(
+                        physicalPlan));
+        try {
+            finisher.visit();
+        } catch (VisitorException e) {
+            int errCode = 2121;
+            String msg = "Error while calling finish method on UDFs.";
+            throw new VisitorException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    private void sparkOperToRDD(SparkOperator sparkOperator) throws InterruptedException, VisitorException, JobCreationException, ExecException {
+        List<SparkOperator> predecessors = sparkPlan
+                .getPredecessors(sparkOperator);
+        Set<OperatorKey> predecessorOfPreviousSparkOp = new HashSet<OperatorKey>();
+        if (predecessors != null) {
+            for (SparkOperator pred : predecessors) {
+                predecessorOfPreviousSparkOp.add(pred.getOperatorKey());
+            }
+        }
+
+        boolean isFail = false;
+        Exception exception = null;
+        if (sparkOperator instanceof NativeSparkOperator) {
+            ((NativeSparkOperator) sparkOperator).runJob();
+        } else {
+            List<PhysicalOperator> leafPOs = sparkOperator.physicalPlan.getLeaves();
+
+            //One SparkOperator may have multiple leaves(POStores) after multiquery feature is enabled
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("sparkOperator.physicalPlan have " + sparkOperator.physicalPlan.getLeaves().size() + " leaves");
+            }
+            for (PhysicalOperator leafPO : leafPOs) {
+                try {
+                    physicalToRDD(sparkOperator, sparkOperator.physicalPlan, leafPO,
+                            predecessorOfPreviousSparkOp);
+                    sparkOpRdds.put(sparkOperator.getOperatorKey(),
+                            physicalOpRdds.get(leafPO.getOperatorKey()));
+                } catch (Exception e) {
+                    LOG.error("throw exception in sparkOperToRDD: ", e);
+                    exception = e;
+                    isFail = true;
+                    boolean stopOnFailure = Boolean.valueOf(pc
+                            .getProperties().getProperty("stop.on.failure",
+                                    "false"));
+                    if (stopOnFailure) {
+                        int errCode = 6017;
+                        throw new ExecException(e.getMessage(), errCode,
+                                PigException.REMOTE_ENVIRONMENT);
+                    }
+                }
+            }
+
+
+            List<POStore> poStores = PlanHelper.getPhysicalOperators(
+                    sparkOperator.physicalPlan, POStore.class);
+            Collections.sort(poStores);
+            if (poStores.size() > 0) {
+                int i = 0;
+                if (!isFail) {
+                    List<Integer> jobIDs = getJobIDs(seenJobIDs);
+                    for (POStore poStore : poStores) {
+                        if (jobIDs.size() == 0) {
+                            /**
+                             * Spark internally misses information about its jobs that mapped 0 partitions.
+                             * Although these have valid jobIds, Spark itself is unable to tell anything about them.
+                             * If the store rdd had 0 partitions we return a dummy success stat with jobId =
+                             * NULLPART_JOB_ID, in any other cases we throw exception if no new jobId was seen.
+                             */
+                            if (physicalOpRdds.get(poStore.getOperatorKey()).partitions().length == 0) {
+                                sparkStats.addJobStats(poStore, sparkOperator, NULLPART_JOB_ID, null, sparkContext);
+                                return;
+                            } else {
+                                throw new RuntimeException("Expected at least one unseen jobID "
+                                        + " in this call to getJobIdsForGroup, but got 0");
+                            }
+                        }
+                        SparkStatsUtil.waitForJobAddStats(jobIDs.get(i++), poStore, sparkOperator,
+                                jobMetricsListener, sparkContext, sparkStats);
+                    }
+                } else {
+                    for (POStore poStore : poStores) {
+                        String failJobID = sparkOperator.name().concat("_fail");
+                        SparkStatsUtil.addFailJobStats(failJobID, poStore, sparkOperator, sparkStats, exception);
+                    }
+                }
+            }
+        }
+    }
+
+    private void physicalToRDD(SparkOperator sparkOperator, PhysicalPlan plan,
+                               PhysicalOperator physicalOperator,
+                               Set<OperatorKey> predsFromPreviousSparkOper)
+            throws IOException {
+        RDD<Tuple> nextRDD = null;
+        List<PhysicalOperator> predecessorsOfCurrentPhysicalOp = getPredecessors(plan, physicalOperator);
+        LinkedHashSet<OperatorKey> operatorKeysOfAllPreds = new LinkedHashSet<OperatorKey>();
+        addPredsFromPrevoiousSparkOp(sparkOperator, physicalOperator, operatorKeysOfAllPreds);
+        if (predecessorsOfCurrentPhysicalOp != null) {
+            for (PhysicalOperator predecessor : predecessorsOfCurrentPhysicalOp) {
+                physicalToRDD(sparkOperator, plan, predecessor, predsFromPreviousSparkOper);
+                operatorKeysOfAllPreds.add(predecessor.getOperatorKey());
+            }
+
+        } else {
+            if (predsFromPreviousSparkOper != null
+                    && predsFromPreviousSparkOper.size() > 0) {
+                for (OperatorKey predFromPreviousSparkOper : predsFromPreviousSparkOper) {
+                    operatorKeysOfAllPreds.add(predFromPreviousSparkOper);
+                }
+            }
+        }
+
+        if (physicalOperator instanceof POSplit) {
+            List<PhysicalPlan> successorPlans = ((POSplit) physicalOperator).getPlans();
+            for (PhysicalPlan successorPlan : successorPlans) {
+                List<PhysicalOperator> leavesOfSuccessPlan = successorPlan.getLeaves();
+                if (leavesOfSuccessPlan.size() != 1) {
+                    throw new RuntimeException("the size of the leaves of successorPlan should be 1");
+                }
+                PhysicalOperator leafOfSuccessPlan = leavesOfSuccessPlan.get(0);
+                physicalToRDD(sparkOperator, successorPlan, leafOfSuccessPlan, operatorKeysOfAllPreds);
+            }
+        } else {
+            RDDConverter converter = convertMap.get(physicalOperator.getClass());
+            if (converter == null) {
+                throw new IllegalArgumentException(
+                        "Pig on Spark does not support Physical Operator: " + physicalOperator);
+            }
+
+            LOG.info("Converting operator "
+                    + physicalOperator.getClass().getSimpleName() + " "
+                    + physicalOperator);
+            List<RDD<Tuple>> allPredRDDs = sortPredecessorRDDs(operatorKeysOfAllPreds);
+
+            if (converter instanceof FRJoinConverter) {
+                setReplicatedInputs(physicalOperator, (FRJoinConverter) converter);
+            }
+
+            if (sparkOperator.isSkewedJoin() && converter instanceof SkewedJoinConverter) {
+                SkewedJoinConverter skewedJoinConverter = (SkewedJoinConverter) converter;
+                skewedJoinConverter.setSkewedJoinPartitionFile(sparkOperator.getSkewedJoinPartitionFile());
+            }
+            adjustRuntimeParallelismForSkewedJoin(physicalOperator, sparkOperator, allPredRDDs);
+            nextRDD = converter.convert(allPredRDDs, physicalOperator);
+
+            if (nextRDD == null) {
+                throw new IllegalArgumentException(
+                        "RDD should not be null after PhysicalOperator: "
+                                + physicalOperator);
+            }
+
+            physicalOpRdds.put(physicalOperator.getOperatorKey(), nextRDD);
+        }
+    }
+
+    private void setReplicatedInputs(PhysicalOperator physicalOperator, FRJoinConverter converter) {
+        Set<String> replicatedInputs = new HashSet<>();
+        for (PhysicalOperator operator : physicalOperator.getInputs()) {
+            if (operator instanceof POBroadcastSpark) {
+                replicatedInputs.add(((POBroadcastSpark) operator).getBroadcastedVariableName());
+            }
+        }
+        converter.setReplicatedInputs(replicatedInputs);
+    }
+
+    private List<PhysicalOperator> getPredecessors(PhysicalPlan plan, PhysicalOperator op) {
+        List preds = null;
+        if (!(op instanceof POJoinGroupSpark)) {
+            preds = plan.getPredecessors(op);
+            if (preds != null && preds.size() > 1 && !(op instanceof POSkewedJoin)) {
+                Collections.sort(preds);
+            }
+        } else {
+            //For POJoinGroupSpark, we could not use plan.getPredecessors(op)+ sort to get
+            //the predecessors with correct order, more detail see JoinOptimizerSpark#restructSparkOp
+            preds = ((POJoinGroupSpark) op).getPredecessors();
+        }
+        return preds;
+    }
+
+    //get all rdds of predecessors sorted by the OperatorKey
+    private List<RDD<Tuple>> sortPredecessorRDDs(LinkedHashSet <OperatorKey> operatorKeysOfAllPreds) {
+        List<RDD<Tuple>> predecessorRDDs = Lists.newArrayList();
+        for (OperatorKey operatorKeyOfAllPred : operatorKeysOfAllPreds) {
+            predecessorRDDs.add(physicalOpRdds.get(operatorKeyOfAllPred));
+        }
+        return predecessorRDDs;
+    }
+
+    //deal special cases containing operators with multiple predecessors when multiquery is enabled to get the predecessors of specified
+    // physicalOp in previous SparkOp(see PIG-4675)
+    private void addPredsFromPrevoiousSparkOp(SparkOperator sparkOperator, PhysicalOperator physicalOperator, Set<OperatorKey> operatorKeysOfPredecessors) {
+        // the relationship is stored in sparkOperator.getMultiQueryOptimizeConnectionItem()
+        List<OperatorKey> predOperatorKeys = sparkOperator.getMultiQueryOptimizeConnectionItem().get(physicalOperator.getOperatorKey());
+        if (predOperatorKeys != null) {
+            for (OperatorKey predOperator : predOperatorKeys) {
+                LOG.debug(String.format("add predecessor(OperatorKey:%s) for OperatorKey:%s", predOperator, physicalOperator.getOperatorKey()));
+                operatorKeysOfPredecessors.add(predOperator);
+            }
+        }
+    }
+
+    /**
+     * In Spark, currently only async actions return job id. There is no async
+     * equivalent of actions like saveAsNewAPIHadoopFile()
+     * <p/>
+     * The only other way to get a job id is to register a "job group ID" with
+     * the spark context and request all job ids corresponding to that job group
+     * via getJobIdsForGroup.
+     * <p/>
+     * However getJobIdsForGroup does not guarantee the order of the elements in
+     * it's result.
+     * <p/>
+     * This method simply returns the previously unseen job ids.
+     *
+     * @param seenJobIDs job ids in the job group that are already seen
+     * @return Spark job ids not seen before
+     */
+    private List<Integer> getJobIDs(Set<Integer> seenJobIDs) {
+        Set<Integer> groupjobIDs = new HashSet<Integer>(
+                Arrays.asList(ArrayUtils.toObject(sparkContext.statusTracker()
+                        .getJobIdsForGroup(jobGroupID))));
+        groupjobIDs.removeAll(seenJobIDs);
+        List<Integer> unseenJobIDs = new ArrayList<Integer>(groupjobIDs);
+        seenJobIDs.addAll(unseenJobIDs);
+        return unseenJobIDs;
+    }
+
+
+    /**
+     * if the parallelism of skewed join is NOT specified by user in the script when sampling,
+     * set a default parallelism for sampling
+     *
+     * @param physicalOperator
+     * @param sparkOperator
+     * @param allPredRDDs
+     * @throws VisitorException
+     */
+    private void adjustRuntimeParallelismForSkewedJoin(PhysicalOperator physicalOperator,
+                                                       SparkOperator sparkOperator,
+                                                       List<RDD<Tuple>> allPredRDDs) throws VisitorException {
+        // We need to calculate the final number of reducers of the next job (skew-join)
+        // adjust parallelism of ConstantExpression
+        if (sparkOperator.isSampler() && sparkPlan.getSuccessors(sparkOperator) != null
+                && physicalOperator instanceof POPoissonSampleSpark) {
+            // set the runtime #reducer of the next job as the #partition
+
+            int defaultParallelism = SparkPigContext.get().getParallelism(allPredRDDs, physicalOperator);
+
+            ParallelConstantVisitor visitor =
+                    new ParallelConstantVisitor(sparkOperator.physicalPlan, defaultParallelism);
+            visitor.visit();
+        }
+    }
+
+    /**
+     * here, we don't reuse MR/Tez's ParallelConstantVisitor
+     * To automatic adjust reducer parallelism for skewed join, we only adjust the
+     * ConstantExpression operator after POPoissionSampleSpark operator
+     */
+    private static class ParallelConstantVisitor extends PhyPlanVisitor {
+
+        private int rp;
+        private boolean replaced = false;
+        private boolean isAfterSampleOperator = false;
+
+        public ParallelConstantVisitor(PhysicalPlan plan, int rp) {
+            super(plan, new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(
+                    plan));
+            this.rp = rp;
+        }
+
+        @Override
+        public void visitConstant(ConstantExpression cnst) throws VisitorException {
+            if (isAfterSampleOperator && cnst.getRequestedParallelism() == -1) {
+                Object obj = cnst.getValue();
+                if (obj instanceof Integer) {
+                    if (replaced) {
+                        // sample job should have only one ConstantExpression
+                        throw new VisitorException("Invalid reduce plan: more " +
+                                "than one ConstantExpression found in sampling job");
+                    }
+                    cnst.setValue(rp);
+                    cnst.setRequestedParallelism(rp);
+                    replaced = true;
+                }
+            }
+        }
+
+        @Override
+        public void visitPoissonSample(POPoissonSample po) {
+            isAfterSampleOperator = true;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/JobMetricsListener.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/JobMetricsListener.java
new file mode 100644
index 0000000..f813412
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/JobMetricsListener.java
@@ -0,0 +1,227 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.spark.executor.TaskMetrics;
+import org.apache.spark.scheduler.SparkListener;
+import org.apache.spark.scheduler.SparkListenerApplicationEnd;
+import org.apache.spark.scheduler.SparkListenerApplicationStart;
+import org.apache.spark.scheduler.SparkListenerBlockManagerAdded;
+import org.apache.spark.scheduler.SparkListenerBlockManagerRemoved;
+import org.apache.spark.scheduler.SparkListenerBlockUpdated;
+import org.apache.spark.scheduler.SparkListenerEnvironmentUpdate;
+import org.apache.spark.scheduler.SparkListenerExecutorAdded;
+import org.apache.spark.scheduler.SparkListenerExecutorMetricsUpdate;
+import org.apache.spark.scheduler.SparkListenerExecutorRemoved;
+import org.apache.spark.scheduler.SparkListenerJobEnd;
+import org.apache.spark.scheduler.SparkListenerJobStart;
+import org.apache.spark.scheduler.SparkListenerStageCompleted;
+import org.apache.spark.scheduler.SparkListenerStageSubmitted;
+import org.apache.spark.scheduler.SparkListenerTaskEnd;
+import org.apache.spark.scheduler.SparkListenerTaskGettingResult;
+import org.apache.spark.scheduler.SparkListenerTaskStart;
+import org.apache.spark.scheduler.SparkListenerUnpersistRDD;
+
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+public class JobMetricsListener implements SparkListener {
+
+    private static final Log LOG = LogFactory.getLog(JobMetricsListener.class);
+
+    private final Map<Integer, int[]> jobIdToStageId = Maps.newHashMap();
+    private final Map<Integer, Integer> stageIdToJobId = Maps.newHashMap();
+    private final Map<Integer, Map<String, List<TaskMetrics>>> allJobMetrics = Maps.newHashMap();
+    private final Set<Integer> finishedJobIds = Sets.newHashSet();
+
+    @Override
+    public void onStageCompleted(SparkListenerStageCompleted stageCompleted) {
+//        uncomment and remove the code onTaskEnd until we fix PIG-5157. It is better to update taskMetrics of stage when stage completes
+//        if we update taskMetrics in onTaskEnd(), it consumes lot of memory.
+//        int stageId = stageCompleted.stageInfo().stageId();
+//        int stageAttemptId = stageCompleted.stageInfo().attemptId();
+//        String stageIdentifier = stageId + "_" + stageAttemptId;
+//        Integer jobId = stageIdToJobId.get(stageId);
+//        if (jobId == null) {
+//            LOG.warn("Cannot find job id for stage[" + stageId + "].");
+//        } else {
+//            Map<String, List<TaskMetrics>> jobMetrics = allJobMetrics.get(jobId);
+//            if (jobMetrics == null) {
+//                jobMetrics = Maps.newHashMap();
+//                allJobMetrics.put(jobId, jobMetrics);
+//            }
+//            List<TaskMetrics> stageMetrics = jobMetrics.get(stageIdentifier);
+//            if (stageMetrics == null) {
+//                stageMetrics = Lists.newLinkedList();
+//                jobMetrics.put(stageIdentifier, stageMetrics);
+//            }
+//            // uncomment until we fix PIG-5157. after we upgrade to spark2.0 StageInfo().taskMetrics() api is available
+//            // stageMetrics.add(stageCompleted.stageInfo().taskMetrics());
+//        }
+    }
+
+    @Override
+    public void onStageSubmitted(SparkListenerStageSubmitted stageSubmitted) {
+
+    }
+
+    @Override
+    public void onTaskStart(SparkListenerTaskStart taskStart) {
+
+    }
+
+    @Override
+    public void onTaskGettingResult(SparkListenerTaskGettingResult taskGettingResult) {
+
+    }
+
+    @Override
+    public void onExecutorRemoved(SparkListenerExecutorRemoved executorRemoved) {
+
+    }
+
+    @Override
+    public void onExecutorAdded(SparkListenerExecutorAdded executorAdded) {
+
+    }
+
+    @Override
+    public void onBlockUpdated(SparkListenerBlockUpdated blockUpdated){
+
+    }
+
+    @Override
+    public synchronized void onTaskEnd(SparkListenerTaskEnd taskEnd) {
+        int stageId = taskEnd.stageId();
+        int stageAttemptId = taskEnd.stageAttemptId();
+        String stageIdentifier = stageId + "_" + stageAttemptId;
+        Integer jobId = stageIdToJobId.get(stageId);
+        if (jobId == null) {
+            LOG.warn("Cannot find job id for stage[" + stageId + "].");
+        } else {
+            Map<String, List<TaskMetrics>> jobMetrics = allJobMetrics.get(jobId);
+            if (jobMetrics == null) {
+                jobMetrics = Maps.newHashMap();
+                allJobMetrics.put(jobId, jobMetrics);
+            }
+            List<TaskMetrics> stageMetrics = jobMetrics.get(stageIdentifier);
+            if (stageMetrics == null) {
+                stageMetrics = Lists.newLinkedList();
+                jobMetrics.put(stageIdentifier, stageMetrics);
+            }
+            stageMetrics.add(taskEnd.taskMetrics());
+        }
+    }
+
+    @Override
+    public synchronized void onJobStart(SparkListenerJobStart jobStart) {
+        int jobId = jobStart.jobId();
+        int size = jobStart.stageIds().size();
+        int[] intStageIds = new int[size];
+        for (int i = 0; i < size; i++) {
+            Integer stageId = (Integer) jobStart.stageIds().apply(i);
+            intStageIds[i] = stageId;
+            stageIdToJobId.put(stageId, jobId);
+        }
+        jobIdToStageId.put(jobId, intStageIds);
+    }
+
+    @Override
+    public synchronized void onJobEnd(SparkListenerJobEnd jobEnd) {
+        finishedJobIds.add(jobEnd.jobId());
+        notify();
+    }
+
+    @Override
+    public void onEnvironmentUpdate(SparkListenerEnvironmentUpdate environmentUpdate) {
+
+    }
+
+    @Override
+    public void onBlockManagerAdded(SparkListenerBlockManagerAdded blockManagerAdded) {
+
+    }
+
+    @Override
+    public void onBlockManagerRemoved(SparkListenerBlockManagerRemoved blockManagerRemoved) {
+
+    }
+
+    @Override
+    public void onUnpersistRDD(SparkListenerUnpersistRDD unpersistRDD) {
+
+    }
+
+    @Override
+    public void onApplicationStart(SparkListenerApplicationStart applicationStart) {
+
+    }
+
+    @Override
+    public void onApplicationEnd(SparkListenerApplicationEnd applicationEnd) {
+
+    }
+
+    @Override
+    public void onExecutorMetricsUpdate(SparkListenerExecutorMetricsUpdate executorMetricsUpdate) {
+
+    }
+
+
+    public synchronized Map<String, List<TaskMetrics>> getJobMetric(int jobId) {
+        return allJobMetrics.get(jobId);
+    }
+
+    public synchronized boolean waitForJobToEnd(int jobId) throws InterruptedException {
+        if (finishedJobIds.contains(jobId)) {
+            finishedJobIds.remove(jobId);
+            return true;
+        }
+
+        wait();
+        return false;
+    }
+
+    public synchronized void cleanup(int jobId) {
+        allJobMetrics.remove(jobId);
+        jobIdToStageId.remove(jobId);
+        Iterator<Map.Entry<Integer, Integer>> iterator = stageIdToJobId.entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<Integer, Integer> entry = iterator.next();
+            if (entry.getValue() == jobId) {
+                iterator.remove();
+            }
+        }
+    }
+
+    public synchronized void reset() {
+        stageIdToJobId.clear();
+        jobIdToStageId.clear();
+        allJobMetrics.clear();
+        finishedJobIds.clear();
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/KryoSerializer.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/KryoSerializer.java
new file mode 100644
index 0000000..93b8155
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/KryoSerializer.java
@@ -0,0 +1,70 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.mapred.JobConf;
+
+import com.esotericsoftware.kryo.io.Input;
+import com.esotericsoftware.kryo.io.Output;
+
+/**
+ * Pulled class from Hive on Spark
+ */
+public class KryoSerializer {
+    private static final Log LOG = LogFactory.getLog(KryoSerializer.class);
+
+    public static byte[] serializeJobConf(JobConf jobConf) {
+        ByteArrayOutputStream out = new ByteArrayOutputStream();
+        try {
+            jobConf.write(new DataOutputStream(out));
+        } catch (IOException e) {
+            LOG.error("Error serializing job configuration", e);
+            return null;
+        } finally {
+            try {
+                out.close();
+            } catch (IOException e) {
+                LOG.error("Error closing output stream", e);
+            }
+        }
+
+        return out.toByteArray();
+
+    }
+
+    public static JobConf deserializeJobConf(byte[] buffer) {
+        JobConf conf = new JobConf();
+        try {
+            conf.readFields(new DataInputStream(new ByteArrayInputStream(buffer)));
+        } catch (IOException e) {
+            LOG.error("Error de-serializing job configuration");
+            return null;
+        }
+        return conf;
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/MapReducePartitionerWrapper.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/MapReducePartitionerWrapper.java
new file mode 100644
index 0000000..f6837a1
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/MapReducePartitionerWrapper.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.lang.reflect.Method;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.util.ReflectionUtils;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.IndexedKey;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.io.PigNullableWritable;
+import org.apache.spark.Partitioner;
+
+/**
+ * Spark Partitioner that wraps a custom partitioner that implements
+ * org.apache.hadoop.mapreduce.Partitioner interface.
+ *
+ * Since Spark's shuffle API takes a different parititioner class
+ * (@see org.apache.spark.Partitioner) compared to MapReduce, we need to
+ * wrap custom partitioners written for MapReduce inside this Spark Partitioner.
+ *
+ * MR Custom partitioners are expected to implement getPartition() with
+ * specific arguments:
+ *   public int getPartition(PigNullableWritable key, Writable value, int numPartitions)
+ * For an example of such a partitioner,
+ * @see org.apache.pig.test.utils.SimpleCustomPartitioner
+ */
+public class MapReducePartitionerWrapper extends Partitioner {
+    private static final Log LOG = LogFactory.getLog(MapReducePartitionerWrapper.class);
+
+    private int numPartitions;
+    private String partitionerName;
+    // MR's Partitioner interface is not serializable.
+    // And since it is not serializable, it cannot be initialized in the constructor
+    // (in Spark's DAG scheduler thread in Spark Driver),
+    // To workaround this, It will be lazily initialized inside the map task
+    // (Executor thread) first time that getPartitions() gets called.
+    transient private org.apache.hadoop.mapreduce.Partitioner<PigNullableWritable, Writable>
+            mapredPartitioner = null;
+    transient private Method getPartitionMethod = null;
+
+    public MapReducePartitionerWrapper(String partitionerName,
+                      int numPartitions) {
+        if (partitionerName == null) {
+            throw new RuntimeException("MapReduce Partitioner cannot be null.");
+        }
+
+        this.partitionerName = partitionerName;
+        this.numPartitions = numPartitions;
+    }
+
+    public int numPartitions() {
+        return numPartitions;
+    }
+
+    public int getPartition(final Object key) {
+        try {
+
+            PigNullableWritable writeableKey = new PigNullableWritable() {
+                public Object getValueAsPigType() {
+                    if (key instanceof IndexedKey) {
+                        IndexedKey indexedKey = (IndexedKey) key;
+                        this.setIndex(indexedKey.getIndex());
+                        return indexedKey.getKey();
+                    } else {
+                        return key;
+                    }
+                }
+            };
+
+
+            // Lazy initialization
+            // Synchronized because multiple (map) tasks in the same Spark Executor
+            // may call getPartition, attempting to initialize at the same time.
+            if (mapredPartitioner == null) {
+                synchronized (this) {
+                    // check again for race condition
+                    if (mapredPartitioner == null) {
+                        Class<?> mapredPartitionerClass =
+                                PigContext.resolveClassName(partitionerName);
+                        Configuration conf = new Configuration();
+                        mapredPartitioner = (org.apache.hadoop.mapreduce.Partitioner<PigNullableWritable, Writable>)
+                                ReflectionUtils.newInstance(mapredPartitionerClass, conf);
+                        getPartitionMethod = mapredPartitionerClass.getMethod(
+                                "getPartition",
+                                PigNullableWritable.class,
+                                org.apache.hadoop.io.Writable.class,
+                                int.class);
+                    }
+                }
+            }
+
+            // MR Parititioner getPartition takes a value argument as well, but
+            // Spark's Partitioner only accepts the key argument.
+            // In practice, MR Partitioners ignore the value. However, it's
+            // possible that some don't.
+            // TODO: We could handle this case by packaging the value inside the
+            // key (conditioned on some config option, since this will balloon
+            // memory usage). PIG-4575.
+            int partition = (Integer) getPartitionMethod.invoke(mapredPartitioner,
+                    writeableKey, null, numPartitions);
+
+            return partition;
+        } catch (Exception e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkEngineConf.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkEngineConf.java
new file mode 100644
index 0000000..ecf6c06
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkEngineConf.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Properties;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.UDFContext;
+
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+
+/**
+ * The object of SparkEngineConf is to solve the initialization problem of PigContext.properties.get("udf.import.list"),
+ * UDFContext#udfConfs, UDFContext#clientSysProps in spark mode. These variables can not be
+ * serialized because they are ThreadLocal variables. In MR mode, they are serialized in JobConfiguration
+ * in JobControlCompiler#getJob and deserialized by JobConfiguration in PigGenericMapBase#setup. But there is no
+ * setup() in spark like what in mr, so these variables can be correctly deserialized before spark programs call them.
+ * Here we use following solution to solve:
+ * these variables are saved in SparkEngineConf#writeObject and available and then initialized
+ * in SparkEngineConf#readObject in spark executor thread.
+ */
+public class SparkEngineConf implements Serializable {
+
+    private static final Log LOG = LogFactory.getLog(SparkEngineConf.class);
+    private static String SPARK_UDF_IMPORT_LIST = "pig.spark.udf.import.list";
+    private static String SPARK_UDFCONTEXT_UDFCONFS = "pig.spark.udfcontext.udfConfs";
+    private static String SPARK_UDFCONTEXT_CLIENTSYSPROPS = "pig.spark.udfcontext.clientSysProps";
+
+    private Properties properties = new Properties();
+
+    public SparkEngineConf() {
+    }
+
+    private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+        ArrayList<String> udfImportList = (ArrayList<String>) in.readObject();
+        PigContext.setPackageImportList(udfImportList);
+        String udfConfsStr = (String) in.readObject();
+        String clientSysPropsStr = (String) in.readObject();
+        UDFContext.getUDFContext().deserializeForSpark(udfConfsStr, clientSysPropsStr);
+    }
+
+    private void writeObject(ObjectOutputStream out) throws IOException {
+        ArrayList<String> udfImportList = Lists.newArrayList(Splitter.on(",").split(properties.getProperty(SPARK_UDF_IMPORT_LIST)));
+        out.writeObject(udfImportList);
+        //2 threads call SparkEngineConf#writeObject
+        //In main thread: SparkLauncher#initialize->SparkUtil#newJobConf
+        //                ->ObjectSerializer#serialize-> SparkEngineConf#writeObject
+        //In dag-scheduler-event-loop thread: DAGScheduler.submitMissingTasks->JavaSerializationStream.writeObject
+        //
+        //In main thread,UDFContext#getUDFContext is not empty, we store UDFContext#udfConfs and UDFContext#clientSysProps
+        //into properties and serialize them.
+        //In dag-scheduler-event-loop thread, UDFContext#getUDFContext is empty, we get value of UDFContext#udfConfs and UDFContext#clientSysProps
+        //from properties and serialize them.
+        if (!UDFContext.getUDFContext().isUDFConfEmpty()) {
+            //In SparkUtil#newJobConf(), sparkEngineConf is serialized in job configuration and will call
+            //SparkEngineConf#writeObject(at this time UDFContext#udfConfs and UDFContext#clientSysProps is not null)
+            //later spark will call JavaSerializationStream.writeObject to serialize all objects when submit spark
+            //jobs(at that time, UDFContext#udfConfs and UDFContext#clientSysProps is null so we need to save their
+            //value in SparkEngineConf#properties after these two variables are correctly initialized in
+            //SparkUtil#newJobConf, More detailed see PIG-4920
+            String udfConfsStr = UDFContext.getUDFContext().serialize();
+            String clientSysPropsStr = ObjectSerializer.serialize(UDFContext.getUDFContext().getClientSystemProps());
+            this.properties.setProperty(SPARK_UDFCONTEXT_UDFCONFS, udfConfsStr);
+            this.properties.setProperty(SPARK_UDFCONTEXT_CLIENTSYSPROPS, clientSysPropsStr);
+            out.writeObject(udfConfsStr);
+            out.writeObject(clientSysPropsStr);
+        } else {
+            out.writeObject(this.properties.getProperty(SPARK_UDFCONTEXT_UDFCONFS));
+            out.writeObject(this.properties.getProperty(SPARK_UDFCONTEXT_CLIENTSYSPROPS));
+        }
+    }
+
+    public void setSparkUdfImportListStr(String sparkUdfImportListStr) {
+        this.properties.setProperty(SPARK_UDF_IMPORT_LIST, sparkUdfImportListStr);
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkExecType.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkExecType.java
new file mode 100644
index 0000000..67ecce0
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkExecType.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.util.Properties;
+
+import org.apache.pig.ExecType;
+import org.apache.pig.backend.executionengine.ExecutionEngine;
+import org.apache.pig.impl.PigContext;
+
+public class SparkExecType implements ExecType {
+
+    private static final long serialVersionUID = 1L;
+    private static final String mode = "SPARK";
+
+    @Override
+    public boolean accepts(Properties properties) {
+        String execTypeSpecified = properties.getProperty("exectype", "")
+                .toUpperCase();
+        if (execTypeSpecified.equals(mode))
+            return true;
+        return false;
+    }
+
+    @Override
+    public ExecutionEngine getExecutionEngine(PigContext pigContext) {
+        return new SparkExecutionEngine(pigContext);
+    }
+
+    @Override
+    public Class<? extends ExecutionEngine> getExecutionEngineClass() {
+        return SparkExecutionEngine.class;
+    }
+
+    @Override
+    public boolean isLocal() {
+        return false;
+    }
+
+    @Override
+    public String name() {
+        return "SPARK";
+    }
+
+    public String toString() {
+        return name();
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkExecutionEngine.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkExecutionEngine.java
new file mode 100644
index 0000000..3748b24
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkExecutionEngine.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.util.UUID;
+
+import org.apache.pig.backend.hadoop.executionengine.HExecutionEngine;
+import org.apache.pig.backend.hadoop.executionengine.spark.streaming.SparkExecutableManager;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.streaming.ExecutableManager;
+import org.apache.pig.tools.pigstats.PigStats;
+import org.apache.pig.tools.pigstats.ScriptState;
+import org.apache.pig.tools.pigstats.spark.SparkPigStats;
+import org.apache.pig.tools.pigstats.spark.SparkScriptState;
+
+public class SparkExecutionEngine extends HExecutionEngine {
+
+    public SparkExecutionEngine(PigContext pigContext) {
+        super(pigContext);
+        this.launcher = new SparkLauncher();
+    }
+
+    @Override
+    public ScriptState instantiateScriptState() {
+        SparkScriptState ss = new SparkScriptState(UUID.randomUUID().toString());
+        ss.setPigContext(pigContext);
+        return ss;
+    }
+
+    @Override
+    public ExecutableManager getExecutableManager() {
+        return new SparkExecutableManager();
+    }
+
+    @Override
+    public PigStats instantiatePigStats() {
+        return new SparkPigStats();
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
new file mode 100644
index 0000000..391e83f
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLauncher.java
@@ -0,0 +1,735 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintStream;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.UUID;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.TransformerException;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.PigException;
+import org.apache.pig.PigWarning;
+import org.apache.pig.backend.BackendException;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.backend.hadoop.executionengine.Launcher;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCollectedGroup;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCounter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODistinct;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoinSpark;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeCogroup;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPreCombinerLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PORank;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSkewedJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.BroadcastConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.CollectedGroupConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.CounterConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.DistinctConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.FRJoinConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.FilterConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.ForEachConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.GlobalRearrangeConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.JoinGroupSparkConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.LimitConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.LoadConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.LocalRearrangeConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.MergeCogroupConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.MergeJoinConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.PackageConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.PoissonSampleConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.RDDConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.RankConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.ReduceByConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.SkewedJoinConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.SortConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.SparkSampleSortConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.SplitConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.StoreConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.StreamConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.UnionConverter;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POJoinGroupSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POPoissonSampleSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POReduceBySpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POSampleSortSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.AccumulatorOptimizer;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.CombinerOptimizer;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.JoinGroupOptimizerSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.MultiQueryOptimizerSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.NoopFilterRemover;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.ParallelismSetter;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.SecondaryKeyOptimizerSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.DotSparkPrinter;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkCompiler;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkPOPackageAnnotator;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkPrinter;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.XMLSparkPrinter;
+import org.apache.pig.data.SchemaTupleBackend;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.util.JarManager;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.UDFContext;
+import org.apache.pig.impl.util.Utils;
+import org.apache.pig.tools.pigstats.OutputStats;
+import org.apache.pig.tools.pigstats.PigStats;
+import org.apache.pig.tools.pigstats.spark.SparkCounterGroup;
+import org.apache.pig.tools.pigstats.spark.SparkCounters;
+import org.apache.pig.tools.pigstats.spark.SparkPigStats;
+import org.apache.pig.tools.pigstats.spark.SparkPigStatusReporter;
+import org.apache.spark.SparkConf;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.scheduler.JobLogger;
+import org.apache.spark.scheduler.StatsReportListener;
+
+import com.google.common.base.Joiner;
+
+/**
+ * Main class that launches pig for Spark
+ */
+public class SparkLauncher extends Launcher {
+
+    private static final Log LOG = LogFactory.getLog(SparkLauncher.class);
+
+    // Our connection to Spark. It needs to be static so that it can be reused
+    // across jobs, because a
+    // new SparkLauncher gets created for each job.
+    private static JavaSparkContext sparkContext = null;
+    private static JobMetricsListener jobMetricsListener = new JobMetricsListener();
+    private String jobGroupID;
+    private PigContext pigContext = null;
+    private JobConf jobConf = null;
+    private String currentDirectoryPath = null;
+    private SparkEngineConf sparkEngineConf = new SparkEngineConf();
+    private static final String PIG_WARNING_FQCN = PigWarning.class.getCanonicalName();
+
+    @Override
+    public PigStats launchPig(PhysicalPlan physicalPlan, String grpName,
+                              PigContext pigContext) throws Exception {
+        if (LOG.isDebugEnabled())
+            LOG.debug(physicalPlan);
+        this.pigContext = pigContext;
+        initialize(physicalPlan);
+        SparkOperPlan sparkplan = compile(physicalPlan, pigContext);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug(sparkplan);
+        }
+        SparkPigStats sparkStats = (SparkPigStats) pigContext
+                .getExecutionEngine().instantiatePigStats();
+        sparkStats.initialize(pigContext, sparkplan, jobConf);
+        PigStats.start(sparkStats);
+
+        startSparkIfNeeded(pigContext);
+
+        jobGroupID = String.format("%s-%s",sparkContext.getConf().getAppId(),
+                UUID.randomUUID().toString());
+        jobConf.set(MRConfiguration.JOB_ID,jobGroupID);
+
+        sparkContext.setJobGroup(jobGroupID, "Pig query to Spark cluster",
+                false);
+        jobMetricsListener.reset();
+
+        this.currentDirectoryPath = Paths.get(".").toAbsolutePath()
+                .normalize().toString()
+                + "/";
+
+        new ParallelismSetter(sparkplan, jobConf).visit();
+
+        prepareSparkCounters(jobConf);
+
+        // Create conversion map, mapping between pig operator and spark convertor
+        Map<Class<? extends PhysicalOperator>, RDDConverter> convertMap
+                = new HashMap<Class<? extends PhysicalOperator>, RDDConverter>();
+        convertMap.put(POLoad.class, new LoadConverter(pigContext,
+                physicalPlan, sparkContext.sc(), jobConf, sparkEngineConf));
+        convertMap.put(POStore.class, new StoreConverter(jobConf));
+        convertMap.put(POForEach.class, new ForEachConverter(jobConf));
+        convertMap.put(POFilter.class, new FilterConverter());
+        convertMap.put(POPackage.class, new PackageConverter());
+        convertMap.put(POLocalRearrange.class, new LocalRearrangeConverter());
+        convertMap.put(POGlobalRearrangeSpark.class, new GlobalRearrangeConverter());
+        convertMap.put(POJoinGroupSpark.class, new JoinGroupSparkConverter());
+        convertMap.put(POLimit.class, new LimitConverter());
+        convertMap.put(PODistinct.class, new DistinctConverter());
+        convertMap.put(POUnion.class, new UnionConverter(sparkContext.sc()));
+        convertMap.put(POSort.class, new SortConverter());
+        convertMap.put(POSplit.class, new SplitConverter());
+        convertMap.put(POSkewedJoin.class, new SkewedJoinConverter());
+        convertMap.put(POMergeJoin.class, new MergeJoinConverter());
+        convertMap.put(POCollectedGroup.class, new CollectedGroupConverter());
+        convertMap.put(POCounter.class, new CounterConverter());
+        convertMap.put(PORank.class, new RankConverter());
+        convertMap.put(POStream.class, new StreamConverter());
+        convertMap.put(POFRJoinSpark.class, new FRJoinConverter());
+        convertMap.put(POMergeCogroup.class, new MergeCogroupConverter());
+        convertMap.put(POReduceBySpark.class, new ReduceByConverter());
+        convertMap.put(POPreCombinerLocalRearrange.class, new LocalRearrangeConverter());
+        convertMap.put(POBroadcastSpark.class, new BroadcastConverter(sparkContext));
+        convertMap.put(POSampleSortSpark.class, new SparkSampleSortConverter());
+        convertMap.put(POPoissonSampleSpark.class, new PoissonSampleConverter());
+        //Print SPARK plan before launching if needed
+        Configuration conf = ConfigurationUtil.toConfiguration(pigContext.getProperties());
+        if (conf.getBoolean(PigConfiguration.PIG_PRINT_EXEC_PLAN, false)) {
+            LOG.info(sparkplan);
+        }
+        uploadResources(sparkplan);
+
+        new JobGraphBuilder(sparkplan, convertMap, sparkStats, sparkContext, jobMetricsListener, jobGroupID, jobConf, pigContext).visit();
+        cleanUpSparkJob(sparkStats);
+        sparkStats.finish();
+        resetUDFContext();
+        return sparkStats;
+    }
+
+    private void resetUDFContext() {
+        UDFContext.getUDFContext().addJobConf(null);
+    }
+
+    private void uploadResources(SparkOperPlan sparkPlan) throws IOException {
+        addFilesToSparkJob(sparkPlan);
+        addJarsToSparkJob(sparkPlan);
+    }
+
+    private void optimize(SparkOperPlan plan, PigContext pigContext) throws IOException {
+
+        Configuration conf = ConfigurationUtil.toConfiguration(pigContext.getProperties());
+
+        // Should be the first optimizer as it introduces new operators to the plan.
+        boolean noCombiner = conf.getBoolean(PigConfiguration.PIG_EXEC_NO_COMBINER, false);
+        if (!pigContext.inIllustrator && !noCombiner)  {
+            CombinerOptimizer combinerOptimizer = new CombinerOptimizer(plan);
+            combinerOptimizer.visit();
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("After combiner optimization:");
+                LOG.debug(plan);
+            }
+        }
+
+        boolean noSecondaryKey = conf.getBoolean(PigConfiguration.PIG_EXEC_NO_SECONDARY_KEY, false);
+        if (!pigContext.inIllustrator && !noSecondaryKey) {
+            SecondaryKeyOptimizerSpark skOptimizer = new SecondaryKeyOptimizerSpark(plan);
+            skOptimizer.visit();
+        }
+
+        boolean isAccum = conf.getBoolean(PigConfiguration.PIG_OPT_ACCUMULATOR, true);
+        if (isAccum) {
+            AccumulatorOptimizer accum = new AccumulatorOptimizer(plan);
+            accum.visit();
+        }
+
+        // removes the filter(constant(true)) operators introduced by
+        // splits.
+        NoopFilterRemover fRem = new NoopFilterRemover(plan);
+        fRem.visit();
+
+        boolean isMultiQuery = conf.getBoolean(PigConfiguration.PIG_OPT_MULTIQUERY, true);
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Before multiquery optimization:");
+            LOG.debug(plan);
+        }
+
+        if (isMultiQuery) {
+            // reduces the number of SparkOpers in the Spark plan generated
+            // by multi-query (multi-store) script.
+            MultiQueryOptimizerSpark mqOptimizer = new MultiQueryOptimizerSpark(plan);
+            mqOptimizer.visit();
+        }
+
+        //since JoinGroupOptimizerSpark modifies the plan and collapses LRA+GLA+PKG into POJoinGroupSpark while
+        //CombinerOptimizer collapses GLA+PKG into ReduceBy, so if JoinGroupOptimizerSpark first, the spark plan will be
+        //changed and not suitable for CombinerOptimizer.More detail see PIG-4797
+        JoinGroupOptimizerSpark joinOptimizer = new JoinGroupOptimizerSpark(plan);
+        joinOptimizer.visit();
+
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("After multiquery optimization:");
+            LOG.debug(plan);
+        }
+    }
+
+    private void cleanUpSparkJob(SparkPigStats sparkStats) throws ExecException {
+        LOG.info("Clean up Spark Job");
+        boolean isLocal = System.getenv("SPARK_MASTER") != null ? System
+                .getenv("SPARK_MASTER").equalsIgnoreCase("LOCAL") : true;
+        if (isLocal) {
+            String shipFiles = pigContext.getProperties().getProperty(
+                    "pig.streaming.ship.files");
+            if (shipFiles != null) {
+                for (String file : shipFiles.split(",")) {
+                    File shipFile = new File(file);
+                    File deleteFile = new File(currentDirectoryPath + "/"
+                            + shipFile.getName());
+                    if (deleteFile.exists()) {
+                        LOG.info(String.format("Delete ship file result: %b",
+                                deleteFile.delete()));
+                    }
+                }
+            }
+            String cacheFiles = pigContext.getProperties().getProperty(
+                    "pig.streaming.cache.files");
+            if (cacheFiles != null) {
+                for (String file : cacheFiles.split(",")) {
+                    String fileName = extractFileName(file.trim());
+                    File deleteFile = new File(currentDirectoryPath + "/"
+                            + fileName);
+                    if (deleteFile.exists()) {
+                        LOG.info(String.format("Delete cache file result: %b",
+                                deleteFile.delete()));
+                    }
+                }
+            }
+        }
+
+        // run cleanup for all of the stores
+        for (OutputStats output : sparkStats.getOutputStats()) {
+            POStore store = output.getPOStore();
+            try {
+                if (!output.isSuccessful()) {
+                    store.getStoreFunc().cleanupOnFailure(
+                            store.getSFile().getFileName(),
+                            Job.getInstance(output.getConf()));
+                } else {
+                    store.getStoreFunc().cleanupOnSuccess(
+                            store.getSFile().getFileName(),
+                            Job.getInstance(output.getConf()));
+                }
+            } catch (IOException e) {
+                throw new ExecException(e);
+            } catch (AbstractMethodError nsme) {
+                // Just swallow it.  This means we're running against an
+                // older instance of a StoreFunc that doesn't implement
+                // this method.
+            }
+        }
+    }
+
+    private void addFilesToSparkJob(SparkOperPlan sparkPlan) throws IOException {
+        LOG.info("Add files Spark Job");
+        String shipFiles = pigContext.getProperties().getProperty(
+                "pig.streaming.ship.files");
+        shipFiles(shipFiles);
+        String cacheFiles = pigContext.getProperties().getProperty(
+                "pig.streaming.cache.files");
+        cacheFiles(cacheFiles);
+        addUdfResourcesToSparkJob(sparkPlan);
+    }
+
+    private void addUdfResourcesToSparkJob(SparkOperPlan sparkPlan) throws IOException {
+        SparkPOUserFuncVisitor sparkPOUserFuncVisitor = new SparkPOUserFuncVisitor(sparkPlan);
+        sparkPOUserFuncVisitor.visit();
+        Joiner joiner = Joiner.on(",");
+        String shipFiles = joiner.join(sparkPOUserFuncVisitor.getShipFiles());
+        shipFiles(shipFiles);
+        String cacheFiles = joiner.join(sparkPOUserFuncVisitor.getCacheFiles());
+        cacheFiles(cacheFiles);
+    }
+
+    private void shipFiles(String shipFiles)
+            throws IOException {
+        if (shipFiles != null && !shipFiles.isEmpty()) {
+            for (String file : shipFiles.split(",")) {
+                File shipFile = new File(file.trim());
+                if (shipFile.exists()) {
+                    addResourceToSparkJobWorkingDirectory(shipFile,
+                            shipFile.getName(), ResourceType.FILE);
+                }
+            }
+        }
+    }
+
+    private void cacheFiles(String cacheFiles) throws IOException {
+        if (cacheFiles != null && !cacheFiles.isEmpty()) {
+            File tmpFolder = Files.createTempDirectory("cache").toFile();
+            tmpFolder.deleteOnExit();
+            for (String file : cacheFiles.split(",")) {
+                String fileName = extractFileName(file.trim());
+                if( fileName != null) {
+                    String fileUrl = extractFileUrl(file.trim());
+                    if( fileUrl != null) {
+                        Path src = new Path(fileUrl);
+                        File tmpFile = new File(tmpFolder, fileName);
+                        Path tmpFilePath = new Path(tmpFile.getAbsolutePath());
+                        FileSystem fs = tmpFilePath.getFileSystem(jobConf);
+                        //TODO:PIG-5241 Specify the hdfs path directly to spark and avoid the unnecessary download and upload in SparkLauncher.java
+                        fs.copyToLocalFile(src, tmpFilePath);
+                        tmpFile.deleteOnExit();
+                        LOG.info(String.format("CacheFile:%s", fileName));
+                        addResourceToSparkJobWorkingDirectory(tmpFile, fileName,
+                                ResourceType.FILE);
+                    }
+                }
+            }
+        }
+    }
+
+    public static enum ResourceType {
+        JAR,
+        FILE
+    }
+
+
+    private void addJarsToSparkJob(SparkOperPlan sparkPlan) throws IOException {
+        Set<String> allJars = new HashSet<String>();
+        LOG.info("Add default jars to Spark Job");
+        allJars.addAll(JarManager.getDefaultJars());
+        LOG.info("Add extra jars to Spark Job");
+        for (String scriptJar : pigContext.scriptJars) {
+            allJars.add(scriptJar);
+        }
+
+        LOG.info("Add udf jars to Spark Job");
+        UDFJarsFinder udfJarsFinder = new UDFJarsFinder(sparkPlan, pigContext);
+        udfJarsFinder.visit();
+        Set<String> udfJars = udfJarsFinder.getUdfJars();
+        for (String udfJar : udfJars) {
+            allJars.add(udfJar);
+        }
+
+        File scriptUDFJarFile = JarManager.createPigScriptUDFJar(pigContext);
+        if (scriptUDFJarFile != null) {
+            LOG.info("Add script udf jar to Spark job");
+            allJars.add(scriptUDFJarFile.getAbsolutePath().toString());
+        }
+
+        //Upload all jars to spark working directory
+        for (String jar : allJars) {
+            File jarFile = new File(jar);
+            addResourceToSparkJobWorkingDirectory(jarFile, jarFile.getName(),
+                    ResourceType.JAR);
+        }
+    }
+
+    private void addResourceToSparkJobWorkingDirectory(File resourcePath,
+                                                       String resourceName, ResourceType resourceType) throws IOException {
+        if (resourceType == ResourceType.JAR) {
+            LOG.info("Added jar " + resourceName);
+        } else {
+            LOG.info("Added file " + resourceName);
+        }
+        boolean isLocal = System.getenv("SPARK_MASTER") != null ? System
+                .getenv("SPARK_MASTER").equalsIgnoreCase("LOCAL") : true;
+        if (isLocal) {
+            File localFile = new File(currentDirectoryPath + "/"
+                    + resourceName);
+            if (resourcePath.getAbsolutePath().equals(localFile.getAbsolutePath())
+                    && resourcePath.exists()) {
+                return;
+            }
+            // When multiple threads start SparkLauncher, delete/copy actions should be in a critical section
+            synchronized(SparkLauncher.class) {
+                if (localFile.exists()) {
+                    LOG.info(String.format(
+                            "Jar file %s exists, ready to delete",
+                            localFile.getAbsolutePath()));
+                    localFile.delete();
+                } else {
+                    LOG.info(String.format("Jar file %s not exists,",
+                            localFile.getAbsolutePath()));
+                }
+                Files.copy(Paths.get(new Path(resourcePath.getAbsolutePath()).toString()),
+                    Paths.get(localFile.getAbsolutePath()));
+            }
+        } else {
+            if(resourceType == ResourceType.JAR){
+                sparkContext.addJar(resourcePath.toURI().toURL()
+                        .toExternalForm());
+            }else if( resourceType == ResourceType.FILE){
+                sparkContext.addFile(resourcePath.toURI().toURL()
+                        .toExternalForm());
+            }
+        }
+    }
+
+    private String extractFileName(String cacheFileUrl) {
+        String[] tmpAry = cacheFileUrl.split("#");
+        String fileName = tmpAry != null && tmpAry.length == 2 ? tmpAry[1]
+                : null;
+        return fileName;
+    }
+
+    private String extractFileUrl(String cacheFileUrl) {
+        String[] tmpAry = cacheFileUrl.split("#");
+        String fileName = tmpAry != null && tmpAry.length == 2 ? tmpAry[0]
+                : null;
+        return fileName;
+    }
+
+    public SparkOperPlan compile(PhysicalPlan physicalPlan,
+                                  PigContext pigContext) throws PlanException, IOException,
+            VisitorException {
+        SparkCompiler sparkCompiler = new SparkCompiler(physicalPlan,
+                pigContext);
+        sparkCompiler.compile();
+        sparkCompiler.connectSoftLink();
+        SparkOperPlan sparkPlan = sparkCompiler.getSparkPlan();
+
+        // optimize key - value handling in package
+        SparkPOPackageAnnotator pkgAnnotator = new SparkPOPackageAnnotator(
+                sparkPlan);
+        pkgAnnotator.visit();
+
+        optimize(sparkPlan, pigContext);
+        return sparkPlan;
+    }
+
+    /**
+     * Only one SparkContext may be active per JVM (SPARK-2243). When multiple threads start SparkLaucher,
+     * the static member sparkContext should be initialized only once
+     */
+    private static synchronized void startSparkIfNeeded(PigContext pc) throws PigException {
+        if (sparkContext == null) {
+            String master = null;
+            if (pc.getExecType().isLocal()) {
+                master = "local";
+            } else {
+                master = System.getenv("SPARK_MASTER");
+                if (master == null) {
+                    LOG.info("SPARK_MASTER not specified, using \"local\"");
+                    master = "local";
+                }
+            }
+
+            String sparkHome = System.getenv("SPARK_HOME");
+            if (!master.startsWith("local") && !master.equals("yarn-client")) {
+                // Check that we have the Mesos native library and Spark home
+                // are set
+                if (sparkHome == null) {
+                    System.err
+                            .println("You need to set SPARK_HOME to run on a Mesos cluster!");
+                    throw new PigException("SPARK_HOME is not set");
+                }
+            }
+
+            SparkConf sparkConf = new SparkConf();
+            Properties pigCtxtProperties = pc.getProperties();
+
+            sparkConf.setMaster(master);
+            sparkConf.setAppName(pigCtxtProperties.getProperty(PigContext.JOB_NAME,"pig"));
+            // On Spark 1.6, Netty file server doesn't allow adding the same file with the same name twice
+            // This is a problem for streaming using a script + explicit ship the same script combination (PIG-5134)
+            // HTTP file server doesn't have this restriction, it overwrites the file if added twice
+            String useNettyFileServer = pigCtxtProperties.getProperty(PigConfiguration.PIG_SPARK_USE_NETTY_FILESERVER, "false");
+            sparkConf.set("spark.rpc.useNettyFileServer", useNettyFileServer);
+
+            if (sparkHome != null && !sparkHome.isEmpty()) {
+                sparkConf.setSparkHome(sparkHome);
+            } else {
+                LOG.warn("SPARK_HOME is not set");
+            }
+
+            //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));
+                }
+            }
+
+            //see PIG-5200 why need to set spark.executor.userClassPathFirst as true
+            sparkConf.set("spark.executor.userClassPathFirst", "true");
+            checkAndConfigureDynamicAllocation(master, sparkConf);
+
+            sparkContext = new JavaSparkContext(sparkConf);
+            sparkContext.sc().addSparkListener(new StatsReportListener());
+            sparkContext.sc().addSparkListener(new JobLogger());
+            sparkContext.sc().addSparkListener(jobMetricsListener);
+        }
+    }
+
+    private static void checkAndConfigureDynamicAllocation(String master, SparkConf sparkConf) {
+        if (sparkConf.getBoolean("spark.dynamicAllocation.enabled", false)) {
+            if (!master.startsWith("yarn")) {
+                LOG.warn("Dynamic allocation is enabled, but " +
+                        "script isn't running on yarn. Ignoring ...");
+            }
+            if (!sparkConf.getBoolean("spark.shuffle.service.enabled", false)) {
+                LOG.info("Spark shuffle service is being enabled as dynamic " +
+                        "allocation is enabled");
+                sparkConf.set("spark.shuffle.service.enabled", "true");
+            }
+        }
+    }
+
+    // You can use this in unit tests to stop the SparkContext between tests.
+    static void stopSpark() {
+        if (sparkContext != null) {
+            sparkContext.stop();
+            sparkContext = null;
+        }
+    }
+
+
+    @Override
+    public void explain(PhysicalPlan pp, PigContext pc, PrintStream ps,
+                        String format, boolean verbose) throws IOException {
+        SparkOperPlan sparkPlan = compile(pp, pc);
+        explain(sparkPlan, ps, format, verbose);
+    }
+
+    private void explain(SparkOperPlan sparkPlan, PrintStream ps,
+                         String format, boolean verbose)
+            throws IOException {
+        Map<OperatorKey, SparkOperator> allOperKeys = sparkPlan.getKeys();
+        List<OperatorKey> operKeyList = new ArrayList<>(allOperKeys.keySet());
+        Collections.sort(operKeyList);
+
+        if (format.equals("text")) {
+            for (OperatorKey operatorKey : operKeyList) {
+                SparkOperator op = sparkPlan.getOperator(operatorKey);
+                ps.print(op.getOperatorKey());
+                List<SparkOperator> successors = sparkPlan.getSuccessors(op);
+                if (successors != null) {
+                    ps.print("->");
+                    for (SparkOperator suc : successors) {
+                        ps.print(suc.getOperatorKey() + " ");
+                    }
+                }
+                ps.println();
+            }
+            SparkPrinter printer = new SparkPrinter(ps, sparkPlan);
+            printer.setVerbose(verbose);
+            printer.visit();
+        } else if (format.equals("dot")) {
+            ps.println("#--------------------------------------------------");
+            ps.println("# Spark Plan");
+            ps.println("#--------------------------------------------------");
+
+            DotSparkPrinter printer = new DotSparkPrinter(sparkPlan, ps);
+            printer.setVerbose(verbose);
+            printer.dump();
+            ps.println("");
+        } else if (format.equals("xml")) {
+            try {
+                XMLSparkPrinter printer = new XMLSparkPrinter(ps, sparkPlan);
+                printer.visit();
+                printer.closePlan();
+            } catch (ParserConfigurationException e) {
+                e.printStackTrace();
+            } catch (TransformerException e) {
+                e.printStackTrace();
+            }
+        }
+        else {
+            throw new IOException(
+                    "Unsupported explain format. Supported formats are: text, dot, xml");
+        }
+    }
+
+    @Override
+    public void kill() throws BackendException {
+        if (sparkContext != null) {
+            sparkContext.stop();
+            sparkContext = null;
+        }
+    }
+
+    @Override
+    public void killJob(String jobID, Configuration conf)
+            throws BackendException {
+        if (sparkContext != null) {
+            sparkContext.stop();
+            sparkContext = null;
+        }
+    }
+
+    /**
+     * We store the value of udf.import.list in SparkEngineConf#properties
+     * Later we will serialize it in SparkEngineConf#writeObject and deserialize in SparkEngineConf#readObject. More
+     * detail see PIG-4920
+     */
+    private void saveUdfImportList() {
+        String udfImportList = Joiner.on(",").join(PigContext.getPackageImportList());
+        sparkEngineConf.setSparkUdfImportListStr(udfImportList);
+    }
+
+    private void initialize(PhysicalPlan physicalPlan) throws IOException {
+        saveUdfImportList();
+        jobConf = SparkUtil.newJobConf(pigContext, physicalPlan, sparkEngineConf);
+        SchemaTupleBackend.initialize(jobConf, pigContext);
+        Utils.setDefaultTimeZone(jobConf);
+        PigMapReduce.sJobConfInternal.set(jobConf);
+        String parallelism = pigContext.getProperties().getProperty("spark.default.parallelism");
+        if (parallelism != null) {
+            SparkPigContext.get().setDefaultParallelism(Integer.parseInt(parallelism));
+        }
+    }
+
+    /**
+     * Creates new SparkCounters instance for the job, initializes aggregate warning counters if required
+     * @param jobConf
+     * @throws IOException
+     */
+    private static void prepareSparkCounters(JobConf jobConf) throws IOException {
+        SparkPigStatusReporter statusReporter = SparkPigStatusReporter.getInstance();
+        SparkCounters counters = new SparkCounters(sparkContext);
+
+        if ("true".equalsIgnoreCase(jobConf.get("aggregate.warning"))) {
+            SparkCounterGroup pigWarningGroup = new SparkCounterGroup.MapSparkCounterGroup(
+                    PIG_WARNING_FQCN, PIG_WARNING_FQCN,sparkContext
+            );
+            pigWarningGroup.createCounter(PigWarning.SPARK_WARN.name(), new HashMap<String,Long>());
+            pigWarningGroup.createCounter(PigWarning.SPARK_CUSTOM_WARN.name(), new HashMap<String,Long>());
+            counters.getSparkCounterGroups().put(PIG_WARNING_FQCN, pigWarningGroup);
+        }
+        statusReporter.setCounters(counters);
+        jobConf.set("pig.spark.counters", ObjectSerializer.serialize(counters));
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLocalExecType.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLocalExecType.java
new file mode 100644
index 0000000..b330505
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkLocalExecType.java
@@ -0,0 +1,49 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.util.Properties;
+
+/**
+ * SparkLocalExecType is the ExecType for local mode in Spark.
+ */
+public class SparkLocalExecType extends SparkExecType {
+
+    private static final long serialVersionUID = 1L;
+    private static final String mode = "SPARK_LOCAL";
+
+    @Override
+    public boolean accepts(Properties properties) {
+        String execTypeSpecified = properties.getProperty("exectype", "")
+                .toUpperCase();
+        if (execTypeSpecified.equals(mode))
+            return true;
+        return false;
+    }
+
+    @Override
+    public boolean isLocal() {
+        return true;
+    }
+
+    @Override
+    public String name() {
+        return "SPARK_LOCAL";
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPOUserFuncVisitor.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPOUserFuncVisitor.java
new file mode 100644
index 0000000..f7d6ea4
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPOUserFuncVisitor.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.UdfCacheShipFilesVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+import java.util.HashSet;
+import java.util.Set;
+
+public class SparkPOUserFuncVisitor extends SparkOpPlanVisitor {
+    private Set<String> cacheFiles = new HashSet<>();
+    private Set<String> shipFiles = new HashSet<>();
+
+    public SparkPOUserFuncVisitor(SparkOperPlan plan) {
+        super(plan, new DepthFirstWalker<>(plan));
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOperator) throws VisitorException {
+        if(!sparkOperator.physicalPlan.isEmpty()) {
+            UdfCacheShipFilesVisitor udfCacheFileVisitor = new UdfCacheShipFilesVisitor(sparkOperator.physicalPlan);
+            udfCacheFileVisitor.visit();
+            cacheFiles.addAll(udfCacheFileVisitor.getCacheFiles());
+            shipFiles.addAll(udfCacheFileVisitor.getShipFiles());
+        }
+    }
+
+    public Set<String> getCacheFiles() {
+        return cacheFiles;
+    }
+
+    public Set<String> getShipFiles() {
+        return shipFiles;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigContext.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigContext.java
new file mode 100644
index 0000000..8e4bc43
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigContext.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.rdd.RDD;
+
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * singleton class like PigContext
+ */
+public class SparkPigContext {
+
+    private static SparkPigContext context =  null;
+    private static ThreadLocal<Integer> defaultParallelism = null;
+    private static ConcurrentHashMap<String, Broadcast<List<Tuple>>> broadcastedVars = new ConcurrentHashMap() ;
+
+    public static SparkPigContext get(){
+        if( context == null){
+            context = new SparkPigContext();
+        }
+        return context;
+    }
+    public static int getDefaultParallelism() {
+        return defaultParallelism.get();
+    }
+
+
+    public static int getParallelism(List<RDD<Tuple>> predecessors,
+                                     PhysicalOperator physicalOperator) {
+        if (defaultParallelism != null) {
+           return getDefaultParallelism();
+        }
+
+        int parallelism = physicalOperator.getRequestedParallelism();
+        if (parallelism <= 0) {
+            //Spark automatically sets the number of "map" tasks to run on each file according to its size (though
+            // you can control it through optional parameters to SparkContext.textFile, etc), and for distributed
+            //"reduce" operations, such as groupByKey and reduceByKey, it uses the largest parent RDD's number of
+            // partitions.
+            int maxParallism = 0;
+            for (int i = 0; i < predecessors.size(); i++) {
+                int tmpParallelism = predecessors.get(i).getNumPartitions();
+                if (tmpParallelism > maxParallism) {
+                    maxParallism = tmpParallelism;
+                }
+            }
+            parallelism = maxParallism;
+        }
+        return parallelism;
+    }
+
+    public static void setDefaultParallelism(int defaultParallelism) {
+        SparkPigContext.defaultParallelism.set(defaultParallelism);
+    }
+
+     public static ConcurrentHashMap<String, Broadcast<List<Tuple>>> getBroadcastedVars() {
+        return broadcastedVars;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigRecordReader.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigRecordReader.java
new file mode 100644
index 0000000..375f7f7
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigRecordReader.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.IOException;
+
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigRecordReader;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
+
+/**
+ * Record reader for Spark mode - handles SparkPigSplit
+ */
+public class SparkPigRecordReader extends PigRecordReader {
+
+
+    /**
+     * @param inputformat
+     * @param pigSplit
+     * @param loadFunc
+     * @param context
+     * @param limit
+     */
+    public SparkPigRecordReader(InputFormat<?, ?> inputformat, PigSplit pigSplit, LoadFunc loadFunc, TaskAttemptContext context, long limit) throws IOException, InterruptedException {
+        super(inputformat, pigSplit, loadFunc, context, limit);
+    }
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context) throws IOException, InterruptedException {
+        SparkPigSplit sparkPigSplit = (SparkPigSplit)split;
+        super.initialize(sparkPigSplit.getWrappedPigSplit(), context);
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigSplit.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigSplit.java
new file mode 100644
index 0000000..c746d76
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkPigSplit.java
@@ -0,0 +1,304 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.Serializable;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapred.SplitLocationInfo;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
+
+/**
+ * Wrapper class for PigSplits in Spark mode
+ *
+ * Spark only counts HDFS bytes read if we provide a FileSplit, so we have to wrap PigSplits and have the wrapper
+ * class extend FileSplit
+ */
+public interface SparkPigSplit extends Writable, Configurable, Serializable {
+
+   InputSplit getWrappedSplit();
+
+   InputSplit getWrappedSplit(int idx);
+
+   SplitLocationInfo[] getLocationInfo() throws IOException;
+
+   long getLength(int idx) throws IOException, InterruptedException;
+
+   int getSplitIndex();
+
+   void setMultiInputs(boolean b);
+
+   boolean isMultiInputs();
+
+   int getNumPaths();
+
+   void setDisableCounter(boolean disableCounter);
+
+   boolean disableCounter();
+
+   void setCurrentIdx(int idx);
+
+   PigSplit getWrappedPigSplit();
+
+   public static class FileSparkPigSplit extends FileSplit implements SparkPigSplit {
+
+       private PigSplit pigSplit;
+
+       /**
+        * Spark executor's deserializer calls this, and we have to instantiate a default wrapped object
+        */
+       public FileSparkPigSplit () {
+           pigSplit = new PigSplit();
+       }
+
+       public FileSparkPigSplit(PigSplit pigSplit) {
+           this.pigSplit = pigSplit;
+       }
+
+       @Override
+       public SplitLocationInfo[] getLocationInfo() throws IOException {
+           return pigSplit.getLocationInfo();
+       }
+
+       @Override
+       public String toString() {
+           return pigSplit.toString();
+       }
+
+       @Override
+       public long getLength() {
+           try {
+               return pigSplit.getLength();
+           } catch (IOException | InterruptedException e) {
+               throw new RuntimeException(e);
+           }
+       }
+
+       @Override
+       public String[] getLocations() throws IOException {
+           try {
+               return pigSplit.getLocations();
+           } catch (InterruptedException e) {
+               throw new RuntimeException(e);
+           }
+       }
+
+       @Override
+       public InputSplit getWrappedSplit() {
+           return pigSplit.getWrappedSplit();
+       }
+
+       @Override
+       public InputSplit getWrappedSplit(int idx) {
+           return pigSplit.getWrappedSplit(idx);
+       }
+
+       @Override
+       public long getLength(int idx) throws IOException, InterruptedException {
+           return pigSplit.getLength(idx);
+       }
+
+       @Override
+       public void readFields(DataInput is) throws IOException {
+           pigSplit.readFields(is);
+       }
+
+       @Override
+       public void write(DataOutput os) throws IOException {
+           pigSplit.write(os);
+       }
+
+       @Override
+       public int getSplitIndex() {
+           return pigSplit.getSplitIndex();
+       }
+
+       @Override
+       public void setMultiInputs(boolean b) {
+           pigSplit.setMultiInputs(b);
+       }
+
+       @Override
+       public boolean isMultiInputs() {
+           return pigSplit.isMultiInputs();
+       }
+
+       @Override
+       public Configuration getConf() {
+           return pigSplit.getConf();
+       }
+
+       @Override
+       public void setConf(Configuration conf) {
+           pigSplit.setConf(conf);
+       }
+
+       @Override
+       public int getNumPaths() {
+           return pigSplit.getNumPaths();
+       }
+
+       @Override
+       public void setDisableCounter(boolean disableCounter) {
+           pigSplit.setDisableCounter(disableCounter);
+       }
+
+       @Override
+       public boolean disableCounter() {
+           return pigSplit.disableCounter();
+       }
+
+       @Override
+       public void setCurrentIdx(int idx) {
+           pigSplit.setCurrentIdx(idx);
+       }
+
+       @Override
+       public PigSplit getWrappedPigSplit() {
+           return this.pigSplit;
+       }
+
+       @Override
+       public Path getPath() {
+           return ((FileSplit)getWrappedPigSplit().getWrappedSplit()).getPath();
+       }
+   }
+
+    public static class GenericSparkPigSplit extends InputSplit implements SparkPigSplit {
+
+        private static final long serialVersionUID = 1L;
+
+        private PigSplit pigSplit;
+
+        /**
+         * Spark executor's deserializer calls this, and we have to instantiate a default wrapped object
+         */
+        public GenericSparkPigSplit() {
+            pigSplit = new PigSplit();
+        }
+
+        public GenericSparkPigSplit(PigSplit pigSplit) {
+            this.pigSplit = pigSplit;
+        }
+
+        @Override
+        public SplitLocationInfo[] getLocationInfo() throws IOException {
+            return pigSplit.getLocationInfo();
+        }
+
+        @Override
+        public String toString() {
+            return pigSplit.toString();
+        }
+
+        @Override
+        public long getLength() throws IOException, InterruptedException {
+            return pigSplit.getLength();
+        }
+
+        @Override
+        public String[] getLocations() throws IOException, InterruptedException {
+            return pigSplit.getLocations();
+        }
+
+        @Override
+        public InputSplit getWrappedSplit() {
+            return pigSplit.getWrappedSplit();
+        }
+
+        @Override
+        public InputSplit getWrappedSplit(int idx) {
+            return pigSplit.getWrappedSplit(idx);
+        }
+
+        @Override
+        public long getLength(int idx) throws IOException, InterruptedException {
+            return pigSplit.getLength(idx);
+        }
+
+        @Override
+        public void readFields(DataInput is) throws IOException {
+            pigSplit.readFields(is);
+        }
+
+        @Override
+        public void write(DataOutput os) throws IOException {
+            pigSplit.write(os);
+        }
+
+        @Override
+        public int getSplitIndex() {
+            return pigSplit.getSplitIndex();
+        }
+
+        @Override
+        public void setMultiInputs(boolean b) {
+            pigSplit.setMultiInputs(b);
+        }
+
+        @Override
+        public boolean isMultiInputs() {
+            return pigSplit.isMultiInputs();
+        }
+
+        @Override
+        public Configuration getConf() {
+            return pigSplit.getConf();
+        }
+
+        @Override
+        public void setConf(Configuration conf) {
+            pigSplit.setConf(conf);
+        }
+
+        @Override
+        public int getNumPaths() {
+            return pigSplit.getNumPaths();
+        }
+
+        @Override
+        public void setDisableCounter(boolean disableCounter) {
+            pigSplit.setDisableCounter(disableCounter);
+        }
+
+        @Override
+        public boolean disableCounter() {
+            return pigSplit.disableCounter();
+        }
+
+        @Override
+        public void setCurrentIdx(int idx) {
+            pigSplit.setCurrentIdx(idx);
+        }
+
+        @Override
+        public PigSplit getWrappedPigSplit() {
+            return this.pigSplit;
+        }
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java
new file mode 100644
index 0000000..8195e5b
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/SparkUtil.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Random;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
+
+import scala.Product2;
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.Seq;
+import scala.reflect.ClassTag;
+import scala.reflect.ClassTag$;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.pig.PigConstants;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POProject;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.NodeIdGenerator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.Partitioner;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.rdd.RDD;
+
+public class SparkUtil {
+    public static <T> ClassTag<T> getManifest(Class<T> clazz) {
+        return ClassTag$.MODULE$.apply(clazz);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <K, V> ClassTag<Tuple2<K, V>> getTuple2Manifest() {
+        return (ClassTag<Tuple2<K, V>>) (Object) getManifest(Tuple2.class);
+    }
+
+    @SuppressWarnings("unchecked")
+    public static <K, V> ClassTag<Product2<K, V>> getProduct2Manifest() {
+        return (ClassTag<Product2<K, V>>) (Object) getManifest(Product2.class);
+    }
+
+    public static JobConf newJobConf(PigContext pigContext, PhysicalPlan physicalPlan, SparkEngineConf sparkEngineConf) throws
+            IOException {
+        JobConf jobConf = new JobConf(
+                ConfigurationUtil.toConfiguration(pigContext.getProperties()));
+        //Serialize the thread local variable UDFContext#udfConfs, UDFContext#clientSysProps and PigContext#packageImportList
+        //inside SparkEngineConf separately
+        jobConf.set("spark.engine.conf",ObjectSerializer.serialize(sparkEngineConf));
+        //Serialize the PigContext so it's available in Executor thread.
+        jobConf.set("pig.pigContext", ObjectSerializer.serialize(pigContext));
+        // Serialize the thread local variable inside PigContext separately
+        // Although after PIG-4920, we store udf.import.list in SparkEngineConf
+        // but we still need store it in jobConf because it will be used in PigOutputFormat#setupUdfEnvAndStores
+        jobConf.set("udf.import.list",
+                ObjectSerializer.serialize(PigContext.getPackageImportList()));
+
+        Random rand = new Random();
+        jobConf.set(MRConfiguration.JOB_APPLICATION_ATTEMPT_ID, Integer.toString(rand.nextInt()));
+        jobConf.set(PigConstants.LOCAL_CODE_DIR,
+                System.getProperty("java.io.tmpdir"));
+        jobConf.set(MRConfiguration.JOB_ID, UUID.randomUUID().toString());
+
+        LinkedList<POStore> stores = PlanHelper.getPhysicalOperators(
+                physicalPlan, POStore.class);
+        POStore firstStore = stores.getFirst();
+        if (firstStore != null) {
+            MapRedUtil.setupStreamingDirsConfSingle(firstStore, pigContext,
+                    jobConf);
+        }
+        return jobConf;
+    }
+
+    public static <T> Seq<T> toScalaSeq(List<T> list) {
+        return JavaConversions.asScalaBuffer(list);
+    }
+
+    public static void assertPredecessorSize(List<RDD<Tuple>> predecessors,
+            PhysicalOperator physicalOperator, int size) {
+        if (predecessors.size() != size) {
+            throw new RuntimeException("Should have " + size
+                    + " predecessors for " + physicalOperator.getClass()
+                    + ". Got : " + predecessors.size());
+        }
+    }
+
+    public static void assertPredecessorSizeGreaterThan(
+            List<RDD<Tuple>> predecessors, PhysicalOperator physicalOperator,
+            int size) {
+        if (predecessors.size() <= size) {
+            throw new RuntimeException("Should have greater than" + size
+                    + " predecessors for " + physicalOperator.getClass()
+                    + ". Got : " + predecessors.size());
+        }
+    }
+
+    public static Partitioner getPartitioner(String customPartitioner, int parallelism) {
+        if (customPartitioner == null) {
+            return new HashPartitioner(parallelism);
+        } else {
+            return new MapReducePartitionerWrapper(customPartitioner, parallelism);
+        }
+    }
+
+    // createIndexerSparkNode is a utility to create an indexer spark node with baseSparkOp
+    static public void createIndexerSparkNode(SparkOperator baseSparkOp, String scope, NodeIdGenerator nig) throws PlanException, ExecException {
+        List<PhysicalPlan> eps = new ArrayList<PhysicalPlan>();
+        PhysicalPlan ep = new PhysicalPlan();
+        POProject prj = new POProject(new OperatorKey(scope,
+                nig.getNextNodeId(scope)));
+        prj.setStar(true);
+        prj.setOverloaded(false);
+        prj.setResultType(DataType.TUPLE);
+        ep.add(prj);
+        eps.add(ep);
+
+        List<Boolean> ascCol = new ArrayList<Boolean>();
+        ascCol.add(true);
+
+        int requestedParallelism = baseSparkOp.requestedParallelism;
+        POSort sort = new POSort(new OperatorKey(scope, nig.getNextNodeId(scope)), requestedParallelism, null, eps, ascCol, null);
+        //POSort is added to sort the index tuples genereated by MergeJoinIndexer.More detail, see PIG-4601
+        baseSparkOp.physicalPlan.addAsLeaf(sort);
+    }
+
+
+
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/UDFJarsFinder.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/UDFJarsFinder.java
new file mode 100644
index 0000000..81f1db2
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/UDFJarsFinder.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.util.JarManager;
+
+//find udf jars which will be downloaded with spark job on every nodes
+public class UDFJarsFinder extends SparkOpPlanVisitor {
+    private PigContext pigContext = null;
+    private Set<String> udfJars = new HashSet();
+
+    public UDFJarsFinder(SparkOperPlan plan, PigContext pigContext) {
+        super(plan, new DependencyOrderWalker(plan));
+        this.pigContext = pigContext;
+    }
+
+    public void visitSparkOp(SparkOperator sparkOp)
+            throws VisitorException {
+        for (String udf : sparkOp.UDFs) {
+            try {
+                Class clazz = this.pigContext.getClassForAlias(udf);
+                if (clazz != null) {
+                    String jar = JarManager.findContainingJar(clazz);
+                    if (jar != null) {
+                        this.udfJars.add(jar);
+                    }
+                }
+            } catch (IOException e) {
+                throw new VisitorException(e);
+            }
+        }
+    }
+
+    public Set<String> getUdfJars() {
+        return this.udfJars;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java
new file mode 100644
index 0000000..eb45324
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/BroadcastConverter.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.rdd.RDD;
+
+import java.util.List;
+
+public class BroadcastConverter implements RDDConverter<Tuple, Tuple, POBroadcastSpark> {
+
+    private final JavaSparkContext sc;
+
+    public BroadcastConverter(JavaSparkContext sc) {
+        this.sc = sc;
+    }
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POBroadcastSpark po) {
+        SparkUtil.assertPredecessorSize(predecessors, po, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+
+        // Just collect the predecessor RDD, and broadcast it
+        JavaRDD<Tuple> javaRDD = new JavaRDD<>(rdd, SparkUtil.getManifest(Tuple.class));
+        Broadcast<List<Tuple>> broadcastedRDD = sc.broadcast(javaRDD.collect());
+
+        // Save the broadcast variable to broadcastedVars map, so that this
+        // broadcasted variable can be referenced by the driver client.
+        SparkPigContext.get().getBroadcastedVars().put(po.getBroadcastedVariableName(), broadcastedRDD);
+
+        return rdd;
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CollectedGroupConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CollectedGroupConverter.java
new file mode 100644
index 0000000..83311df
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CollectedGroupConverter.java
@@ -0,0 +1,88 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCollectedGroup;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings({"serial"})
+public class CollectedGroupConverter implements RDDConverter<Tuple, Tuple, POCollectedGroup> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POCollectedGroup physicalOperator) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, physicalOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        CollectedGroupFunction collectedGroupFunction
+                = new CollectedGroupFunction(physicalOperator);
+        return rdd.toJavaRDD().mapPartitions(collectedGroupFunction, true)
+                .rdd();
+    }
+
+    private static class CollectedGroupFunction
+            implements FlatMapFunction<Iterator<Tuple>, Tuple> {
+
+        private POCollectedGroup poCollectedGroup;
+
+        public long current_val;
+        public boolean proceed;
+
+        private CollectedGroupFunction(POCollectedGroup poCollectedGroup) {
+            this.poCollectedGroup = poCollectedGroup;
+            this.current_val = 0;
+        }
+
+        public Iterable<Tuple> call(final Iterator<Tuple> input) {
+
+            return new Iterable<Tuple>() {
+
+                @Override
+                public Iterator<Tuple> iterator() {
+
+                    return new OutputConsumerIterator(input) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poCollectedGroup.setInputs(null);
+                            poCollectedGroup.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return poCollectedGroup.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                            poCollectedGroup.setEndOfInput(true);
+                        }
+                    };
+                }
+            };
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CounterConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CounterConverter.java
new file mode 100644
index 0000000..324e08d
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/CounterConverter.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCounter;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function2;
+import org.apache.spark.rdd.RDD;
+
+public class CounterConverter implements RDDConverter<Tuple, Tuple, POCounter> {
+
+    private static final Log LOG = LogFactory.getLog(CounterConverter.class);
+    
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, 
+            POCounter poCounter) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, poCounter, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        CounterConverterFunction f = new CounterConverterFunction(poCounter);
+        JavaRDD<Tuple> jRdd = rdd.toJavaRDD().mapPartitionsWithIndex(f, true);
+//        jRdd = jRdd.cache();
+        return jRdd.rdd();
+    }
+    
+    @SuppressWarnings("serial")
+    private static class CounterConverterFunction implements 
+        Function2<Integer, Iterator<Tuple>, Iterator<Tuple>>, Serializable {
+
+        private final POCounter poCounter;
+        private long localCount = 1L;
+        private long sparkCount = 0L;
+        
+        private CounterConverterFunction(POCounter poCounter) {
+            this.poCounter = poCounter;
+        }
+        
+        @Override
+        public Iterator<Tuple> call(Integer index, final 
+                Iterator<Tuple> input) {
+            Tuple inp = null;
+            Tuple output = null;
+            long sizeBag = 0L;
+
+            List<Tuple> listOutput = new ArrayList<Tuple>();
+            
+            try {
+                while (input.hasNext()) {
+                    inp = input.next();
+                    output = TupleFactory.getInstance()
+                            .newTuple(inp.getAll().size() + 3);
+                    
+                    for (int i = 0; i < inp.getAll().size(); i++) {
+                        output.set(i + 3, inp.get(i));
+                    }
+                    
+                    if (poCounter.isRowNumber() || poCounter.isDenseRank()) {
+                        output.set(2, getLocalCounter());
+                        incrementSparkCounter();
+                        incrementLocalCounter();
+                    } else if (!poCounter.isDenseRank()) {
+                        int positionBag = inp.getAll().size()-1;
+                        if (inp.getType(positionBag) == DataType.BAG) {
+                            sizeBag = ((org.apache.pig.data.DefaultAbstractBag)
+                                    inp.get(positionBag)).size();
+                        }
+                        
+                        output.set(2, getLocalCounter());
+                        
+                        addToSparkCounter(sizeBag);
+                        addToLocalCounter(sizeBag);
+                    }
+                    
+                    output.set(0, index);
+                    output.set(1, getSparkCounter());
+                    listOutput.add(output);
+                }
+            } catch(ExecException e) {
+                throw new RuntimeException(e);
+            }
+            
+                    
+            return listOutput.iterator();
+        }
+        
+        private long getLocalCounter() {
+            return localCount;
+        }
+        
+        private long incrementLocalCounter() {
+            return localCount++;
+        }
+        
+        private long addToLocalCounter(long amount) {
+            return localCount += amount;
+        }
+        
+        private long getSparkCounter() {
+            return sparkCount;
+        }
+        
+        private long incrementSparkCounter() {
+            return sparkCount++;
+        }
+        
+        private long addToSparkCounter(long amount) {
+            return sparkCount += amount;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/DistinctConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/DistinctConverter.java
new file mode 100644
index 0000000..878fd25
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/DistinctConverter.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODistinct;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.rdd.PairRDDFunctions;
+import org.apache.spark.rdd.RDD;
+
+import scala.Tuple2;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.AbstractFunction2;
+
+@SuppressWarnings({ "serial" })
+public class DistinctConverter implements RDDConverter<Tuple, Tuple, PODistinct> {
+    private static final Log LOG = LogFactory.getLog(DistinctConverter.class);
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            PODistinct op) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, op, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+
+        // In DISTINCT operation, the key is the entire tuple.
+        // RDD<Tuple> -> RDD<Tuple2<Tuple, null>>
+        RDD<Tuple2<Tuple, Object>> keyValRDD = rdd.map(new ToKeyValueFunction(),
+                SparkUtil.<Tuple, Object> getTuple2Manifest());
+        PairRDDFunctions<Tuple, Object> pairRDDFunctions
+          = new PairRDDFunctions<Tuple, Object>(keyValRDD,
+                SparkUtil.getManifest(Tuple.class),
+                SparkUtil.getManifest(Object.class), null);
+        int parallelism = SparkPigContext.get().getParallelism(predecessors, op);
+        return pairRDDFunctions.reduceByKey(
+                SparkUtil.getPartitioner(op.getCustomPartitioner(), parallelism),
+                new MergeValuesFunction())
+                .map(new ToValueFunction(), SparkUtil.getManifest(Tuple.class));
+    }
+
+    /**
+     * Tuple -> Tuple2<Tuple, null>
+     */
+    private static final class ToKeyValueFunction extends
+            AbstractFunction1<Tuple, Tuple2<Tuple, Object>> implements
+            Serializable {
+        @Override
+        public Tuple2<Tuple, Object> apply(Tuple t) {
+            Tuple key = t;
+            Object value = null;
+            Tuple2<Tuple, Object> out = new Tuple2<Tuple, Object>(key, value);
+            return out;
+        }
+    }
+
+    /**
+     * No-op
+     */
+    private static final class MergeValuesFunction extends
+            AbstractFunction2<Object, Object, Object> implements Serializable {
+        @Override
+        public Object apply(Object arg0, Object arg1) {
+            return null;
+        }
+    }
+
+    /**
+     * Tuple2<Tuple, null> -> Tuple
+     */
+    private static final class ToValueFunction extends
+            AbstractFunction1<Tuple2<Tuple, Object>, Tuple> implements
+            Serializable {
+        @Override
+        public Tuple apply(Tuple2<Tuple, Object> input) {
+            return input._1;
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FRJoinConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FRJoinConverter.java
new file mode 100644
index 0000000..382258e
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FRJoinConverter.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoinSpark;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoin;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings("serial")
+public class FRJoinConverter implements
+        RDDConverter<Tuple, Tuple, POFRJoin> {
+    private static final Log LOG = LogFactory.getLog(FRJoinConverter.class);
+
+    private Set<String> replicatedInputs;
+
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POFRJoin poFRJoin) throws IOException {
+        SparkUtil.assertPredecessorSizeGreaterThan(predecessors, poFRJoin, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+
+        attachReplicatedInputs((POFRJoinSpark) poFRJoin);
+
+        FRJoinFunction frJoinFunction = new FRJoinFunction(poFRJoin);
+        return rdd.toJavaRDD().mapPartitions(frJoinFunction, true).rdd();
+    }
+
+    private void attachReplicatedInputs(POFRJoinSpark poFRJoin) {
+        Map<String, List<Tuple>> replicatedInputMap = new HashMap<>();
+
+        for (String replicatedInput : replicatedInputs) {
+            replicatedInputMap.put(replicatedInput, SparkPigContext.get().getBroadcastedVars().get(replicatedInput).value());
+        }
+
+        poFRJoin.attachInputs(replicatedInputMap);
+    }
+
+    private static class FRJoinFunction implements
+            FlatMapFunction<Iterator<Tuple>, Tuple>, Serializable {
+
+        private POFRJoin poFRJoin;
+        private FRJoinFunction(POFRJoin poFRJoin) {
+            this.poFRJoin = poFRJoin;
+        }
+
+        @Override
+        public Iterable<Tuple> call(final Iterator<Tuple> input) throws Exception {
+
+            return new Iterable<Tuple>() {
+
+                @Override
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(input) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poFRJoin.setInputs(null);
+                            poFRJoin.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return poFRJoin.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                        }
+                    };
+                }
+            };
+        }
+
+    }
+
+    public void setReplicatedInputs(Set<String> replicatedInputs) {
+        this.replicatedInputs = replicatedInputs;
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FilterConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FilterConverter.java
new file mode 100644
index 0000000..51d7730
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/FilterConverter.java
@@ -0,0 +1,84 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.Serializable;
+import java.util.List;
+
+import scala.runtime.AbstractFunction1;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.rdd.RDD;
+
+/**
+ * Converter that converts an RDD to a filtered RRD using POFilter
+ */
+@SuppressWarnings({ "serial" })
+public class FilterConverter implements RDDConverter<Tuple, Tuple, POFilter> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            POFilter physicalOperator) {
+        SparkUtil.assertPredecessorSize(predecessors, physicalOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        FilterFunction filterFunction = new FilterFunction(physicalOperator);
+        return rdd.filter(filterFunction);
+    }
+
+    private static class FilterFunction extends
+            AbstractFunction1<Tuple, Object> implements Serializable {
+
+        private POFilter poFilter;
+
+        private FilterFunction(POFilter poFilter) {
+            this.poFilter = poFilter;
+        }
+
+        @Override
+        public Boolean apply(Tuple v1) {
+            Result result;
+            try {
+                poFilter.setInputs(null);
+                poFilter.attachInput(v1);
+                result = poFilter.getNextTuple();
+            } catch (ExecException e) {
+                throw new RuntimeException("Couldn't filter tuple", e);
+            }
+
+            if (result == null) {
+                return false;
+            }
+
+            switch (result.returnStatus) {
+            case POStatus.STATUS_OK:
+                return true;
+            case POStatus.STATUS_EOP: // TODO: probably also ok for EOS,
+                                      // END_OF_BATCH
+                return false;
+            default:
+                throw new RuntimeException(
+                        "Unexpected response code from filter: " + result);
+            }
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ForEachConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ForEachConverter.java
new file mode 100644
index 0000000..b58415e
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ForEachConverter.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.ProgressableReporter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.spark.KryoSerializer;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.SchemaTupleBackend;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+/**
+ * Convert that is able to convert an RRD to another RRD using a POForEach
+ */
+@SuppressWarnings({"serial" })
+public class ForEachConverter implements RDDConverter<Tuple, Tuple, POForEach> {
+
+    private JobConf jobConf;
+
+    public ForEachConverter(JobConf jobConf) {
+        this.jobConf = jobConf;
+    }
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            POForEach physicalOperator) {
+
+        byte[] confBytes = KryoSerializer.serializeJobConf(jobConf);
+
+        SparkUtil.assertPredecessorSize(predecessors, physicalOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        ForEachFunction forEachFunction = new ForEachFunction(physicalOperator, confBytes);
+
+        return rdd.toJavaRDD().mapPartitions(forEachFunction, true).rdd();
+    }
+
+    private static class ForEachFunction implements
+            FlatMapFunction<Iterator<Tuple>, Tuple>, Serializable {
+
+        private POForEach poForEach;
+        private byte[] confBytes;
+        private transient JobConf jobConf;
+
+        private ForEachFunction(POForEach poForEach, byte[] confBytes) {
+            this.poForEach = poForEach;
+            this.confBytes = confBytes;
+        }
+
+        public Iterable<Tuple> call(final Iterator<Tuple> input) {
+
+            initialize();
+
+            // Initialize a reporter as the UDF might want to report progress.
+            PhysicalOperator.setReporter(new ProgressableReporter());
+            PhysicalOperator[] planLeafOps= poForEach.getPlanLeafOps();
+            if (planLeafOps != null) {
+                for (PhysicalOperator op : planLeafOps) {
+                    if (op.getClass() == POUserFunc.class) {
+                        POUserFunc udf = (POUserFunc) op;
+                          udf.setFuncInputSchema();
+                    }
+                }
+            }
+
+
+            return new Iterable<Tuple>() {
+
+                @Override
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(input) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poForEach.setInputs(null);
+                            poForEach.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return poForEach.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                        }
+                    };
+                }
+            };
+        }
+
+        private void initialize() {
+            if (this.jobConf == null) {
+                try {
+                    this.jobConf = KryoSerializer.deserializeJobConf(this.confBytes);
+                    PigContext pc = (PigContext) ObjectSerializer.deserialize(jobConf.get("pig.pigContext"));
+                    SchemaTupleBackend.initialize(jobConf, pc);
+                } catch (IOException e) {
+                    throw new RuntimeException("Couldn't initialize ForEachConverter");
+                }
+            }
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/GlobalRearrangeConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/GlobalRearrangeConverter.java
new file mode 100644
index 0000000..130c8b9
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/GlobalRearrangeConverter.java
@@ -0,0 +1,384 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.rdd.CoGroupedRDD;
+import org.apache.spark.rdd.RDD;
+
+import scala.Product2;
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.Seq;
+import scala.runtime.AbstractFunction1;
+
+@SuppressWarnings({ "serial" })
+public class GlobalRearrangeConverter implements
+        RDDConverter<Tuple, Tuple, POGlobalRearrangeSpark> {
+    private static final Log LOG = LogFactory
+            .getLog(GlobalRearrangeConverter.class);
+
+    private static final TupleFactory tf = TupleFactory.getInstance();
+ 
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POGlobalRearrangeSpark op) throws IOException {
+        SparkUtil.assertPredecessorSizeGreaterThan(predecessors,
+                op, 0);
+        int parallelism = SparkPigContext.get().getParallelism(predecessors,
+                op);
+
+//         TODO: Figure out the tradeoff of using CoGroupRDD (even for 1 input),
+//         vs using groupBy (like we do in this commented code), vs using
+//         reduceByKey(). This is a pending task in Pig on Spark Milestone 1
+//         Once we figure that out, we can allow custom partitioning for
+//         secondary sort case as well.
+//        if (predecessors.size() == 1) {
+//            // GROUP BY
+//            JavaPairRDD<Object, Iterable<Tuple>> prdd;
+//            if (op.isUseSecondaryKey()) {
+//                prdd = handleSecondarySort(predecessors.get(0), op, parallelism);
+//            } else {
+//                JavaRDD<Tuple> jrdd = predecessors.get(0).toJavaRDD();
+//                prdd = jrdd.groupBy(new GetKeyFunction(op), parallelism);
+//                prdd.groupByKey(new CustomPartitioner(op.getCustomPartitioner(),
+//                        parallelism));
+//            }
+//            JavaRDD<Tuple> jrdd2 = prdd.map(new GroupTupleFunction(op));
+//            return jrdd2.rdd();
+//
+//        if (predecessors.size() == 1 && op.isUseSecondaryKey()) {
+//            return handleSecondarySort(predecessors.get(0), op, parallelism);
+//        }
+        List<RDD<Tuple2<IndexedKey, Tuple>>> rddPairs = new ArrayList<RDD<Tuple2<IndexedKey, Tuple>>>();
+
+        if (predecessors.size() == 1 && op.isUseSecondaryKey()) {
+            rddPairs.add(handleSecondarySort(predecessors.get(0), op, parallelism).rdd());
+        } else {
+            for (RDD<Tuple> rdd : predecessors) {
+                JavaRDD<Tuple> jrdd = JavaRDD.fromRDD(rdd, SparkUtil.getManifest(Tuple.class));
+                JavaRDD<Tuple2<IndexedKey, Tuple>> rddPair = jrdd.map(new ToKeyValueFunction());
+                rddPairs.add(rddPair.rdd());
+            }
+        }
+
+        // Something's wrong with the type parameters of CoGroupedRDD
+        // key and value are the same type ???
+        CoGroupedRDD<Object> coGroupedRDD = new CoGroupedRDD<Object>(
+                (Seq<RDD<? extends Product2<Object, ?>>>) (Object) (JavaConversions
+                        .asScalaBuffer(rddPairs).toSeq()),
+                SparkUtil.getPartitioner(op.getCustomPartitioner(), parallelism), SparkUtil.getManifest(Object.class));
+
+        RDD<Tuple2<IndexedKey, Seq<Seq<Tuple>>>> rdd =
+            (RDD<Tuple2<IndexedKey, Seq<Seq<Tuple>>>>) (Object) coGroupedRDD;
+        return rdd.toJavaRDD().map(new ToGroupKeyValueFunction()).rdd();
+    }
+
+    private JavaRDD<Tuple2<IndexedKey,Tuple>> handleSecondarySort(
+            RDD<Tuple> rdd, POGlobalRearrangeSpark op, int parallelism) {
+
+        RDD<Tuple2<Tuple, Object>> rddPair = rdd.map(new ToKeyNullValueFunction(),
+                SparkUtil.<Tuple, Object>getTuple2Manifest());
+
+        JavaPairRDD<Tuple, Object> pairRDD = new JavaPairRDD<Tuple, Object>(rddPair,
+                SparkUtil.getManifest(Tuple.class),
+                SparkUtil.getManifest(Object.class));
+
+        //first sort the tuple by secondary key if enable useSecondaryKey sort
+        JavaPairRDD<Tuple, Object> sorted = pairRDD.repartitionAndSortWithinPartitions(
+                new HashPartitioner(parallelism),
+                new PigSecondaryKeyComparatorSpark(op.getSecondarySortOrder()));
+        JavaRDD<Tuple> jrdd = sorted.keys();
+        JavaRDD<Tuple2<IndexedKey,Tuple>> jrddPair = jrdd.map(new ToKeyValueFunction(op));
+        return jrddPair;
+    }
+
+    private static class RemoveValueFunction implements
+            FlatMapFunction<Iterator<Tuple2<Tuple, Object>>, Tuple>, Serializable {
+
+        private class Tuple2TransformIterable implements Iterable<Tuple> {
+
+            Iterator<Tuple2<Tuple, Object>> in;
+
+            Tuple2TransformIterable(Iterator<Tuple2<Tuple, Object>> input) {
+                in = input;
+            }
+
+            public Iterator<Tuple> iterator() {
+                return new IteratorTransform<Tuple2<Tuple, Object>, Tuple>(in) {
+                    @Override
+                    protected Tuple transform(Tuple2<Tuple, Object> next) {
+                        return next._1();
+                    }
+                };
+            }
+        }
+
+        @Override
+        public Iterable<Tuple> call(Iterator<Tuple2<Tuple, Object>> input) {
+            return new Tuple2TransformIterable(input);
+        }
+    }
+
+    private static class ToKeyNullValueFunction extends
+            AbstractFunction1<Tuple, Tuple2<Tuple, Object>> implements
+            Serializable {
+
+        @Override
+        public Tuple2<Tuple, Object> apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("ToKeyNullValueFunction in " + t);
+            }
+
+            Tuple2<Tuple, Object> out = new Tuple2<Tuple, Object>(t, null);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("ToKeyNullValueFunction out " + out);
+            }
+
+            return out;
+        }
+    }
+
+    /**
+     * Function that extract keys from locally rearranged tuples.
+     */
+    private static class GetKeyFunction implements Function<Tuple, Object>, Serializable {
+        public final POGlobalRearrangeSpark glrSpark;
+
+        public GetKeyFunction(POGlobalRearrangeSpark globalRearrangeSpark) {
+            this.glrSpark = globalRearrangeSpark;
+        }
+
+        public Object call(Tuple t) {
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("GetKeyFunction in " + t);
+                }
+
+                Object key;
+                if ((glrSpark != null) && (glrSpark.isUseSecondaryKey())) {
+                    key = ((Tuple) t.get(1)).get(0);
+                } else {
+                    key = t.get(1);
+                }
+
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("GetKeyFunction out " + key);
+                }
+
+                return key;
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    /**
+     * Function that converts elements of PairRDD to regular RDD.
+     * - Input (PairRDD) contains elements of the form
+     * Tuple2<key, Iterable<(index, key, value)>>.
+     * - Output (regular RDD) contains elements of the form
+     * Tuple<(key, iterator to (index, key, value))>
+     */
+    private static class GroupTupleFunction
+            implements Function<Tuple2<Object, Iterable<Tuple>>, Tuple>,
+            Serializable {
+        public final POGlobalRearrangeSpark glrSpark;
+
+        public GroupTupleFunction(POGlobalRearrangeSpark globalRearrangeSpark) {
+            this.glrSpark = globalRearrangeSpark;
+        }
+
+        public Tuple call(Tuple2<Object, Iterable<Tuple>> v1) {
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("GroupTupleFunction in " + v1);
+                }
+
+                Tuple tuple = tf.newTuple(2);
+                tuple.set(0, v1._1()); // key
+                // Note that v1._2() is (index, key, value) tuple, and
+                // v1._2().iterator() is iterator to Seq<Tuple> (aka bag of values)
+                tuple.set(1, v1._2().iterator());
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("GroupTupleFunction out " + tuple);
+                }
+
+                return tuple;
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+
+    /**
+     * Converts incoming locally rearranged tuple, which is of the form
+     * (index, key, value) into Tuple2<key, value>
+     */
+    private static class ToKeyValueFunction implements
+            Function<Tuple, Tuple2<IndexedKey, Tuple>>, Serializable {
+
+        private POGlobalRearrangeSpark glrSpark = null;
+
+        public ToKeyValueFunction(POGlobalRearrangeSpark glrSpark) {
+            this.glrSpark = glrSpark;
+        }
+
+        public ToKeyValueFunction() {
+
+        }
+
+        @Override
+        public Tuple2<IndexedKey, Tuple> call(Tuple t) {
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("ToKeyValueFunction in " + t);
+                }
+
+                Object key = null;
+                if ((glrSpark != null) && (glrSpark.isUseSecondaryKey())) {
+                    key = ((Tuple) t.get(1)).get(0);
+                } else {
+                    key = t.get(1);
+                }
+
+                Tuple2<IndexedKey, Tuple> out = new Tuple2<IndexedKey, Tuple>(
+                        new IndexedKey((Byte) t.get(0), key),
+                        (Tuple) t.get(2));
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("ToKeyValueFunction out " + out);
+                }
+
+                return out;
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    /**
+     * Converts cogroup output where each element is {key, bag[]} represented
+     * as Tuple2<Object, Seq<Seq<Tuple>>> into tuple of form
+     * (key, Iterator<(index, key, value)>)
+     */
+    private static class ToGroupKeyValueFunction implements
+            Function<Tuple2<IndexedKey, Seq<Seq<Tuple>>>, Tuple>, Serializable {
+
+        @Override
+        public Tuple call(Tuple2<IndexedKey, Seq<Seq<Tuple>>> input) {
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("ToGroupKeyValueFunction in " + input);
+                }
+
+                final Object key = input._1().getKey();
+                Object obj = input._2();
+                // XXX this is a hack for Spark 1.1.0: the type is an Array, not Seq
+                Seq<Tuple>[] bags = (Seq<Tuple>[])obj;
+                int i = 0;
+                List<Iterator<Tuple>> tupleIterators = new ArrayList<Iterator<Tuple>>();
+                for (int j = 0; j < bags.length; j ++) {
+                    Seq<Tuple> bag = bags[j];
+                    Iterator<Tuple> iterator = JavaConversions
+                            .asJavaCollection(bag).iterator();
+                    final int index = i;
+                    tupleIterators.add(new IteratorTransform<Tuple, Tuple>(
+                            iterator) {
+                        @Override
+                        protected Tuple transform(Tuple next) {
+                            try {
+                                Tuple tuple = tf.newTuple(3);
+                                tuple.set(0, index);
+                                tuple.set(1, next);
+                                return tuple;
+                            } catch (ExecException e) {
+                                throw new RuntimeException(e);
+                            }
+                        }
+                    });
+                    ++ i;
+                }
+
+                Tuple out = tf.newTuple(2);
+                out.set(0, key);
+                out.set(1, new IteratorUnion<Tuple>(tupleIterators.iterator()));
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("ToGroupKeyValueFunction out " + out);
+                }
+
+                return out;
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    private static class IteratorUnion<T> implements Iterator<T> {
+
+        private final Iterator<Iterator<T>> iterators;
+
+        private Iterator<T> current;
+
+        public IteratorUnion(Iterator<Iterator<T>> iterators) {
+            super();
+            this.iterators = iterators;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (current != null && current.hasNext()) {
+                return true;
+            } else if (iterators.hasNext()) {
+                current = iterators.next();
+                return hasNext();
+            } else {
+                return false;
+            }
+        }
+
+        @Override
+        public T next() {
+            return current.next();
+        }
+
+        @Override
+        public void remove() {
+            current.remove();
+        }
+
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/IndexedKey.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/IndexedKey.java
new file mode 100644
index 0000000..70049a8
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/IndexedKey.java
@@ -0,0 +1,193 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.Serializable;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+
+import org.joda.time.DateTime;
+
+/**
+ * IndexedKey records the index and key info.
+ * This is used as key for JOINs. It addresses the case where key is
+ * either empty (or is a tuple with one or more empty fields). In this case,
+ * we must respect the SQL standard as documented in the equals() method.
+ */
+public class IndexedKey implements Serializable, Comparable {
+    private static final Log LOG = LogFactory.getLog(IndexedKey.class);
+    private byte index;
+    private Object key;
+    private boolean useSecondaryKey;
+    private boolean[] secondarySortOrder;
+
+    public IndexedKey(byte index, Object key) {
+        this.index = index;
+        this.key = key;
+    }
+
+    public byte getIndex() {
+        return index;
+    }
+
+    public Object getKey() {
+        return key;
+    }
+
+    @Override
+    public String toString() {
+        return "IndexedKey{" +
+                "index=" + index +
+                ", key=" + key +
+                '}';
+    }
+
+    /**
+     * If key is empty, we'd like compute equality based on key and index.
+     * If key is not empty, we'd like to compute equality based on just the key (like we normally do).
+     * There are two possible cases when two tuples are compared:
+     * 1) Compare tuples of same table (same index)
+     * 2) Compare tuples of different tables (different index values)
+     * In 1)
+     * key1    key2    equal?
+     * null    null      Y
+     * foo     null      N
+     * null    foo       N
+     * foo     foo       Y
+     * (1,1)   (1,1)     Y
+     * (1,)    (1,)      Y
+     * (1,2)   (1,2)     Y
+     * <p/>
+     * <p/>
+     * In 2)
+     * key1    key2    equal?
+     * null    null     N
+     * foo     null     N
+     * null    foo      N
+     * foo     foo      Y
+     * (1,1)   (1,1)    Y
+     * (1,)    (1,)     N
+     * (1,2)   (1,2)    Y
+     *
+     * @param o
+     * @return
+     */
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        IndexedKey that = (IndexedKey) o;
+        if (index == that.index) {
+            if (key == null && that.key == null) {
+                return true;
+            } else if (key == null || that.key == null) {
+                return false;
+            } else {
+                if (key instanceof DateTime) {
+                    //In case of DateTime we use the less strict isEqual method so that
+                    //e.g. 2017.01.01T10:00:00.000Z and 2017.01.01T11:00:00.000+01:00 are equal
+                    return ((DateTime) key).isEqual((DateTime)that.key);
+                } else {
+                    return key.equals(that.key);
+                }
+            }
+        } else {
+            if (key == null || that.key == null) {
+                return false;
+            } else if (key.equals(that.key) && !containNullfields(key)) {
+                return true;
+            } else {
+                return false;
+            }
+        }
+    }
+
+    private boolean containNullfields(Object key) {
+        if (key instanceof Tuple) {
+            for (int i = 0; i < ((Tuple) key).size(); i++) {
+                try {
+                    if (((Tuple) key).get(i) == null) {
+                        return true;
+                    }
+                } catch (ExecException e) {
+                    throw new RuntimeException("exception found in " +
+                            "containNullfields", e);
+
+                }
+            }
+        }
+        return false;
+
+    }
+
+    /**
+     * Calculate hashCode by index and key
+     * if key is empty, return index value
+     * if key is not empty, return the key.hashCode()
+     */
+    @Override
+    public int hashCode() {
+        int result = 0;
+        if (key == null) {
+            result = (int) index;
+        } else {
+            if (key instanceof DateTime) {
+                //In case of DateTime we use a custom hashCode() to avoid chronology taking part in the hash value
+                DateTime dt = (DateTime)key;
+                result = (int) (dt.getMillis() ^ dt.getMillis() >>> 32);
+            } else {
+                result = key.hashCode();
+            }
+        }
+        return result;
+    }
+
+    //firstly compare the index
+    //secondly compare the key (both first and secondary key)
+    @Override
+    public int compareTo(Object o) {
+        IndexedKey that = (IndexedKey) o;
+        int res = index - that.getIndex();
+        if (res > 0) {
+            return 1;
+        } else if (res < 0) {
+            return -1;
+        } else {
+            if (useSecondaryKey) {
+                Tuple thisCompoundKey = (Tuple) key;
+                Tuple thatCompoundKey = (Tuple)that.getKey();
+                PigSecondaryKeyComparatorSpark comparator = new PigSecondaryKeyComparatorSpark(secondarySortOrder);
+                return comparator.compareCompoundKey(thisCompoundKey, thatCompoundKey);
+            } else {
+                return DataType.compare(key, that.getKey());
+            }
+        }
+    }
+
+    public void setUseSecondaryKey(boolean useSecondaryKey) {
+        this.useSecondaryKey = useSecondaryKey;
+    }
+
+    public void setSecondarySortOrder(boolean[] secondarySortOrder) {
+        this.secondarySortOrder = secondarySortOrder;
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/IteratorTransform.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/IteratorTransform.java
new file mode 100644
index 0000000..40a166d
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/IteratorTransform.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.util.Iterator;
+
+abstract class IteratorTransform<IN, OUT> implements Iterator<OUT> {
+    private Iterator<IN> delegate;
+
+    public IteratorTransform(Iterator<IN> delegate) {
+        super();
+        this.delegate = delegate;
+    }
+
+    @Override
+    public boolean hasNext() {
+        return delegate.hasNext();
+    }
+
+    @Override
+    public OUT next() {
+        return transform(delegate.next());
+    }
+
+    abstract protected OUT transform(IN next);
+
+    @Override
+    public void remove() {
+        delegate.remove();
+    }
+
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/JoinGroupSparkConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/JoinGroupSparkConverter.java
new file mode 100644
index 0000000..3c037b4
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/JoinGroupSparkConverter.java
@@ -0,0 +1,265 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POJoinGroupSpark;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.io.NullableTuple;
+import org.apache.pig.impl.io.PigNullableWritable;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.rdd.CoGroupedRDD;
+import org.apache.spark.rdd.RDD;
+
+import scala.Product2;
+import scala.Tuple2;
+import scala.collection.JavaConversions;
+import scala.collection.Seq;
+import scala.runtime.AbstractFunction1;
+
+public class JoinGroupSparkConverter implements RDDConverter<Tuple, Tuple, POJoinGroupSpark> {
+    private static final Log LOG = LogFactory
+            .getLog(JoinGroupSparkConverter.class);
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POJoinGroupSpark op) throws IOException {
+        SparkUtil.assertPredecessorSizeGreaterThan(predecessors,
+                op, 0);
+        List<POLocalRearrange> lraOps = op.getLROps();
+        POGlobalRearrangeSpark glaOp = op.getGROp();
+        POPackage pkgOp = op.getPkgOp();
+        int parallelism = SparkPigContext.get().getParallelism(predecessors, glaOp);
+        List<RDD<Tuple2<IndexedKey, Tuple>>> rddAfterLRA = new ArrayList<RDD<Tuple2<IndexedKey, Tuple>>>();
+        boolean useSecondaryKey = glaOp.isUseSecondaryKey();
+
+        for (int i = 0; i < predecessors.size(); i++) {
+            RDD<Tuple> rdd = predecessors.get(i);
+            rddAfterLRA.add(rdd.map(new LocalRearrangeFunction(lraOps.get(i), glaOp),
+                    SparkUtil.<IndexedKey, Tuple>getTuple2Manifest()));
+        }
+        if (rddAfterLRA.size() == 1 && useSecondaryKey) {
+            return SecondaryKeySortUtil.handleSecondarySort(rddAfterLRA.get(0), pkgOp);
+        } else {
+
+            CoGroupedRDD<Object> coGroupedRDD = new CoGroupedRDD<Object>(
+                    (Seq<RDD<? extends Product2<Object, ?>>>) (Object) (JavaConversions
+                            .asScalaBuffer(rddAfterLRA).toSeq()),
+                    SparkUtil.getPartitioner(glaOp.getCustomPartitioner(), parallelism),
+                    SparkUtil.getManifest(Object.class));
+
+            RDD<Tuple2<IndexedKey, Seq<Seq<Tuple>>>> rdd =
+                    (RDD<Tuple2<IndexedKey, Seq<Seq<Tuple>>>>) (Object) coGroupedRDD;
+            return rdd.toJavaRDD().map(new GroupPkgFunction(pkgOp)).rdd();
+        }
+    }
+
+    private static class LocalRearrangeFunction extends
+            AbstractFunction1<Tuple, Tuple2<IndexedKey, Tuple>> implements Serializable {
+
+        private final POLocalRearrange lra;
+
+        private boolean useSecondaryKey;
+        private boolean[] secondarySortOrder;
+
+        public LocalRearrangeFunction(POLocalRearrange lra, POGlobalRearrangeSpark glaOp) {
+            if( glaOp.isUseSecondaryKey()) {
+                this.useSecondaryKey = glaOp.isUseSecondaryKey();
+                this.secondarySortOrder = glaOp.getSecondarySortOrder();
+            }
+            this.lra = lra;
+        }
+
+        @Override
+        public Tuple2<IndexedKey, Tuple> apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("LocalRearrangeFunction in " + t);
+            }
+            Result result;
+            try {
+                lra.setInputs(null);
+                lra.attachInput(t);
+                result = lra.getNextTuple();
+
+                if (result == null) {
+                    throw new RuntimeException(
+                            "Null response found for LocalRearange on tuple: "
+                                    + t);
+                }
+
+                switch (result.returnStatus) {
+                    case POStatus.STATUS_OK:
+                        // (index, key, value without keys)
+                        Tuple resultTuple = (Tuple) result.result;
+                        Object key = resultTuple.get(1);
+                        IndexedKey indexedKey = new IndexedKey((Byte) resultTuple.get(0), key);
+                        if( useSecondaryKey) {
+                            indexedKey.setUseSecondaryKey(useSecondaryKey);
+                            indexedKey.setSecondarySortOrder(secondarySortOrder);
+                        }
+                        Tuple2<IndexedKey, Tuple> out = new Tuple2<IndexedKey, Tuple>(indexedKey,
+                                (Tuple) resultTuple.get(2));
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("LocalRearrangeFunction out " + out);
+                        }
+                        return out;
+                    default:
+                        throw new RuntimeException(
+                                "Unexpected response code from operator "
+                                        + lra + " : " + result);
+                }
+            } catch (ExecException e) {
+                throw new RuntimeException(
+                        "Couldn't do LocalRearange on tuple: " + t, e);
+            }
+        }
+
+    }
+
+    /**
+     * Send cogroup output where each element is {key, bag[]} to PoPackage
+     * then call PoPackage#getNextTuple to get the result
+     */
+    private static class GroupPkgFunction implements
+            Function<Tuple2<IndexedKey, Seq<Seq<Tuple>>>, Tuple>, Serializable {
+
+        private final POPackage pkgOp;
+
+        public GroupPkgFunction(POPackage pkgOp) {
+            this.pkgOp = pkgOp;
+        }
+
+        @Override
+        public Tuple call(final Tuple2<IndexedKey, Seq<Seq<Tuple>>> input) {
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("GroupPkgFunction in " + input);
+                }
+
+                final PigNullableWritable key = new PigNullableWritable() {
+
+                    public Object getValueAsPigType() {
+                        IndexedKey keyTuple = input._1();
+                        return keyTuple.getKey();
+                    }
+                };
+                Object obj = input._2();
+                // XXX this is a hack for Spark 1.1.0: the type is an Array, not Seq
+                Seq<Tuple>[] bags = (Seq<Tuple>[]) obj;
+                int i = 0;
+                List<Iterator<NullableTuple>> tupleIterators = new ArrayList<Iterator<NullableTuple>>();
+                for (int j = 0; j < bags.length; j++) {
+                    Seq<Tuple> bag = bags[j];
+                    Iterator<Tuple> iterator = JavaConversions
+                            .asJavaCollection(bag).iterator();
+                    final int index = i;
+                    tupleIterators.add(new IteratorTransform<Tuple, NullableTuple>(
+                            iterator) {
+                        @Override
+                        protected NullableTuple transform(Tuple next) {
+                            NullableTuple nullableTuple = new NullableTuple(next);
+                            nullableTuple.setIndex((byte) index);
+                            return nullableTuple;
+                        }
+                    });
+                    ++i;
+                }
+
+
+                pkgOp.setInputs(null);
+                pkgOp.attachInput(key, new IteratorUnion<NullableTuple>(tupleIterators.iterator()));
+                Result result = pkgOp.getNextTuple();
+                if (result == null) {
+                    throw new RuntimeException(
+                            "Null response found for Package on key: " + key);
+                }
+                Tuple out;
+                switch (result.returnStatus) {
+                    case POStatus.STATUS_OK:
+                        // (key, {(value)...})
+                        out = (Tuple) result.result;
+                        break;
+                    case POStatus.STATUS_NULL:
+                        out = null;
+                        break;
+                    default:
+                        throw new RuntimeException(
+                                "Unexpected response code from operator "
+                                        + pkgOp + " : " + result + " "
+                                        + result.returnStatus);
+                }
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("GroupPkgFunction out " + out);
+                }
+                return out;
+            } catch (Exception e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+
+    private static class IteratorUnion<T> implements Iterator<T> {
+
+        private final Iterator<Iterator<T>> iterators;
+
+        private Iterator<T> current;
+
+        public IteratorUnion(Iterator<Iterator<T>> iterators) {
+            super();
+            this.iterators = iterators;
+        }
+
+        @Override
+        public boolean hasNext() {
+            if (current != null && current.hasNext()) {
+                return true;
+            } else if (iterators.hasNext()) {
+                current = iterators.next();
+                return hasNext();
+            } else {
+                return false;
+            }
+        }
+
+        @Override
+        public T next() {
+            return current.next();
+        }
+
+        @Override
+        public void remove() {
+            current.remove();
+        }
+
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LimitConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LimitConverter.java
new file mode 100644
index 0000000..fe1b54c
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LimitConverter.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings({ "serial" })
+public class LimitConverter implements RDDConverter<Tuple, Tuple, POLimit> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POLimit poLimit)
+            throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, poLimit, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        LimitFunction limitFunction = new LimitFunction(poLimit);
+        RDD<Tuple> rdd2 = rdd.coalesce(1, false, null);
+        return rdd2.toJavaRDD().mapPartitions(limitFunction, false).rdd();
+    }
+
+    private static class LimitFunction implements FlatMapFunction<Iterator<Tuple>, Tuple> {
+
+        private final POLimit poLimit;
+
+        public LimitFunction(POLimit poLimit) {
+            this.poLimit = poLimit;
+        }
+
+        @Override
+        public Iterable<Tuple> call(final Iterator<Tuple> tuples) {
+
+            return new Iterable<Tuple>() {
+
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(tuples) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poLimit.setInputs(null);
+                            poLimit.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return poLimit.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                        }
+                    };
+                }
+            };
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LoadConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LoadConverter.java
new file mode 100644
index 0000000..67fe7a6
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LoadConverter.java
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.Serializable;
+import java.net.MalformedURLException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import scala.Function1;
+import scala.Tuple2;
+import scala.runtime.AbstractFunction1;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.PigConstants;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkEngineConf;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.backend.hadoop.executionengine.spark.running.PigInputFormatSpark;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.UDFContext;
+import org.apache.pig.tools.pigstats.spark.SparkCounters;
+import org.apache.pig.tools.pigstats.spark.SparkPigStatusReporter;
+import org.apache.pig.tools.pigstats.spark.SparkStatsUtil;
+import org.apache.spark.SparkContext;
+import org.apache.spark.TaskContext;
+import org.apache.spark.rdd.RDD;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Converter that loads data via POLoad and converts it to RRD&lt;Tuple>. Abuses
+ * the interface a bit in that there is no input RRD to convert in this case.
+ * Instead input is the source path of the POLoad.
+ */
+@SuppressWarnings({ "serial" })
+public class LoadConverter implements RDDConverter<Tuple, Tuple, POLoad> {
+    private static Log LOG = LogFactory.getLog(LoadConverter.class);
+
+    private PigContext pigContext;
+    private PhysicalPlan physicalPlan;
+    private SparkContext sparkContext;
+    private JobConf jobConf;
+    private SparkEngineConf sparkEngineConf;
+
+    public LoadConverter(PigContext pigContext, PhysicalPlan physicalPlan,
+            SparkContext sparkContext, JobConf jobConf, SparkEngineConf sparkEngineConf) {
+        this.pigContext = pigContext;
+        this.physicalPlan = physicalPlan;
+        this.sparkContext = sparkContext;
+        this.jobConf = jobConf;
+        this.sparkEngineConf = sparkEngineConf;
+    }
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessorRdds, POLoad op)
+            throws IOException {
+        configureLoader(physicalPlan, op, jobConf);
+
+        // Set the input directory for input formats that are backed by a
+        // filesystem. (Does not apply to HBase, for example).
+        jobConf.set("mapreduce.input.fileinputformat.inputdir",
+                op.getLFile().getFileName());
+
+        // internally set pig.noSplitCombination as true ONLY for
+        // the POLoad operator which has POMergeJoin successor.
+        if (hasMergeJoinSuccessor(op)) {
+            jobConf.set("pig.noSplitCombination", "true");
+        }
+
+
+        //serialize the UDFContext#udfConfs in jobConf
+        UDFContext.getUDFContext().serialize(jobConf);
+
+        //SparkContext.newAPIHadoop will broadcast the jobConf to other worker nodes.
+        //Later in PigInputFormatSpark#createRecordReader, jobConf will be used to
+        //initialize PigContext,UDFContext and SchemaTupleBackend.
+        RDD<Tuple2<Text, Tuple>> hadoopRDD = sparkContext.newAPIHadoopRDD(
+                jobConf, PigInputFormatSpark.class, Text.class, Tuple.class);
+
+        registerUdfFiles();
+
+        ToTupleFunction ttf = new ToTupleFunction(sparkEngineConf);
+
+        //create SparkCounter and set it for ToTupleFunction
+        boolean disableCounter = jobConf.getBoolean("pig.disable.counter", false);
+        if (!op.isTmpLoad() && !disableCounter) {
+            String counterName = SparkStatsUtil.getCounterName(op);
+            SparkPigStatusReporter counterReporter = SparkPigStatusReporter.getInstance();
+            if (counterReporter.getCounters() != null) {
+                counterReporter.getCounters().createCounter(
+                        SparkStatsUtil.SPARK_INPUT_COUNTER_GROUP,
+                        counterName);
+            }
+
+            ttf.setDisableCounter(disableCounter);
+            ttf.setCounterGroupName(SparkStatsUtil.SPARK_INPUT_COUNTER_GROUP);
+            ttf.setCounterName(counterName);
+            ttf.setSparkCounters(SparkPigStatusReporter.getInstance().getCounters());
+        }
+
+        // map to get just RDD<Tuple>
+        return hadoopRDD.map(ttf,
+                SparkUtil.getManifest(Tuple.class));
+    }
+
+    private void registerUdfFiles() throws MalformedURLException{
+        Map<String, File> scriptFiles = pigContext.getScriptFiles();
+        for (Map.Entry<String, File> scriptFile : scriptFiles.entrySet()) {
+            File script = scriptFile.getValue();
+            if (script.exists()) {
+                sparkContext.addFile(script.toURI().toURL().toExternalForm());
+            }
+        }
+    }
+
+    private static class ToTupleFunction extends
+            AbstractFunction1<Tuple2<Text, Tuple>, Tuple> implements
+            Function1<Tuple2<Text, Tuple>, Tuple>, Serializable {
+
+        private String counterGroupName;
+        private String counterName;
+        private SparkCounters sparkCounters;
+        private boolean disableCounter;
+        private SparkEngineConf sparkEngineConf;
+        private boolean initialized;
+
+        public ToTupleFunction(SparkEngineConf sparkEngineConf){
+               this.sparkEngineConf = sparkEngineConf;
+        }
+
+        @Override
+        public Tuple apply(Tuple2<Text, Tuple> v1) {
+            if (!initialized) {
+                long partitionId = TaskContext.get().partitionId();
+                Configuration jobConf = PigMapReduce.sJobConfInternal.get();
+                jobConf.set(PigConstants.TASK_INDEX, Long.toString(partitionId));
+                jobConf.set(MRConfiguration.TASK_ID, Long.toString(partitionId));
+                initialized = true;
+            }
+            if (sparkCounters != null && disableCounter == false) {
+                sparkCounters.increment(counterGroupName, counterName, 1L);
+            }
+            return v1._2();
+        }
+
+        public void setCounterGroupName(String counterGroupName) {
+            this.counterGroupName = counterGroupName;
+        }
+
+        public void setCounterName(String counterName) {
+            this.counterName = counterName;
+        }
+
+        public void setSparkCounters(SparkCounters sparkCounters) {
+            this.sparkCounters = sparkCounters;
+        }
+
+        public void setDisableCounter(boolean disableCounter) {
+            this.disableCounter = disableCounter;
+        }
+    }
+
+    /**
+     * stolen from JobControlCompiler TODO: refactor it to share this
+     *
+     * @param physicalPlan
+     * @param poLoad
+     * @param jobConf
+     * @return
+     * @throws java.io.IOException
+     */
+    private static JobConf configureLoader(PhysicalPlan physicalPlan,
+            POLoad poLoad, JobConf jobConf) throws IOException {
+
+        Job job = new Job(jobConf);
+        LoadFunc loadFunc = poLoad.getLoadFunc();
+
+        loadFunc.setLocation(poLoad.getLFile().getFileName(), job);
+
+        // stolen from JobControlCompiler
+        ArrayList<FileSpec> pigInputs = new ArrayList<FileSpec>();
+        // Store the inp filespecs
+        pigInputs.add(poLoad.getLFile());
+
+        ArrayList<List<OperatorKey>> inpTargets = Lists.newArrayList();
+        ArrayList<String> inpSignatures = Lists.newArrayList();
+        ArrayList<Long> inpLimits = Lists.newArrayList();
+        // Store the target operators for tuples read
+        // from this input
+        List<PhysicalOperator> loadSuccessors = physicalPlan
+                .getSuccessors(poLoad);
+        List<OperatorKey> loadSuccessorsKeys = Lists.newArrayList();
+        if (loadSuccessors != null) {
+            for (PhysicalOperator loadSuccessor : loadSuccessors) {
+                loadSuccessorsKeys.add(loadSuccessor.getOperatorKey());
+            }
+        }
+        inpTargets.add(loadSuccessorsKeys);
+        inpSignatures.add(poLoad.getSignature());
+        inpLimits.add(poLoad.getLimit());
+
+        jobConf.set(PigInputFormat.PIG_INPUTS, ObjectSerializer.serialize(pigInputs));
+        jobConf.set(PigInputFormat.PIG_INPUT_TARGETS, ObjectSerializer.serialize(inpTargets));
+        jobConf.set(PigInputFormat.PIG_INPUT_SIGNATURES,
+                ObjectSerializer.serialize(inpSignatures));
+        jobConf.set(PigInputFormat.PIG_INPUT_LIMITS, ObjectSerializer.serialize(inpLimits));
+        return jobConf;
+    }
+
+    private static boolean hasMergeJoinSuccessor(PhysicalOperator op) {
+        if (op == null || op.getParentPlan() == null) {
+            return false;
+        }
+        List<PhysicalOperator> successors = op.getParentPlan().getSuccessors(op);
+        if (successors == null ) {
+            return false;
+        }
+        for (PhysicalOperator successor : successors){
+            if (successor instanceof POMergeJoin){
+                return true;
+            }
+            if (hasMergeJoinSuccessor(successor)){
+                return true;
+            }
+        }
+        return false;
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LocalRearrangeConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LocalRearrangeConverter.java
new file mode 100644
index 0000000..6ef1331
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/LocalRearrangeConverter.java
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+import scala.runtime.AbstractFunction1;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings({ "serial" })
+public class LocalRearrangeConverter implements
+        RDDConverter<Tuple, Tuple, PhysicalOperator> {
+    private static final Log LOG = LogFactory
+            .getLog(LocalRearrangeConverter.class);
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            PhysicalOperator physicalOperator) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, physicalOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        // call local rearrange to get key and value
+        return rdd.map(new LocalRearrangeFunction(physicalOperator),
+                SparkUtil.getManifest(Tuple.class));
+
+    }
+
+    private static class LocalRearrangeFunction extends
+            AbstractFunction1<Tuple, Tuple> implements Serializable {
+
+        private final PhysicalOperator physicalOperator;
+
+        public LocalRearrangeFunction(PhysicalOperator physicalOperator) {
+            this.physicalOperator = physicalOperator;
+        }
+
+        @Override
+        public Tuple apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("LocalRearrangeFunction in " + t);
+            }
+            Result result;
+            try {
+                physicalOperator.setInputs(null);
+                physicalOperator.attachInput(t);
+                result = physicalOperator.getNextTuple();
+
+                if (result == null) {
+                    throw new RuntimeException(
+                            "Null response found for LocalRearange on tuple: "
+                                    + t);
+                }
+
+                switch (result.returnStatus) {
+                case POStatus.STATUS_OK:
+                    // (index, key, value without keys)
+                    Tuple resultTuple = (Tuple) result.result;
+                    if (LOG.isDebugEnabled())
+                        LOG.debug("LocalRearrangeFunction out " + resultTuple);
+                    return resultTuple;
+                default:
+                    throw new RuntimeException(
+                            "Unexpected response code from operator "
+                                    + physicalOperator + " : " + result);
+                }
+            } catch (ExecException e) {
+                throw new RuntimeException(
+                        "Couldn't do LocalRearange on tuple: " + t, e);
+            }
+        }
+
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeCogroupConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeCogroupConverter.java
new file mode 100644
index 0000000..adf78ec
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeCogroupConverter.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeCogroup;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+
+public class MergeCogroupConverter implements RDDConverter<Tuple, Tuple, POMergeCogroup> {
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POMergeCogroup physicalOperator) {
+        SparkUtil.assertPredecessorSize(predecessors, physicalOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        MergeCogroupFunction mergeCogroupFunction = new MergeCogroupFunction(physicalOperator);
+        return rdd.toJavaRDD().mapPartitions(mergeCogroupFunction, true).rdd();
+    }
+
+    private static class MergeCogroupFunction implements
+            FlatMapFunction<Iterator<Tuple>, Tuple>, Serializable {
+
+        private POMergeCogroup poMergeCogroup;
+
+        @Override
+        public Iterable<Tuple> call(final Iterator<Tuple> input) throws Exception {
+            return new Iterable<Tuple>() {
+
+                @Override
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(input) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poMergeCogroup.setInputs(null);
+                            poMergeCogroup.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return poMergeCogroup.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                            poMergeCogroup.setEndOfInput(true);
+                        }
+                    };
+                }
+            };
+        }
+
+        private MergeCogroupFunction(POMergeCogroup poMergeCogroup) {
+            this.poMergeCogroup = poMergeCogroup;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeJoinConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeJoinConverter.java
new file mode 100644
index 0000000..d1c43b1
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/MergeJoinConverter.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+
+@SuppressWarnings("serial")
+public class MergeJoinConverter implements
+        RDDConverter<Tuple, Tuple, POMergeJoin> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POMergeJoin poMergeJoin) throws IOException {
+
+        SparkUtil.assertPredecessorSize(predecessors, poMergeJoin, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        MergeJoinFunction mergeJoinFunction = new MergeJoinFunction(poMergeJoin);
+
+        return rdd.toJavaRDD().mapPartitions(mergeJoinFunction, true).rdd();
+    }
+
+    private static class MergeJoinFunction implements
+            FlatMapFunction<Iterator<Tuple>, Tuple>, Serializable {
+
+        private POMergeJoin poMergeJoin;
+
+        private MergeJoinFunction(POMergeJoin poMergeJoin) {
+            this.poMergeJoin = poMergeJoin;
+        }
+
+        public Iterable<Tuple> call(final Iterator<Tuple> input) {
+
+            return new Iterable<Tuple>() {
+                @Override
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(input) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poMergeJoin.setInputs(null);
+                            poMergeJoin.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return poMergeJoin.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                            poMergeJoin.setEndOfInput(true);
+                        }
+                    };
+                }
+            };
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/OutputConsumerIterator.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/OutputConsumerIterator.java
new file mode 100644
index 0000000..85b2d1e
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/OutputConsumerIterator.java
@@ -0,0 +1,118 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.data.Tuple;
+
+abstract class OutputConsumerIterator implements java.util.Iterator<Tuple> {
+    private final java.util.Iterator<Tuple> input;
+    private Result result = null;
+    private boolean returned = true;
+    private boolean done = false;
+
+    OutputConsumerIterator(java.util.Iterator<Tuple> input) {
+        this.input = input;
+    }
+
+    abstract protected void attach(Tuple tuple);
+
+    abstract protected Result getNextResult() throws ExecException;
+
+    /**
+     * Certain operators may buffer the output.
+     * We need to flush the last set of records from such operators,
+     * when we encounter the last input record, before calling
+     * getNextTuple() for the last time.
+     */
+    abstract protected void endOfInput();
+
+    private void readNext() {
+        while (true) {
+            try {
+                // result is set in hasNext() call and returned
+                // to the user in next() call
+                if (result != null && !returned) {
+                    return;
+                }
+
+                if (result == null) {
+                    if (!input.hasNext()) {
+                        done = true;
+                        return;
+                    }
+                    Tuple v1 = input.next();
+                    attach(v1);
+                }
+
+                if (!input.hasNext()) {
+                    endOfInput();
+                }
+
+                result = getNextResult();
+                returned = false;
+                switch (result.returnStatus) {
+                    case POStatus.STATUS_OK:
+                        returned = false;
+                        break;
+                    case POStatus.STATUS_NULL:
+                        returned = true;
+                        break;
+                    case POStatus.STATUS_EOP:
+                        done = !input.hasNext();
+                        if (!done) {
+                            result = null;
+                        }
+                        break;
+                    case POStatus.STATUS_ERR:
+                        throw new RuntimeException("Error while processing " + result);
+                }
+
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    @Override
+    public boolean hasNext() {
+        readNext();
+        return !done;
+    }
+
+    @Override
+    public Tuple next() {
+        readNext();
+        if (done) {
+            throw new RuntimeException("Past the end. Call hasNext() before calling next()");
+        }
+        if (result == null || result.returnStatus != POStatus.STATUS_OK) {
+            throw new RuntimeException("Unexpected response code from operator: "
+                    + result);
+        }
+        returned = true;
+        return (Tuple) result.result;
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException();
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PackageConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PackageConverter.java
new file mode 100644
index 0000000..412f73e
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PackageConverter.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+import scala.runtime.AbstractFunction1;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.io.NullableTuple;
+import org.apache.pig.impl.io.PigNullableWritable;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings({ "serial" })
+public class PackageConverter implements RDDConverter<Tuple, Tuple, POPackage> {
+    private static final Log LOG = LogFactory.getLog(PackageConverter.class);
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            POPackage physicalOperator) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, physicalOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        // package will generate the group from the result of the local
+        // rearrange
+        return rdd.map(new PackageFunction(physicalOperator),
+                SparkUtil.getManifest(Tuple.class));
+    }
+
+    private static class PackageFunction extends
+            AbstractFunction1<Tuple, Tuple> implements Serializable {
+
+        private final POPackage physicalOperator;
+
+        public PackageFunction(POPackage physicalOperator) {
+            this.physicalOperator = physicalOperator;
+        }
+
+        @Override
+        public Tuple apply(final Tuple t) {
+            // (key, Seq<Tuple>:{(index, key, value without key)})
+            if (LOG.isDebugEnabled())
+                LOG.debug("PackageFunction in " + t);
+            Result result;
+            try {
+                PigNullableWritable key = new PigNullableWritable() {
+
+                    public Object getValueAsPigType() {
+                        try {
+                            Object keyTuple = t.get(0);
+                            return keyTuple;
+                        } catch (ExecException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+                };
+                final Iterator<Tuple> bagIterator = (Iterator<Tuple>) t.get(1);
+                Iterator<NullableTuple> iterator = new Iterator<NullableTuple>() {
+                    public boolean hasNext() {
+                        return bagIterator.hasNext();
+                    }
+
+                    public NullableTuple next() {
+                        try {
+                            // we want the value and index only
+                            Tuple next = bagIterator.next();
+                            NullableTuple nullableTuple = new NullableTuple(
+                                    (Tuple) next.get(1));
+                            nullableTuple.setIndex(((Number) next.get(0))
+                                    .byteValue());
+                            if (LOG.isDebugEnabled())
+                                LOG.debug("Setting index to " + next.get(0) +
+                                    " for tuple " + (Tuple)next.get(1));
+                            return nullableTuple;
+                        } catch (ExecException e) {
+                            throw new RuntimeException(e);
+                        }
+                    }
+
+                    public void remove() {
+                        throw new UnsupportedOperationException();
+                    }
+                };
+                physicalOperator.setInputs(null);
+                physicalOperator.attachInput(key, iterator);
+                result = physicalOperator.getNextTuple();
+            } catch (ExecException e) {
+                throw new RuntimeException(
+                        "Couldn't do Package on tuple: " + t, e);
+            }
+
+            if (result == null) {
+                throw new RuntimeException(
+                        "Null response found for Package on tuple: " + t);
+            }
+            Tuple out;
+            switch (result.returnStatus) {
+            case POStatus.STATUS_OK:
+                // (key, {(value)...})
+                if (LOG.isDebugEnabled())
+                    LOG.debug("PackageFunction out " + result.result);
+                out = (Tuple) result.result;
+                break;
+            case POStatus.STATUS_NULL:
+                out = null;
+                break;
+            default:
+                throw new RuntimeException(
+                        "Unexpected response code from operator "
+                                + physicalOperator + " : " + result + " "
+                                + result.returnStatus);
+            }
+            if (LOG.isDebugEnabled())
+                LOG.debug("PackageFunction out " + out);
+            return out;
+        }
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PigSecondaryKeyComparatorSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PigSecondaryKeyComparatorSpark.java
new file mode 100644
index 0000000..5feca5a
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PigSecondaryKeyComparatorSpark.java
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.Serializable;
+import java.util.Comparator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+
+/**
+ * Utility class that handles secondary key for sorting.
+ */
+class PigSecondaryKeyComparatorSpark implements Comparator, Serializable {
+    private static final Log LOG = LogFactory.getLog(PigSecondaryKeyComparatorSpark.class);
+    private static final long serialVersionUID = 1L;
+
+    private static boolean[] secondarySortOrder;
+
+    public PigSecondaryKeyComparatorSpark(boolean[] pSecondarySortOrder) {
+        secondarySortOrder = pSecondarySortOrder;
+    }
+
+    //IndexedKeyPartitioner will put the tuple with same mainKey together, in PigSecondaryKeyComparatorSpark#compare
+    // (Object o1, Object o2)
+    //we only compare the secondaryKey
+    @Override
+    public int compare(Object o1, Object o2) {
+        Tuple t1 = (Tuple) o1;
+        Tuple t2 = (Tuple) o2;
+        try {
+            if ((t1.size() < 3) || (t2.size() < 3)) {
+                throw new RuntimeException("tuple size must bigger than 3, tuple[0] stands for index, tuple[1]" +
+                        "stands for the compound key, tuple[3] stands for the value");
+            }
+            Tuple compoundKey1 = (Tuple) t1.get(1);
+            Tuple compoundKey2 = (Tuple) t2.get(1);
+            if ((compoundKey1.size() < 2) || (compoundKey2.size() < 2)) {
+                throw new RuntimeException("compoundKey size must bigger than, compoundKey[0] stands for firstKey," +
+                        "compoundKey[1] stands for secondaryKey");
+            }
+            Object secondaryKey1 = compoundKey1.get(1);
+            Object secondaryKey2 = compoundKey2.get(1);
+            int res = compareKeys(secondaryKey1, secondaryKey2, secondarySortOrder);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("t1:" + t1 + "t2:" + t2 + " res:" + res);
+            }
+            return res;
+        } catch (ExecException e) {
+            throw new RuntimeException("Fail to get the compoundKey", e);
+        }
+    }
+
+    //compare the mainKey and secondaryKey
+    public int compareCompoundKey(Tuple compoundKey1, Tuple compoundKey2){
+        try {
+            if ((compoundKey1.size() < 2) || (compoundKey2.size() < 2)) {
+                throw new RuntimeException("compoundKey size must bigger than, compoundKey[0] stands for firstKey," +
+                        "compoundKey[1] stands for secondaryKey");
+            }
+            Object mainKey1 = compoundKey1.get(0);
+            Object mainKey2 = compoundKey2.get(0);
+            int res = compareKeys(mainKey1,mainKey2, null);
+            if ( res !=0 ){
+                return res;
+            } else {
+                Object secondaryKey1 = compoundKey1.get(1);
+                Object secondaryKey2 = compoundKey2.get(1);
+                res = compareKeys(secondaryKey1, secondaryKey2, secondarySortOrder);
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("compoundKey1:" + compoundKey1 + "compoundKey2:" + compoundKey2 + " res:" + res);
+                }
+                return res;
+            }
+        } catch (ExecException e) {
+            throw new RuntimeException("Fail to get the compoundKey", e);
+        }
+    }
+
+    private int compareKeys(Object o1, Object o2, boolean[] asc) {
+        int rc = 0;
+        if (o1 != null && o2 != null && o1 instanceof Tuple && o2 instanceof Tuple) {
+            // objects are Tuples, we may need to apply sort order inside them
+            Tuple t1 = (Tuple) o1;
+            Tuple t2 = (Tuple) o2;
+            int sz1 = t1.size();
+            int sz2 = t2.size();
+            if (sz2 < sz1) {
+                return 1;
+            } else if (sz2 > sz1) {
+                return -1;
+            } else {
+                for (int i = 0; i < sz1; i++) {
+                    try {
+                        rc = DataType.compare(t1.get(i), t2.get(i));
+                        if (rc != 0 && asc != null && asc.length > 1 && !asc[i])
+                            rc *= -1;
+                        if ((t1.get(i) == null) || (t2.get(i) == null)) {
+                            if (LOG.isDebugEnabled()) {
+                                LOG.debug("t1.get(i) is:" + t1.get(i) + " t2.get(i) is:" + t2.get(i));
+                            }
+                        }
+                        if (rc != 0) break;
+                    } catch (ExecException e) {
+                        throw new RuntimeException("Unable to compare tuples", e);
+                    }
+                }
+            }
+        } else {
+            // objects are NOT Tuples, delegate to DataType.compare()
+            rc = DataType.compare(o1, o2);
+        }
+        // apply sort order for keys that are not tuples or for whole tuples
+        if (asc != null && asc.length == 1 && !asc[0])
+            rc *= -1;
+        return rc;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PoissonSampleConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PoissonSampleConverter.java
new file mode 100644
index 0000000..e003bbd
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/PoissonSampleConverter.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POPoissonSampleSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+
+public class PoissonSampleConverter implements RDDConverter<Tuple, Tuple, POPoissonSampleSpark> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POPoissonSampleSpark po) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, po, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        PoissionSampleFunction poissionSampleFunction = new PoissionSampleFunction(po);
+        return rdd.toJavaRDD().mapPartitions(poissionSampleFunction, false).rdd();
+    }
+
+    private static class PoissionSampleFunction implements FlatMapFunction<Iterator<Tuple>, Tuple> {
+
+        private final POPoissonSampleSpark po;
+
+        public PoissionSampleFunction(POPoissonSampleSpark po) {
+            this.po = po;
+        }
+
+        @Override
+        public Iterable<Tuple> call(final Iterator<Tuple> tuples) {
+
+            return new Iterable<Tuple>() {
+
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(tuples) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            po.setInputs(null);
+                            po.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            return po.getNextTuple();
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                            po.setEndOfInput(true);
+                        }
+                    };
+                }
+            };
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/RDDConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/RDDConverter.java
new file mode 100644
index 0000000..df73510
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/RDDConverter.java
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+
+import org.apache.spark.rdd.RDD;
+
+/**
+ * Given an RDD and a PhysicalOperater, and implementation of this class can
+ * convert the RDD to another RDD.
+ */
+public interface RDDConverter<IN, OUT, T extends PhysicalOperator> {
+    RDD<OUT> convert(List<RDD<IN>> rdd, T physicalOperator) throws IOException;
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/RankConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/RankConverter.java
new file mode 100644
index 0000000..6d263e6
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/RankConverter.java
@@ -0,0 +1,135 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import scala.Tuple2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PORank;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFunction;
+import org.apache.spark.rdd.RDD;
+
+public class RankConverter implements RDDConverter<Tuple, Tuple, PORank> {
+
+    private static final Log LOG = LogFactory.getLog(RankConverter.class);
+    
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, PORank poRank)
+            throws IOException {
+        int parallelism = SparkPigContext.get().getParallelism(predecessors, poRank);
+        SparkUtil.assertPredecessorSize(predecessors, poRank, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        JavaPairRDD<Integer, Long> javaPairRdd = rdd.toJavaRDD()
+                .mapToPair(new ToPairRdd());
+        JavaPairRDD<Integer, Iterable<Long>> groupedByIndex = javaPairRdd
+                .groupByKey(parallelism);
+        JavaPairRDD<Integer, Long> countsByIndex = groupedByIndex
+                .mapToPair(new IndexCounters());
+        JavaPairRDD<Integer, Long> sortedCountsByIndex = countsByIndex
+                .sortByKey(true, parallelism);
+        Map<Integer, Long> counts = sortedCountsByIndex.collectAsMap();
+        JavaRDD<Tuple> finalRdd = rdd.toJavaRDD()
+                .map(new RankFunction(new HashMap<Integer, Long>(counts)));
+        return finalRdd.rdd();
+    }
+
+    @SuppressWarnings("serial")
+    private static class ToPairRdd implements 
+        PairFunction<Tuple, Integer, Long>, Serializable {
+
+        @Override
+        public Tuple2<Integer, Long> call(Tuple t) {
+            try {
+                Integer key = (Integer) t.get(0);
+                Long value = (Long) t.get(1);
+                return new Tuple2<Integer, Long>(key, value);
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+    
+    @SuppressWarnings("serial")
+    private static class IndexCounters implements 
+        PairFunction<Tuple2<Integer, Iterable<Long>>, Integer, Long>, 
+        Serializable {
+        @Override
+        public Tuple2<Integer, Long> call(Tuple2<Integer, 
+                Iterable<Long>> input) {
+            long lastVaue = 0L;
+            
+            for (Long t : input._2()) {
+                lastVaue = (t > lastVaue) ? t : lastVaue;
+            }
+
+            return new Tuple2<Integer, Long>(input._1(), lastVaue);
+        }
+    }
+    
+    @SuppressWarnings("serial")
+    private static class RankFunction implements Function<Tuple, Tuple>, 
+            Serializable {
+        private final HashMap<Integer, Long> counts;
+        
+        private RankFunction(HashMap<Integer, Long> counts) {
+            this.counts = counts;
+        }
+        
+        @Override
+        public Tuple call(Tuple input) throws Exception {
+            Tuple output = TupleFactory.getInstance()
+                    .newTuple(input.getAll().size() - 2);
+            
+            for (int i = 1; i < input.getAll().size() - 2; i ++) {
+                output.set(i, input.get(i+2));
+            }
+            
+            long offset = calculateOffset((Integer) input.get(0));
+            output.set(0, offset + (Long)input.get(2));
+            return output;
+        }
+        
+        private long calculateOffset(Integer index) {
+            long offset = 0;
+            
+            if (index > 0) {
+                for (int i = 0; i < index; i++) {
+                    if (counts.containsKey(i)) {
+                        offset += counts.get(i);
+                    }
+                }
+            }
+            return offset;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ReduceByConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ReduceByConverter.java
new file mode 100644
index 0000000..d14f59a
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/ReduceByConverter.java
@@ -0,0 +1,325 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.AbstractFunction2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POReduceBySpark;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DefaultBagFactory;
+import org.apache.pig.data.DefaultTuple;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.spark.HashPartitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.rdd.PairRDDFunctions;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings({"serial"})
+public class ReduceByConverter implements RDDConverter<Tuple, Tuple, POReduceBySpark> {
+    private static final Log LOG = LogFactory.getLog(ReduceByConverter.class);
+
+    private static final TupleFactory tf = TupleFactory.getInstance();
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POReduceBySpark op) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, op, 1);
+        int parallelism = SparkPigContext.get().getParallelism(predecessors, op);
+
+        RDD<Tuple> rdd = predecessors.get(0);
+        RDD<Tuple2<IndexedKey, Tuple>> rddPair
+                = rdd.map(new LocalRearrangeFunction(op.getLROp(), op.isUseSecondaryKey(), op.getSecondarySortOrder())
+                , SparkUtil.<IndexedKey, Tuple>getTuple2Manifest());
+        if (op.isUseSecondaryKey()) {
+            return SecondaryKeySortUtil.handleSecondarySort(rddPair, op.getPKGOp());
+        } else {
+            PairRDDFunctions<IndexedKey, Tuple> pairRDDFunctions
+                    = new PairRDDFunctions<>(rddPair,
+                    SparkUtil.getManifest(IndexedKey.class),
+                    SparkUtil.getManifest(Tuple.class), null);
+
+            RDD<Tuple2<IndexedKey, Tuple>> tupleRDD = pairRDDFunctions.reduceByKey(
+                    SparkUtil.getPartitioner(op.getCustomPartitioner(), parallelism),
+                    new MergeValuesFunction(op));
+            LOG.debug("Custom Partitioner and parallelims used : " + op.getCustomPartitioner() + ", " + parallelism);
+
+            return tupleRDD.map(new ToTupleFunction(op), SparkUtil.getManifest(Tuple.class));
+        }
+    }
+
+    private JavaRDD<Tuple2<IndexedKey, Tuple>> handleSecondarySort(
+            RDD<Tuple> rdd, POReduceBySpark op, int parallelism) {
+
+        RDD<Tuple2<Tuple, Object>> rddPair = rdd.map(new ToKeyNullValueFunction(),
+                SparkUtil.<Tuple, Object>getTuple2Manifest());
+
+        JavaPairRDD<Tuple, Object> pairRDD = new JavaPairRDD<Tuple, Object>(rddPair,
+                SparkUtil.getManifest(Tuple.class),
+                SparkUtil.getManifest(Object.class));
+
+        //first sort the tuple by secondary key if enable useSecondaryKey sort
+        JavaPairRDD<Tuple, Object> sorted = pairRDD.repartitionAndSortWithinPartitions(
+                new HashPartitioner(parallelism),
+                new PigSecondaryKeyComparatorSpark(op.getSecondarySortOrder()));
+        JavaRDD<Tuple> jrdd = sorted.keys();
+        JavaRDD<Tuple2<IndexedKey, Tuple>> jrddPair = jrdd.map(new ToKeyValueFunction(op));
+        return jrddPair;
+    }
+
+    private static class ToKeyNullValueFunction extends
+            AbstractFunction1<Tuple, Tuple2<Tuple, Object>> implements
+            Serializable {
+
+        @Override
+        public Tuple2<Tuple, Object> apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("ToKeyNullValueFunction in " + t);
+            }
+
+            Tuple2<Tuple, Object> out = new Tuple2<Tuple, Object>(t, null);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("ToKeyNullValueFunction out " + out);
+            }
+
+            return out;
+        }
+    }
+
+    /**
+     * Converts incoming locally rearranged tuple, which is of the form
+     * (index, key, value) into Tuple2<key, Tuple(key, value)>
+     */
+    private static class ToKeyValueFunction implements
+            Function<Tuple, Tuple2<IndexedKey, Tuple>>, Serializable {
+
+        private POReduceBySpark poReduce = null;
+
+        public ToKeyValueFunction(POReduceBySpark poReduce) {
+            this.poReduce = poReduce;
+        }
+
+        @Override
+        public Tuple2<IndexedKey, Tuple> call(Tuple t) {
+            try {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("ToKeyValueFunction in " + t);
+                }
+
+                Object key;
+                if ((poReduce != null) && (poReduce.isUseSecondaryKey())) {
+                    key = ((Tuple) t.get(1)).get(0);
+                } else {
+                    key = t.get(1);
+                }
+
+                Tuple tupleWithKey = tf.newTuple();
+                tupleWithKey.append(key);
+                tupleWithKey.append(t.get(2));
+
+                Tuple2<IndexedKey, Tuple> out = new Tuple2<IndexedKey, Tuple>(new IndexedKey((Byte) t.get(0), key), tupleWithKey);
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("ToKeyValueFunction out " + out);
+                }
+
+                return out;
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    /**
+     * Given two input tuples, this function outputs the resultant tuple.
+     * Additionally, it packages the input tuples to ensure the Algebraic Functions can work on them.
+     */
+    private static final class MergeValuesFunction extends AbstractFunction2<Tuple, Tuple, Tuple>
+            implements Serializable {
+        private final POReduceBySpark poReduce;
+
+        public MergeValuesFunction(POReduceBySpark poReduce) {
+            this.poReduce = poReduce;
+        }
+
+        @Override
+        public Tuple apply(Tuple v1, Tuple v2) {
+            LOG.debug("MergeValuesFunction in : " + v1 + " , " + v2);
+            Tuple result = tf.newTuple(2);
+            DataBag bag = DefaultBagFactory.getInstance().newDefaultBag();
+            Tuple t = new DefaultTuple();
+            try {
+                // Package the input tuples so they can be processed by Algebraic functions.
+                Object key = v1.get(0);
+                if (key == null) {
+                    key = "";
+                } else {
+                    result.set(0, key);
+                }
+                bag.add((Tuple) v1.get(1));
+                bag.add((Tuple) v2.get(1));
+                t.append(key);
+                t.append(bag);
+
+                poReduce.getPKGOp().getPkgr().attachInput(key, new DataBag[]{(DataBag) t.get(1)}, new boolean[]{true});
+                Tuple packagedTuple = (Tuple) poReduce.getPKGOp().getPkgr().getNext().result;
+
+                // Perform the operation
+                LOG.debug("MergeValuesFunction packagedTuple : " + t);
+                poReduce.attachInput(packagedTuple);
+                Result r = poReduce.getNext(poReduce.getResultType());
+
+                // Ensure output is consistent with the output of KeyValueFunction
+                // If we return r.result, the result will be something like this:
+                // (ABC,(2),(3)) - A tuple with key followed by values.
+                // But, we want the result to look like this:
+                // (ABC,((2),(3))) - A tuple with key and a value tuple (containing values).
+                // Hence, the construction of a new value tuple
+
+                Tuple valueTuple = tf.newTuple();
+                for (Object o : ((Tuple) r.result).getAll()) {
+                    if (!o.equals(key)) {
+                        valueTuple.append(o);
+                    }
+                }
+                result.set(1,valueTuple);
+                LOG.debug("MergeValuesFunction out : " + result);
+                return result;
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
+    /**
+     * This function transforms the Tuple to ensure it is packaged as per requirements of the Operator's packager.
+     */
+    private static final class ToTupleFunction extends AbstractFunction1<Tuple2<IndexedKey, Tuple>, Tuple>
+            implements Serializable {
+
+        private final POReduceBySpark poReduce;
+
+        public ToTupleFunction(POReduceBySpark poReduce) {
+            this.poReduce = poReduce;
+        }
+
+        @Override
+        public Tuple apply(Tuple2<IndexedKey, Tuple> v1) {
+            LOG.debug("ToTupleFunction in : " + v1);
+            DataBag bag = DefaultBagFactory.getInstance().newDefaultBag();
+            Tuple t = new DefaultTuple();
+            Tuple packagedTuple = null;
+            try {
+                Object key = v1._2().get(0);
+                bag.add((Tuple) v1._2().get(1));
+                t.append(key);
+                t.append(bag);
+                poReduce.getPKGOp().getPkgr().attachInput(key, new DataBag[]{(DataBag) t.get(1)}, new boolean[]{true});
+                packagedTuple = (Tuple) poReduce.getPKGOp().getPkgr().getNext().result;
+            } catch (ExecException e) {
+                throw new RuntimeException(e);
+            }
+            LOG.debug("ToTupleFunction out : " + packagedTuple);
+            return packagedTuple;
+        }
+    }
+
+    /**
+     * Converts incoming locally rearranged tuple, which is of the form
+     * (index, key, value) into Tuple2<key, Tuple(key, value)>
+     */
+    private static class LocalRearrangeFunction extends
+            AbstractFunction1<Tuple, Tuple2<IndexedKey, Tuple>> implements Serializable {
+
+        private final POLocalRearrange lra;
+
+        private boolean useSecondaryKey;
+        private boolean[] secondarySortOrder;
+
+        public LocalRearrangeFunction(POLocalRearrange lra, boolean useSecondaryKey, boolean[] secondarySortOrder) {
+            if( useSecondaryKey ) {
+                this.useSecondaryKey = useSecondaryKey;
+                this.secondarySortOrder = secondarySortOrder;
+            }
+            this.lra = lra;
+        }
+
+        @Override
+        public Tuple2<IndexedKey, Tuple> apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("LocalRearrangeFunction in " + t);
+            }
+            Result result;
+            try {
+                lra.setInputs(null);
+                lra.attachInput(t);
+                result = lra.getNextTuple();
+
+                if (result == null) {
+                    throw new RuntimeException(
+                            "Null response found for LocalRearange on tuple: "
+                                    + t);
+                }
+
+                switch (result.returnStatus) {
+                    case POStatus.STATUS_OK:
+                        // (index, key, Tuple(key, value))
+                        Tuple resultTuple = (Tuple) result.result;
+                        Object key = resultTuple.get(1);
+                        IndexedKey indexedKey = new IndexedKey((Byte) resultTuple.get(0), key);
+                        if( useSecondaryKey) {
+                            indexedKey.setUseSecondaryKey(useSecondaryKey);
+                            indexedKey.setSecondarySortOrder(secondarySortOrder);
+                        }
+                        Tuple outValue =  TupleFactory.getInstance().newTuple();
+                        outValue.append(key);
+                        outValue.append(resultTuple.get(2));
+                        Tuple2<IndexedKey, Tuple> out = new Tuple2<IndexedKey, Tuple>(indexedKey,
+                               outValue);
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("LocalRearrangeFunction out " + out);
+                        }
+                        return out;
+                    default:
+                        throw new RuntimeException(
+                                "Unexpected response code from operator "
+                                        + lra + " : " + result);
+                }
+            } catch (ExecException e) {
+                throw new RuntimeException(
+                        "Couldn't do LocalRearange on tuple: " + t, e);
+            }
+        }
+
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SecondaryKeySortUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SecondaryKeySortUtil.java
new file mode 100644
index 0000000..00d29b4
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SecondaryKeySortUtil.java
@@ -0,0 +1,207 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Objects;
+
+import scala.Tuple2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.io.NullableTuple;
+import org.apache.pig.impl.io.PigNullableWritable;
+import org.apache.spark.Partitioner;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+/**
+ * Provide utility functions which is used by ReducedByConverter and JoinGroupSparkConverter.
+ */
+public class SecondaryKeySortUtil {
+    private static final Log LOG = LogFactory
+            .getLog(SecondaryKeySortUtil.class);
+
+    public static RDD<Tuple> handleSecondarySort(
+            RDD<Tuple2<IndexedKey, Tuple>> rdd, POPackage pkgOp) {
+        JavaPairRDD<IndexedKey, Tuple> pairRDD = JavaPairRDD.fromRDD(rdd, SparkUtil.getManifest(IndexedKey.class),
+                SparkUtil.getManifest(Tuple.class));
+
+        int partitionNums = pairRDD.partitions().size();
+        //repartition to group tuples with same indexedkey to same partition
+        JavaPairRDD<IndexedKey, Tuple> sorted = pairRDD.repartitionAndSortWithinPartitions(
+                new IndexedKeyPartitioner(partitionNums));
+        //Package tuples with same indexedkey as the result: (key,(val1,val2,val3,...))
+        return sorted.mapPartitions(new AccumulateByKey(pkgOp), true).rdd();
+    }
+
+    //Package tuples with same indexedkey as the result: (key,(val1,val2,val3,...))
+    //Send (key,Iterator) to POPackage, use POPackage#getNextTuple to get the result
+    private static class AccumulateByKey implements FlatMapFunction<Iterator<Tuple2<IndexedKey, Tuple>>, Tuple>,
+            Serializable {
+        private POPackage pkgOp;
+
+        public AccumulateByKey(POPackage pkgOp) {
+            this.pkgOp = pkgOp;
+        }
+
+        @Override
+        public Iterable<Tuple> call(final Iterator<Tuple2<IndexedKey, Tuple>> it) throws Exception {
+            return new Iterable<Tuple>() {
+                Object curKey = null;
+                ArrayList curValues = new ArrayList();
+                boolean initialized = false;
+
+                @Override
+                public Iterator<Tuple> iterator() {
+                    return new Iterator<Tuple>() {
+
+                        @Override
+                        public boolean hasNext() {
+                            return it.hasNext() || curKey != null;
+                        }
+
+                        @Override
+                        public Tuple next() {
+                            while (it.hasNext()) {
+                                Tuple2<IndexedKey, Tuple> t = it.next();
+                                //key changes, restruct the last tuple by curKey, curValues and return
+                                Object tMainKey = null;
+                                try {
+                                    tMainKey = ((Tuple) (t._1()).getKey()).get(0);
+
+                                    //If the key has changed and we've seen at least 1 already
+                                    if (initialized &&
+                                            ((curKey == null && tMainKey != null) ||
+                                                    (curKey != null && !curKey.equals(tMainKey)))){
+                                        Tuple result = restructTuple(curKey, new ArrayList(curValues));
+                                        curValues.clear();
+                                        curKey = tMainKey;
+                                        curValues.add(t._2());
+                                        return result;
+                                    }
+                                    curKey = tMainKey;
+                                    //if key does not change, just append the value to the same key
+                                    curValues.add(t._2());
+                                    initialized = true;
+
+                                } catch (ExecException e) {
+                                    throw new RuntimeException("AccumulateByKey throw exception: ", e);
+                                }
+                            }
+                            if (!initialized) {
+                                throw new RuntimeException("No tuples seen");
+                            }
+
+                            //if we get here, this should be the last record
+                            Tuple res = restructTuple(curKey, curValues);
+                            curKey = null;
+                            return res;
+                        }
+
+
+                        @Override
+                        public void remove() {
+                            // Not implemented.
+                            // throw Unsupported Method Invocation Exception.
+                            throw new UnsupportedOperationException();
+                        }
+                    };
+                }
+            };
+        }
+
+        private Tuple restructTuple(final Object curKey, final ArrayList<Tuple> curValues) {
+            try {
+                Tuple retVal = null;
+                PigNullableWritable retKey = new PigNullableWritable() {
+
+                    public Object getValueAsPigType() {
+                        return curKey;
+                    }
+                };
+
+                //Here restruct a tupleIterator, later POPackage#tupIter will use it.
+                final Iterator<Tuple> tupleItearator = curValues.iterator();
+                Iterator<NullableTuple> iterator = new Iterator<NullableTuple>() {
+                    public boolean hasNext() {
+                        return tupleItearator.hasNext();
+                    }
+
+                    public NullableTuple next() {
+                        Tuple t = tupleItearator.next();
+                        return new NullableTuple(t);
+                    }
+
+                    public void remove() {
+                        throw new UnsupportedOperationException();
+                    }
+                };
+                pkgOp.setInputs(null);
+                pkgOp.attachInput(retKey, iterator);
+                Result res = pkgOp.getNextTuple();
+                if (res.returnStatus == POStatus.STATUS_OK) {
+                    retVal = (Tuple) res.result;
+                }
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("AccumulateByKey out: " + retVal);
+                }
+                return retVal;
+            } catch (ExecException e) {
+                throw new RuntimeException("AccumulateByKey#restructTuple throws exception: ", e);
+            }
+        }
+    }
+
+    //Group tuples with same IndexKey into same partition
+    private static class IndexedKeyPartitioner extends Partitioner {
+        private int partition;
+
+        public IndexedKeyPartitioner(int partition) {
+            this.partition = partition;
+        }
+
+        @Override
+        public int getPartition(Object obj) {
+            IndexedKey indexedKey = (IndexedKey) obj;
+            Tuple key = (Tuple) indexedKey.getKey();
+
+            int hashCode = 0;
+            try {
+                hashCode = Objects.hashCode(key.get(0));
+            } catch (ExecException e) {
+                throw new RuntimeException("IndexedKeyPartitioner#getPartition: ", e);
+            }
+            return Math.abs(hashCode) % partition;
+        }
+
+        @Override
+        public int numPartitions() {
+            return partition;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SkewedJoinConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SkewedJoinConverter.java
new file mode 100644
index 0000000..c55ba31
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SkewedJoinConverter.java
@@ -0,0 +1,641 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.HashMap;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.impl.builtin.PartitionSkewedKeys;
+import org.apache.pig.impl.util.Pair;
+import org.apache.spark.Partitioner;
+import org.apache.spark.broadcast.Broadcast;
+import scala.Tuple2;
+import scala.runtime.AbstractFunction1;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSkewedJoin;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.pig.impl.plan.NodeIdGenerator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.util.MultiMap;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+public class SkewedJoinConverter implements
+        RDDConverter<Tuple, Tuple, POSkewedJoin>, Serializable {
+
+    private static Log log = LogFactory.getLog(SkewedJoinConverter.class);
+
+    private POLocalRearrange[] LRs;
+    private POSkewedJoin poSkewedJoin;
+
+    private String skewedJoinPartitionFile;
+
+    public void setSkewedJoinPartitionFile(String partitionFile) {
+        skewedJoinPartitionFile = partitionFile;
+    }
+
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+                              POSkewedJoin poSkewedJoin) throws IOException {
+
+        SparkUtil.assertPredecessorSize(predecessors, poSkewedJoin, 2);
+        LRs = new POLocalRearrange[2];
+        this.poSkewedJoin = poSkewedJoin;
+
+        createJoinPlans(poSkewedJoin.getJoinPlans());
+
+        // extract the two RDDs
+        RDD<Tuple> rdd1 = predecessors.get(0);
+        RDD<Tuple> rdd2 = predecessors.get(1);
+
+        Broadcast<List<Tuple>> keyDist = SparkPigContext.get().getBroadcastedVars().get(skewedJoinPartitionFile);
+
+        // if no keyDist,  we need  defaultParallelism
+        Integer defaultParallelism = SparkPigContext.get().getParallelism(predecessors, poSkewedJoin);
+
+        // with partition id
+        SkewPartitionIndexKeyFunction skewFun = new SkewPartitionIndexKeyFunction(this, keyDist, defaultParallelism);
+        RDD<Tuple2<PartitionIndexedKey, Tuple>> skewIdxKeyRDD = rdd1.map(skewFun,
+                SparkUtil.<PartitionIndexedKey, Tuple>getTuple2Manifest());
+
+        // Tuple2 RDD to Pair RDD
+        JavaPairRDD<PartitionIndexedKey, Tuple> skewIndexedJavaPairRDD = new JavaPairRDD<PartitionIndexedKey, Tuple>(
+                skewIdxKeyRDD, SparkUtil.getManifest(PartitionIndexedKey.class),
+                SparkUtil.getManifest(Tuple.class));
+
+        // with partition id
+        StreamPartitionIndexKeyFunction streamFun = new StreamPartitionIndexKeyFunction(this, keyDist, defaultParallelism);
+        JavaRDD<Tuple2<PartitionIndexedKey, Tuple>> streamIdxKeyJavaRDD = rdd2.toJavaRDD().flatMap(streamFun);
+
+        // Tuple2 RDD to Pair RDD
+        JavaPairRDD<PartitionIndexedKey, Tuple> streamIndexedJavaPairRDD = new JavaPairRDD<PartitionIndexedKey, Tuple>(
+                streamIdxKeyJavaRDD.rdd(), SparkUtil.getManifest(PartitionIndexedKey.class),
+                SparkUtil.getManifest(Tuple.class));
+
+        JavaRDD<Tuple> result = doJoin(skewIndexedJavaPairRDD,
+                streamIndexedJavaPairRDD,
+                buildPartitioner(keyDist, defaultParallelism),
+                keyDist);
+
+        // return type is RDD<Tuple>, so take it from JavaRDD<Tuple>
+        return result.rdd();
+    }
+
+    private void createJoinPlans(MultiMap<PhysicalOperator, PhysicalPlan> inpPlans) throws PlanException {
+
+        int i = -1;
+        for (PhysicalOperator inpPhyOp : inpPlans.keySet()) {
+            ++i;
+            POLocalRearrange lr = new POLocalRearrange(genKey());
+            try {
+                lr.setIndex(i);
+            } catch (ExecException e) {
+                throw new PlanException(e.getMessage(), e.getErrorCode(), e.getErrorSource(), e);
+            }
+            lr.setResultType(DataType.TUPLE);
+            lr.setKeyType(DataType.TUPLE);//keyTypes.get(i).size() > 1 ? DataType.TUPLE : keyTypes.get(i).get(0));
+            lr.setPlans(inpPlans.get(inpPhyOp));
+            LRs[i] = lr;
+        }
+    }
+
+    private OperatorKey genKey() {
+        return new OperatorKey(poSkewedJoin.getOperatorKey().scope, NodeIdGenerator.getGenerator().getNextNodeId(poSkewedJoin.getOperatorKey().scope));
+    }
+
+    /**
+     * @param <L> be generic because it can be Optional<Tuple> or Tuple
+     * @param <R> be generic because it can be Optional<Tuple> or Tuple
+     */
+    private static class ToValueFunction<L, R> implements
+            FlatMapFunction<Iterator<Tuple2<PartitionIndexedKey, Tuple2<L, R>>>, Tuple>, Serializable {
+
+        private boolean[] innerFlags;
+        private int[] schemaSize;
+
+        private final Broadcast<List<Tuple>> keyDist;
+
+        transient private boolean initialized = false;
+        transient protected Map<Tuple, Pair<Integer, Integer>> reducerMap;
+
+        public ToValueFunction(boolean[] innerFlags, int[] schemaSize, Broadcast<List<Tuple>> keyDist) {
+            this.innerFlags = innerFlags;
+            this.schemaSize = schemaSize;
+            this.keyDist = keyDist;
+        }
+
+        private class Tuple2TransformIterable implements Iterable<Tuple> {
+
+            Iterator<Tuple2<PartitionIndexedKey, Tuple2<L, R>>> in;
+
+            Tuple2TransformIterable(
+                    Iterator<Tuple2<PartitionIndexedKey, Tuple2<L, R>>> input) {
+                in = input;
+            }
+
+            public Iterator<Tuple> iterator() {
+                return new IteratorTransform<Tuple2<PartitionIndexedKey, Tuple2<L, R>>, Tuple>(
+                        in) {
+                    @Override
+                    protected Tuple transform(
+                            Tuple2<PartitionIndexedKey, Tuple2<L, R>> next) {
+                        try {
+
+                            L left = next._2._1;
+                            R right = next._2._2;
+
+                            TupleFactory tf = TupleFactory.getInstance();
+                            Tuple result = tf.newTuple();
+
+                            Tuple leftTuple = tf.newTuple();
+                            if (!innerFlags[0]) {
+                                // left should be Optional<Tuple>
+                                Optional<Tuple> leftOption = (Optional<Tuple>) left;
+                                if (!leftOption.isPresent()) {
+                                    // Add an empty left record for RIGHT OUTER JOIN.
+                                    // Notice: if it is a skewed, only join the first reduce key
+                                    if (isFirstReduceKey(next._1)) {
+                                        for (int i = 0; i < schemaSize[0]; i++) {
+                                            leftTuple.append(null);
+                                        }
+                                    } else {
+                                        return this.next();
+                                    }
+                                } else {
+                                    leftTuple = leftOption.get();
+                                }
+                            } else {
+                                leftTuple = (Tuple) left;
+                            }
+                            for (int i = 0; i < leftTuple.size(); i++) {
+                                result.append(leftTuple.get(i));
+                            }
+
+                            Tuple rightTuple = tf.newTuple();
+                            if (!innerFlags[1]) {
+                                // right should be Optional<Tuple>
+                                Optional<Tuple> rightOption = (Optional<Tuple>) right;
+                                if (!rightOption.isPresent()) {
+                                    for (int i = 0; i < schemaSize[1]; i++) {
+                                        rightTuple.append(null);
+                                    }
+                                } else {
+                                    rightTuple = rightOption.get();
+                                }
+                            } else {
+                                rightTuple = (Tuple) right;
+                            }
+                            for (int i = 0; i < rightTuple.size(); i++) {
+                                result.append(rightTuple.get(i));
+                            }
+
+                            if (log.isDebugEnabled()) {
+                                log.debug("MJC: Result = " + result.toDelimitedString(" "));
+                            }
+
+                            return result;
+                        } catch (Exception e) {
+                            log.warn(e);
+                        }
+                        return null;
+                    }
+                };
+            }
+        }
+
+        @Override
+        public Iterable<Tuple> call(
+                Iterator<Tuple2<PartitionIndexedKey, Tuple2<L, R>>> input) {
+            return new Tuple2TransformIterable(input);
+        }
+
+        private boolean isFirstReduceKey(PartitionIndexedKey pKey) {
+            // non-skewed key
+            if (pKey.getPartitionId() == -1) {
+                return true;
+            }
+
+            if (!initialized) {
+                Integer[] reducers = new Integer[1];
+                reducerMap = loadKeyDistribution(keyDist, reducers);
+                initialized = true;
+            }
+
+            Pair<Integer, Integer> indexes = reducerMap.get(pKey.getKey());
+            if (indexes != null && pKey.getPartitionId() != indexes.first) {
+                // return false only when the key is skewed
+                // and it is not the first reduce key.
+                return false;
+            }
+
+            return true;
+        }
+    }
+
+    /**
+     * Utility function.
+     * 1. Get parallelism
+     * 2. build a key distribution map from the broadcasted key distribution file
+     *
+     * @param keyDist
+     * @param totalReducers
+     * @return
+     */
+    private static Map<Tuple, Pair<Integer, Integer>> loadKeyDistribution(Broadcast<List<Tuple>> keyDist,
+                                                                          Integer[] totalReducers) {
+        Map<Tuple, Pair<Integer, Integer>> reducerMap = new HashMap<>();
+        totalReducers[0] = -1; // set a default value
+
+        if (keyDist == null || keyDist.value() == null || keyDist.value().size() == 0) {
+            // this could happen if sampling is empty
+            log.warn("Empty dist file: ");
+            return reducerMap;
+        }
+
+        try {
+            final TupleFactory tf = TupleFactory.getInstance();
+
+            Tuple t = keyDist.value().get(0);
+
+            Map<String, Object> distMap = (Map<String, Object>) t.get(0);
+            DataBag partitionList = (DataBag) distMap.get(PartitionSkewedKeys.PARTITION_LIST);
+
+            totalReducers[0] = Integer.valueOf("" + distMap.get(PartitionSkewedKeys.TOTAL_REDUCERS));
+
+            Iterator<Tuple> it = partitionList.iterator();
+            while (it.hasNext()) {
+                Tuple idxTuple = it.next();
+                Integer maxIndex = (Integer) idxTuple.get(idxTuple.size() - 1);
+                Integer minIndex = (Integer) idxTuple.get(idxTuple.size() - 2);
+                // Used to replace the maxIndex with the number of reducers
+                if (maxIndex < minIndex) {
+                    maxIndex = totalReducers[0] + maxIndex;
+                }
+
+                // remove the last 2 fields of the tuple, i.e: minIndex and maxIndex and store
+                // it in the reducer map
+                Tuple keyTuple = tf.newTuple();
+                for (int i = 0; i < idxTuple.size() - 2; i++) {
+                    keyTuple.append(idxTuple.get(i));
+                }
+
+                // number of reducers
+                Integer cnt = maxIndex - minIndex;
+                reducerMap.put(keyTuple, new Pair(minIndex, cnt));
+            }
+
+        } catch (ExecException e) {
+            log.warn(e.getMessage());
+        }
+
+        return reducerMap;
+    }
+
+    private static class PartitionIndexedKey extends IndexedKey {
+        // for user defined partitioner
+        int partitionId;
+
+        public PartitionIndexedKey(byte index, Object key) {
+            super(index, key);
+            partitionId = -1;
+        }
+
+        public PartitionIndexedKey(byte index, Object key, int pid) {
+            super(index, key);
+            partitionId = pid;
+        }
+
+        public int getPartitionId() {
+            return partitionId;
+        }
+
+        private void setPartitionId(int pid) {
+            partitionId = pid;
+        }
+
+        @Override
+        public String toString() {
+            return "PartitionIndexedKey{" +
+                    "index=" + getIndex() +
+                    ", partitionId=" + getPartitionId() +
+                    ", key=" + getKey() +
+                    '}';
+        }
+    }
+
+    /**
+     * append a Partition id to the records from skewed table.
+     * so that the SkewedJoinPartitioner can send skewed records to different reducer
+     * <p>
+     * see: https://wiki.apache.org/pig/PigSkewedJoinSpec
+     */
+    private static class SkewPartitionIndexKeyFunction extends
+            AbstractFunction1<Tuple, Tuple2<PartitionIndexedKey, Tuple>> implements
+            Serializable {
+
+        private final SkewedJoinConverter poSkewedJoin;
+
+        private final Broadcast<List<Tuple>> keyDist;
+        private final Integer defaultParallelism;
+
+        transient private boolean initialized = false;
+        transient protected Map<Tuple, Pair<Integer, Integer>> reducerMap;
+        transient private Integer parallelism = -1;
+        transient private Map<Tuple, Integer> currentIndexMap;
+
+        public SkewPartitionIndexKeyFunction(SkewedJoinConverter poSkewedJoin,
+                                             Broadcast<List<Tuple>> keyDist,
+                                             Integer defaultParallelism) {
+            this.poSkewedJoin = poSkewedJoin;
+            this.keyDist = keyDist;
+            this.defaultParallelism = defaultParallelism;
+        }
+
+        @Override
+        public Tuple2<PartitionIndexedKey, Tuple> apply(Tuple tuple) {
+            // attach tuple to LocalRearrange
+            poSkewedJoin.LRs[0].attachInput(tuple);
+
+            try {
+                Result lrOut = poSkewedJoin.LRs[0].getNextTuple();
+
+                // If tuple is (AA, 5) and key index is $1, then it lrOut is 0 5
+                // (AA), so get(1) returns key
+                Byte index = (Byte) ((Tuple) lrOut.result).get(0);
+                Object key = ((Tuple) lrOut.result).get(1);
+
+                Tuple keyTuple = (Tuple) key;
+                int partitionId = getPartitionId(keyTuple);
+                PartitionIndexedKey pIndexKey = new PartitionIndexedKey(index, keyTuple, partitionId);
+
+                // make a (key, value) pair
+                Tuple2<PartitionIndexedKey, Tuple> tuple_KeyValue = new Tuple2<PartitionIndexedKey, Tuple>(
+                        pIndexKey,
+                        tuple);
+
+                return tuple_KeyValue;
+            } catch (Exception e) {
+                System.out.print(e);
+                return null;
+            }
+        }
+
+        private Integer getPartitionId(Tuple keyTuple) {
+            if (!initialized) {
+                Integer[] reducers = new Integer[1];
+                reducerMap = loadKeyDistribution(keyDist, reducers);
+                parallelism = reducers[0];
+
+                if (parallelism <= 0) {
+                    parallelism = defaultParallelism;
+                }
+
+                currentIndexMap = Maps.newHashMap();
+
+                initialized = true;
+            }
+
+            // for partition table, compute the index based on the sampler output
+            Pair<Integer, Integer> indexes;
+            Integer curIndex = -1;
+
+            indexes = reducerMap.get(keyTuple);
+
+            // if the reducerMap does not contain the key return -1 so that the
+            // partitioner will do the default hash based partitioning
+            if (indexes == null) {
+                return -1;
+            }
+
+            if (currentIndexMap.containsKey(keyTuple)) {
+                curIndex = currentIndexMap.get(keyTuple);
+            }
+
+            if (curIndex >= (indexes.first + indexes.second) || curIndex == -1) {
+                curIndex = indexes.first;
+            } else {
+                curIndex++;
+            }
+
+            // set it in the map
+            currentIndexMap.put(keyTuple, curIndex);
+            return (curIndex % parallelism);
+        }
+
+    }
+
+    /**
+     * POPartitionRearrange is not used in spark mode now,
+     * Here, use flatMap and CopyStreamWithPidFunction to copy the
+     * stream records to the multiple reducers
+     * <p>
+     * see: https://wiki.apache.org/pig/PigSkewedJoinSpec
+     */
+    private static class StreamPartitionIndexKeyFunction implements FlatMapFunction<Tuple, Tuple2<PartitionIndexedKey, Tuple>> {
+
+        private SkewedJoinConverter poSkewedJoin;
+        private final Broadcast<List<Tuple>> keyDist;
+        private final Integer defaultParallelism;
+
+        private transient boolean initialized = false;
+        protected transient Map<Tuple, Pair<Integer, Integer>> reducerMap;
+        private transient Integer parallelism;
+
+        public StreamPartitionIndexKeyFunction(SkewedJoinConverter poSkewedJoin,
+                                               Broadcast<List<Tuple>> keyDist,
+                                               Integer defaultParallelism) {
+            this.poSkewedJoin = poSkewedJoin;
+            this.keyDist = keyDist;
+            this.defaultParallelism = defaultParallelism;
+        }
+
+        public Iterable<Tuple2<PartitionIndexedKey, Tuple>> call(Tuple tuple) throws Exception {
+            if (!initialized) {
+                Integer[] reducers = new Integer[1];
+                reducerMap = loadKeyDistribution(keyDist, reducers);
+                parallelism = reducers[0];
+                if (parallelism <= 0) {
+                    parallelism = defaultParallelism;
+                }
+                initialized = true;
+            }
+
+            // streamed table
+            poSkewedJoin.LRs[1].attachInput(tuple);
+            Result lrOut = poSkewedJoin.LRs[1].getNextTuple();
+
+            Byte index = (Byte) ((Tuple) lrOut.result).get(0);
+            Tuple key = (Tuple) ((Tuple) lrOut.result).get(1);
+
+            ArrayList<Tuple2<PartitionIndexedKey, Tuple>> l = new ArrayList();
+            Pair<Integer, Integer> indexes = reducerMap.get(key);
+
+            // For non skewed keys, we set the partition index to be -1
+            // so that the partitioner will do the default hash based partitioning
+            if (indexes == null) {
+                indexes = new Pair<>(-1, 0);
+            }
+
+            for (Integer reducerIdx = indexes.first, cnt = 0; cnt <= indexes.second; reducerIdx++, cnt++) {
+                if (reducerIdx >= parallelism) {
+                    reducerIdx = 0;
+                }
+
+                // set the partition index
+                int partitionId = reducerIdx.intValue();
+                PartitionIndexedKey pIndexKey = new PartitionIndexedKey(index, key, partitionId);
+
+                l.add(new Tuple2(pIndexKey, tuple));
+            }
+
+            return l;
+        }
+    }
+
+    /**
+     * user defined spark partitioner for skewed join
+     */
+    private static class SkewedJoinPartitioner extends Partitioner {
+        private int numPartitions;
+
+        public SkewedJoinPartitioner(int parallelism) {
+            numPartitions = parallelism;
+        }
+
+        @Override
+        public int numPartitions() {
+            return numPartitions;
+        }
+
+        @Override
+        public int getPartition(Object IdxKey) {
+            if (IdxKey instanceof PartitionIndexedKey) {
+                int partitionId = ((PartitionIndexedKey) IdxKey).getPartitionId();
+                if (partitionId >= 0) {
+                    return partitionId;
+                }
+            }
+
+            //else: by default using hashcode
+            Tuple key = (Tuple) ((PartitionIndexedKey) IdxKey).getKey();
+
+
+            int code = key.hashCode() % numPartitions;
+            if (code >= 0) {
+                return code;
+            } else {
+                return code + numPartitions;
+            }
+        }
+    }
+
+    /**
+     * use parallelism from keyDist or the default parallelism to
+     * create user defined partitioner
+     *
+     * @param keyDist
+     * @param defaultParallelism
+     * @return
+     */
+    private SkewedJoinPartitioner buildPartitioner(Broadcast<List<Tuple>> keyDist, Integer defaultParallelism) {
+        Integer parallelism = -1;
+        Integer[] reducers = new Integer[1];
+        loadKeyDistribution(keyDist, reducers);
+        parallelism = reducers[0];
+        if (parallelism <= 0) {
+            parallelism = defaultParallelism;
+        }
+
+        return new SkewedJoinPartitioner(parallelism);
+    }
+
+    /**
+     * do all kinds of Join (inner, left outer, right outer, full outer)
+     *
+     * @param skewIndexedJavaPairRDD
+     * @param streamIndexedJavaPairRDD
+     * @param partitioner
+     * @return
+     */
+    private JavaRDD<Tuple> doJoin(
+            JavaPairRDD<PartitionIndexedKey, Tuple> skewIndexedJavaPairRDD,
+            JavaPairRDD<PartitionIndexedKey, Tuple> streamIndexedJavaPairRDD,
+            SkewedJoinPartitioner partitioner,
+            Broadcast<List<Tuple>> keyDist) {
+
+        boolean[] innerFlags = poSkewedJoin.getInnerFlags();
+        int[] schemaSize = {0, 0};
+        for (int i = 0; i < 2; i++) {
+            if (poSkewedJoin.getSchema(i) != null) {
+                schemaSize[i] = poSkewedJoin.getSchema(i).size();
+            }
+        }
+
+        ToValueFunction toValueFun = new ToValueFunction(innerFlags, schemaSize, keyDist);
+
+        if (innerFlags[0] && innerFlags[1]) {
+            // inner join
+            JavaPairRDD<PartitionIndexedKey, Tuple2<Tuple, Tuple>> resultKeyValue = skewIndexedJavaPairRDD.
+                    join(streamIndexedJavaPairRDD, partitioner);
+
+            return resultKeyValue.mapPartitions(toValueFun);
+        } else if (innerFlags[0] && !innerFlags[1]) {
+            // left outer join
+            JavaPairRDD<PartitionIndexedKey, Tuple2<Tuple, Optional<Tuple>>> resultKeyValue = skewIndexedJavaPairRDD.
+                    leftOuterJoin(streamIndexedJavaPairRDD, partitioner);
+
+            return resultKeyValue.mapPartitions(toValueFun);
+        } else if (!innerFlags[0] && innerFlags[1]) {
+            // right outer join
+            JavaPairRDD<PartitionIndexedKey, Tuple2<Optional<Tuple>, Tuple>> resultKeyValue = skewIndexedJavaPairRDD.
+                    rightOuterJoin(streamIndexedJavaPairRDD, partitioner);
+
+            return resultKeyValue.mapPartitions(toValueFun);
+        } else {
+            // full outer join
+            JavaPairRDD<PartitionIndexedKey, Tuple2<Optional<Tuple>, Optional<Tuple>>> resultKeyValue = skewIndexedJavaPairRDD.
+                    fullOuterJoin(streamIndexedJavaPairRDD, partitioner);
+
+            return resultKeyValue.mapPartitions(toValueFun);
+        }
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SortConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SortConverter.java
new file mode 100644
index 0000000..baabfa0
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SortConverter.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.runtime.AbstractFunction1;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigContext;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+@SuppressWarnings("serial")
+public class SortConverter implements RDDConverter<Tuple, Tuple, POSort> {
+    private static final Log LOG = LogFactory.getLog(SortConverter.class);
+
+    private static final FlatMapFunction<Iterator<Tuple2<Tuple, Object>>, Tuple> TO_VALUE_FUNCTION = new ToValueFunction();
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POSort sortOperator)
+            throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, sortOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        int parallelism = SparkPigContext.get().getParallelism(predecessors, sortOperator);
+        RDD<Tuple2<Tuple, Object>> rddPair = rdd.map(new ToKeyValueFunction(),
+                SparkUtil.<Tuple, Object> getTuple2Manifest());
+
+        JavaPairRDD<Tuple, Object> r = new JavaPairRDD<Tuple, Object>(rddPair,
+                SparkUtil.getManifest(Tuple.class),
+                SparkUtil.getManifest(Object.class));
+
+        JavaPairRDD<Tuple, Object> sorted = r.sortByKey(
+                sortOperator.getMComparator(), true, parallelism);
+        JavaRDD<Tuple> mapped = sorted.mapPartitions(TO_VALUE_FUNCTION);
+
+        return mapped.rdd();
+    }
+
+    private static class ToValueFunction implements
+            FlatMapFunction<Iterator<Tuple2<Tuple, Object>>, Tuple>, Serializable {
+
+        private class Tuple2TransformIterable implements Iterable<Tuple> {
+
+            Iterator<Tuple2<Tuple, Object>> in;
+
+            Tuple2TransformIterable(Iterator<Tuple2<Tuple, Object>> input) {
+                in = input;
+            }
+
+            public Iterator<Tuple> iterator() {
+                return new IteratorTransform<Tuple2<Tuple, Object>, Tuple>(in) {
+                    @Override
+                    protected Tuple transform(Tuple2<Tuple, Object> next) {
+                        return next._1();
+                    }
+                };
+            }
+        }
+
+        @Override
+        public Iterable<Tuple> call(Iterator<Tuple2<Tuple, Object>> input) {
+            return new Tuple2TransformIterable(input);
+        }
+    }
+
+    private static class ToKeyValueFunction extends
+            AbstractFunction1<Tuple, Tuple2<Tuple, Object>> implements
+            Serializable {
+
+        @Override
+        public Tuple2<Tuple, Object> apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Sort ToKeyValueFunction in " + t);
+            }
+            Tuple key = t;
+            Object value = null;
+            // (key, value)
+            Tuple2<Tuple, Object> out = new Tuple2<Tuple, Object>(key, value);
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Sort ToKeyValueFunction out " + out);
+            }
+            return out;
+        }
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SparkSampleSortConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SparkSampleSortConverter.java
new file mode 100644
index 0000000..3166fdc
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SparkSampleSortConverter.java
@@ -0,0 +1,155 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import scala.Tuple2;
+import scala.runtime.AbstractFunction1;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POSampleSortSpark;
+import org.apache.pig.data.BagFactory;
+import org.apache.pig.data.DataBag;
+import org.apache.pig.data.DefaultBagFactory;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import org.apache.spark.api.java.JavaPairRDD;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.api.java.function.PairFlatMapFunction;
+import org.apache.spark.rdd.RDD;
+  /*
+   sort the sample data and convert the sample data to the format (all,{(sampleEle1),(sampleEle2),...})
+
+   */
+@SuppressWarnings("serial")
+public class SparkSampleSortConverter implements RDDConverter<Tuple, Tuple, POSampleSortSpark> {
+    private static final Log LOG = LogFactory.getLog(SparkSampleSortConverter.class);
+    private static TupleFactory tf = TupleFactory.getInstance();
+    private static BagFactory bf = DefaultBagFactory.getInstance();
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POSampleSortSpark sortOperator)
+            throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, sortOperator, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        RDD<Tuple2<Tuple, Object>> rddPair = rdd.map(new ToKeyValueFunction(),
+                SparkUtil.<Tuple, Object> getTuple2Manifest());
+
+        JavaPairRDD<Tuple, Object> r = new JavaPairRDD<Tuple, Object>(rddPair,
+                SparkUtil.getManifest(Tuple.class),
+                SparkUtil.getManifest(Object.class));
+         //sort sample data
+        JavaPairRDD<Tuple, Object> sorted = r.sortByKey(true);
+         //convert every element in sample data from element to (all, element) format
+        JavaPairRDD<String, Tuple> mapped = sorted.mapPartitionsToPair(new AggregateFunction());
+        //use groupByKey to aggregate all values( the format will be ((all),{(sampleEle1),(sampleEle2),...} )
+        JavaRDD<Tuple> groupByKey= mapped.groupByKey().map(new ToValueFunction());
+        return  groupByKey.rdd();
+    }
+
+
+    private static class MergeFunction implements org.apache.spark.api.java.function.Function2<Tuple, Tuple, Tuple>
+            , Serializable {
+
+        @Override
+        public Tuple call(Tuple v1, Tuple v2) {
+                Tuple res = tf.newTuple();
+                res.append(v1);
+                res.append(v2);
+                LOG.info("MergeFunction out:"+res);
+                return res;
+        }
+    }
+
+    // input: Tuple2<Tuple,Object>
+    // output: Tuple2("all", Tuple)
+    private static class AggregateFunction implements
+            PairFlatMapFunction<Iterator<Tuple2<Tuple, Object>>, String,Tuple>, Serializable {
+
+        private class Tuple2TransformIterable implements Iterable<Tuple2<String,Tuple>> {
+
+            Iterator<Tuple2<Tuple, Object>> in;
+
+            Tuple2TransformIterable(Iterator<Tuple2<Tuple, Object>> input) {
+                in = input;
+            }
+
+            public Iterator<Tuple2<String,Tuple>> iterator() {
+                return new IteratorTransform<Tuple2<Tuple, Object>, Tuple2<String,Tuple>>(in) {
+                    @Override
+                    protected Tuple2<String,Tuple> transform(Tuple2<Tuple, Object> next) {
+                        LOG.info("AggregateFunction in:"+ next._1()) ;
+                        return new Tuple2<String,Tuple>("all",next._1());
+                    }
+                };
+            }
+        }
+
+        @Override
+        public Iterable<Tuple2<String, Tuple>> call(Iterator<Tuple2<Tuple, Object>> input) throws Exception {
+            return new Tuple2TransformIterable(input);
+        }
+
+    }
+
+    private static class ToValueFunction implements Function<Tuple2<String, Iterable<Tuple>>, Tuple> {
+        @Override
+        public Tuple call(Tuple2<String, Iterable<Tuple>> next) throws Exception {
+            Tuple res = tf.newTuple();
+            res.append(next._1());
+            Iterator<Tuple> iter = next._2().iterator();
+            DataBag bag = bf.newDefaultBag();
+            while(iter.hasNext()) {
+                bag.add(iter.next());
+            }
+            res.append(bag);
+            LOG.info("ToValueFunction1 out:" + res);
+            return res;
+        }
+    }
+
+    private static class ToKeyValueFunction extends
+            AbstractFunction1<Tuple, Tuple2<Tuple, Object>> implements
+            Serializable {
+
+        @Override
+        public Tuple2<Tuple, Object> apply(Tuple t) {
+            if (LOG.isDebugEnabled()) {
+                LOG.info("Sort ToKeyValueFunction in " + t);
+            }
+            Tuple key = t;
+            Object value = null;
+            // (key, value)
+            Tuple2<Tuple, Object> out = new Tuple2<Tuple, Object>(key, value);
+            if (LOG.isDebugEnabled()) {
+                LOG.info("Sort ToKeyValueFunction out " + out);
+            }
+            return out;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SplitConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SplitConverter.java
new file mode 100644
index 0000000..4a2c061
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/SplitConverter.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.rdd.RDD;
+
+public class SplitConverter implements RDDConverter<Tuple, Tuple, POSplit> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors, POSplit poSplit)
+            throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, poSplit, 1);
+        return predecessors.get(0);
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/StoreConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/StoreConverter.java
new file mode 100644
index 0000000..bc376d3
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/StoreConverter.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.pig.tools.pigstats.spark.SparkCounters;
+import org.apache.pig.tools.pigstats.spark.SparkPigStatusReporter;
+import org.apache.pig.tools.pigstats.spark.SparkStatsUtil;
+import scala.Tuple2;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.FileOutputFormat;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.lib.output.LazyOutputFormat;
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.StoreFuncInterface;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.JobControlCompiler;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigOutputFormat;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.spark.api.java.JavaRDD;
+import org.apache.spark.api.java.function.Function;
+import org.apache.spark.rdd.PairRDDFunctions;
+import org.apache.spark.rdd.RDD;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Converter that takes a POStore and stores it's content.
+ */
+@SuppressWarnings({ "serial" })
+public class StoreConverter implements
+        RDDConverter<Tuple, Tuple2<Text, Tuple>, POStore> {
+
+    private static final Log LOG = LogFactory.getLog(StoreConverter.class);
+
+    private JobConf jobConf = null;
+    public StoreConverter(JobConf jobConf) {
+        this.jobConf = jobConf;
+    }
+
+    @Override
+    public RDD<Tuple2<Text, Tuple>> convert(List<RDD<Tuple>> predecessors,
+            POStore op) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, op, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+
+        SparkPigStatusReporter.getInstance().createCounter(SparkStatsUtil.SPARK_STORE_COUNTER_GROUP,
+                SparkStatsUtil.getCounterName(op));
+
+        // convert back to KV pairs
+        JavaRDD<Tuple2<Text, Tuple>> rddPairs = rdd.toJavaRDD().map(
+                buildFromTupleFunction(op));
+
+        PairRDDFunctions<Text, Tuple> pairRDDFunctions = new PairRDDFunctions<Text, Tuple>(
+                rddPairs.rdd(), SparkUtil.getManifest(Text.class),
+                SparkUtil.getManifest(Tuple.class), null);
+
+        POStore poStore = configureStorer(jobConf, op);
+
+        if ("true".equalsIgnoreCase(jobConf
+                .get(PigConfiguration.PIG_OUTPUT_LAZY))) {
+            Job storeJob = new Job(jobConf);
+            LazyOutputFormat.setOutputFormatClass(storeJob,
+                    PigOutputFormat.class);
+            jobConf = (JobConf) storeJob.getConfiguration();
+            jobConf.setOutputKeyClass(Text.class);
+            jobConf.setOutputValueClass(Tuple.class);
+            String fileName = poStore.getSFile().getFileName();
+            Path filePath = new Path(fileName);
+            FileOutputFormat.setOutputPath(jobConf,filePath);
+            pairRDDFunctions.saveAsNewAPIHadoopDataset(jobConf);
+        } else {
+            pairRDDFunctions.saveAsNewAPIHadoopFile(poStore.getSFile()
+                    .getFileName(), Text.class, Tuple.class,
+                    PigOutputFormat.class, jobConf);
+        }
+
+        RDD<Tuple2<Text, Tuple>> retRdd = rddPairs.rdd();
+        if (LOG.isDebugEnabled())
+            LOG.debug("RDD lineage: " + retRdd.toDebugString());
+        return retRdd;
+    }
+
+
+    private static POStore configureStorer(JobConf jobConf,
+            PhysicalOperator op) throws IOException {
+        ArrayList<POStore> storeLocations = Lists.newArrayList();
+        POStore poStore = (POStore) op;
+        storeLocations.add(poStore);
+        StoreFuncInterface sFunc = poStore.getStoreFunc();
+        sFunc.setStoreLocation(poStore.getSFile().getFileName(),
+                new org.apache.hadoop.mapreduce.Job(jobConf));
+        poStore.setInputs(null);
+        poStore.setParentPlan(null);
+
+        jobConf.set(JobControlCompiler.PIG_MAP_STORES,
+                ObjectSerializer.serialize(Lists.newArrayList()));
+        jobConf.set(JobControlCompiler.PIG_REDUCE_STORES,
+                ObjectSerializer.serialize(storeLocations));
+        return poStore;
+    }
+
+    private static class FromTupleFunction implements
+            Function<Tuple, Tuple2<Text, Tuple>> {
+
+        private static Text EMPTY_TEXT = new Text();
+        private String counterGroupName;
+        private String counterName;
+        private SparkCounters sparkCounters;
+        private boolean disableCounter;
+
+
+        public Tuple2<Text, Tuple> call(Tuple v1) {
+            if (sparkCounters != null && disableCounter == false) {
+                sparkCounters.increment(counterGroupName, counterName, 1L);
+            }
+            return new Tuple2<Text, Tuple>(EMPTY_TEXT, v1);
+        }
+
+        public void setCounterGroupName(String counterGroupName) {
+            this.counterGroupName = counterGroupName;
+        }
+
+        public void setCounterName(String counterName) {
+            this.counterName = counterName;
+        }
+
+        public void setSparkCounters(SparkCounters sparkCounter) {
+            this.sparkCounters = sparkCounter;
+        }
+
+        public void setDisableCounter(boolean disableCounter) {
+            this.disableCounter = disableCounter;
+        }
+    }
+
+    private FromTupleFunction buildFromTupleFunction(POStore op) {
+        FromTupleFunction ftf = new FromTupleFunction();
+        boolean disableCounter = op.disableCounter();
+        if (!op.isTmpStore() && !disableCounter) {
+            ftf.setDisableCounter(disableCounter);
+            ftf.setCounterGroupName(SparkStatsUtil.SPARK_STORE_COUNTER_GROUP);
+            ftf.setCounterName(SparkStatsUtil.getCounterName(op));
+            SparkPigStatusReporter counterReporter = SparkPigStatusReporter.getInstance();
+            ftf.setSparkCounters(counterReporter.getCounters());
+        }
+        return ftf;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/StreamConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/StreamConverter.java
new file mode 100644
index 0000000..3a50d48
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/StreamConverter.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.api.java.function.FlatMapFunction;
+import org.apache.spark.rdd.RDD;
+
+public class StreamConverter implements
+        RDDConverter<Tuple, Tuple, POStream> {
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            POStream poStream) throws IOException {
+        SparkUtil.assertPredecessorSize(predecessors, poStream, 1);
+        RDD<Tuple> rdd = predecessors.get(0);
+        StreamFunction streamFunction = new StreamFunction(poStream);
+        return rdd.toJavaRDD().mapPartitions(streamFunction, true).rdd();
+    }
+
+    private static class StreamFunction implements
+            FlatMapFunction<Iterator<Tuple>, Tuple>, Serializable {
+        private POStream poStream;
+
+        private StreamFunction(POStream poStream) {
+            this.poStream = poStream;
+        }
+
+        public Iterable<Tuple> call(final Iterator<Tuple> input) {
+            return new Iterable<Tuple>() {
+                @Override
+                public Iterator<Tuple> iterator() {
+                    return new OutputConsumerIterator(input) {
+
+                        @Override
+                        protected void attach(Tuple tuple) {
+                            poStream.setInputs(null);
+                            poStream.attachInput(tuple);
+                        }
+
+                        @Override
+                        protected Result getNextResult() throws ExecException {
+                            Result result = poStream.getNextTuple();
+                            return result;
+                        }
+
+                        @Override
+                        protected void endOfInput() {
+                            poStream.setFetchable(true);
+                        }
+                    };
+                }
+            };
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/UnionConverter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/UnionConverter.java
new file mode 100644
index 0000000..2be49a6
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/converter/UnionConverter.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.converter;
+
+import java.io.IOException;
+import java.util.List;
+
+import scala.collection.JavaConversions;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.data.Tuple;
+import org.apache.spark.SparkContext;
+import org.apache.spark.rdd.RDD;
+import org.apache.spark.rdd.UnionRDD;
+
+public class UnionConverter implements RDDConverter<Tuple, Tuple, POUnion> {
+
+    private final SparkContext sc;
+
+    public UnionConverter(SparkContext sc) {
+        this.sc = sc;
+    }
+
+    @Override
+    public RDD<Tuple> convert(List<RDD<Tuple>> predecessors,
+            POUnion physicalOperator) throws IOException {
+        SparkUtil.assertPredecessorSizeGreaterThan(predecessors,
+                physicalOperator, 0);
+        UnionRDD<Tuple> unionRDD = new UnionRDD<Tuple>(sc,
+                JavaConversions.asScalaBuffer(predecessors),
+                SparkUtil.getManifest(Tuple.class));
+        return unionRDD;
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/NativeSparkOperator.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/NativeSparkOperator.java
new file mode 100644
index 0000000..e4ba245
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/NativeSparkOperator.java
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.operator;
+
+import org.apache.hadoop.util.RunJar;
+import org.apache.pig.backend.hadoop.executionengine.JobCreationException;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.RunJarSecurityManager;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.tools.pigstats.PigStats;
+import org.apache.pig.tools.pigstats.spark.SparkStatsUtil;
+
+/**
+ * NativeSparkOperator:
+ */
+public class NativeSparkOperator extends SparkOperator {
+    private static final long serialVersionUID = 1L;
+    private static int countJobs = 0;
+    private String nativeSparkJar;
+    private String[] params;
+    private String jobId;
+
+    public NativeSparkOperator(OperatorKey k, String sparkJar, String[] parameters) {
+        super(k);
+        nativeSparkJar = sparkJar;
+        params = parameters;
+        jobId = sparkJar + "_" + getJobNumber();
+    }
+
+    private static int getJobNumber() {
+        countJobs++;
+        return countJobs;
+    }
+
+    public String getJobId() {
+        return jobId;
+    }
+
+    public void runJob() throws JobCreationException {
+        RunJarSecurityManager secMan = new RunJarSecurityManager();
+        try {
+            RunJar.main(getNativeMRParams());
+            SparkStatsUtil.addNativeJobStats(PigStats.get(), this);
+        } catch (SecurityException se) {   //java.lang.reflect.InvocationTargetException
+            if (secMan.getExitInvoked()) {
+                if (secMan.getExitCode() != 0) {
+                    JobCreationException e = new JobCreationException("Native job returned with non-zero return code");
+                    SparkStatsUtil.addFailedNativeJobStats(PigStats.get(), this, e);
+                } else {
+                    SparkStatsUtil.addNativeJobStats(PigStats.get(), this);
+                }
+            }
+        } catch (Throwable t) {
+            JobCreationException e = new JobCreationException(
+                    "Cannot run native spark job " + t.getMessage(), t);
+            SparkStatsUtil.addFailedNativeJobStats(PigStats.get(), this, e);
+            throw e;
+        } finally {
+            secMan.retire();
+        }
+    }
+
+    private String[] getNativeMRParams() {
+        String[] paramArr = new String[params.length + 1];
+        paramArr[0] = nativeSparkJar;
+        for (int i = 0; i < params.length; i++) {
+            paramArr[i + 1] = params[i];
+        }
+        return paramArr;
+    }
+
+    public String getCommandString() {
+        StringBuilder sb = new StringBuilder("hadoop jar ");
+        sb.append(nativeSparkJar);
+        for (String pr : params) {
+            sb.append(" ");
+            sb.append(pr);
+        }
+        return sb.toString();
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POGlobalRearrangeSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POGlobalRearrangeSpark.java
new file mode 100644
index 0000000..4c9005c
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POGlobalRearrangeSpark.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.operator;
+
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+
+/**
+ * POGlobalRearrange for spark mode
+ */
+public class POGlobalRearrangeSpark extends POGlobalRearrange {
+    // Use secondary key
+    private boolean useSecondaryKey;
+    // Sort order for secondary keys;
+    private boolean[] secondarySortOrder;
+
+    public POGlobalRearrangeSpark(POGlobalRearrange copy)
+            throws ExecException {
+        super(copy);
+    }
+
+    public boolean isUseSecondaryKey() {
+        return useSecondaryKey;
+    }
+
+    public void setUseSecondaryKey(boolean useSecondaryKey) {
+        this.useSecondaryKey = useSecondaryKey;
+    }
+
+    public boolean[] getSecondarySortOrder() {
+        return secondarySortOrder;
+    }
+
+    public void setSecondarySortOrder(boolean[] secondarySortOrder) {
+        this.secondarySortOrder = secondarySortOrder;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POJoinGroupSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POJoinGroupSpark.java
new file mode 100644
index 0000000..1d2dbb6
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POJoinGroupSpark.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.operator;
+
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * Collapse POLocalRearrange,POGlobalRearrange and POPackage to POJoinGroupSpark to reduce unnecessary map operations in the join/group
+ */
+public class POJoinGroupSpark extends PhysicalOperator {
+    private List<POLocalRearrange> lraOps;
+    private POGlobalRearrangeSpark glaOp;
+    private POPackage pkgOp;
+    private List<PhysicalOperator> predecessors;
+
+    public POJoinGroupSpark(List<POLocalRearrange> lraOps, POGlobalRearrangeSpark glaOp, POPackage pkgOp){
+        super(glaOp.getOperatorKey());
+        this.lraOps = lraOps;
+        this.glaOp = glaOp;
+        this.pkgOp = pkgOp;
+    }
+
+    public List<POLocalRearrange> getLROps() {
+        return lraOps;
+    }
+
+    public POGlobalRearrangeSpark getGROp() {
+        return glaOp;
+    }
+
+    public POPackage getPkgOp() {
+        return pkgOp;
+    }
+
+    @Override
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return true;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return false;
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "POJoinGroupSpark"+ "["
+                + DataType.findTypeName(resultType) + "]" + " - "
+                + mKey.toString();
+    }
+
+    @Override
+    public Tuple illustratorMarkup(Object in, Object out, int eqClassIndex) {
+        return null;
+    }
+
+    public void setPredecessors(List<PhysicalOperator> predecessors) {
+        this.predecessors = predecessors;
+    }
+
+    public List<PhysicalOperator> getPredecessors() {
+        return predecessors;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POPoissonSampleSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POPoissonSampleSpark.java
new file mode 100644
index 0000000..89e0734
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POPoissonSampleSpark.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.operator;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.Result;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPoissonSample;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.builtin.PoissonSampleLoader;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class POPoissonSampleSpark extends POPoissonSample {
+    private static final Log LOG = LogFactory.getLog(POPoissonSampleSpark.class);
+    // Only for Spark
+    private transient boolean endOfInput = false;
+
+    public boolean isEndOfInput() {
+        return endOfInput;
+    }
+
+    public void setEndOfInput(boolean isEndOfInput) {
+        endOfInput = isEndOfInput;
+    }
+
+    public POPoissonSampleSpark(OperatorKey k, int rp, int sr, float hp, long tm) {
+        super(k, rp, sr, hp, tm);
+    }
+
+    @Override
+    public Result getNextTuple() throws ExecException {
+        if (!initialized) {
+            numRowsSampled = 0;
+            avgTupleMemSz = 0;
+            rowNum = 0;
+            skipInterval = -1;
+            if (totalMemory == null) {
+                // Initialize in backend to get memory of task
+                totalMemory = Runtime.getRuntime().maxMemory();
+            }
+            long availRedMem = (long) (totalMemory * heapPerc);
+            memToSkipPerSample = availRedMem/sampleRate;
+            initialized = true;
+        }
+        if (numRowSplTupleReturned) {
+            // row num special row has been returned after all inputs
+            // were read, nothing more to read
+            return RESULT_EOP;
+        }
+
+        Result res;
+        res = processInput();
+
+        // if reaches at the end, pick a record and return
+        if (this.isEndOfInput()) {
+            // if skip enough, and the last record is OK.
+            if ( numSkipped == skipInterval
+                    && res.returnStatus == POStatus.STATUS_OK) {
+                return createNumRowTuple((Tuple) res.result);
+            } else if (newSample != null) {
+                return createNumRowTuple((Tuple) newSample.result);
+            }
+        }
+
+        // just return to read next record from input
+        if (res.returnStatus == POStatus.STATUS_NULL) {
+            return new Result(POStatus.STATUS_NULL, null);
+        } else if (res.returnStatus == POStatus.STATUS_EOP
+                    || res.returnStatus == POStatus.STATUS_ERR) {
+            return res;
+        }
+
+        // got a 'OK' record
+        rowNum++;
+
+        if (numSkipped < skipInterval) {
+            numSkipped++;
+
+            // skip this tuple, and continue to read from input
+            return new Result(POStatus.STATUS_EOP, null);
+        }
+
+        // pick this record as sampled
+        newSample = res;
+        numSkipped = 0;
+        Result pickedSample = newSample;
+        updateSkipInterval((Tuple) pickedSample.result);
+
+        if( LOG.isDebugEnabled()) {
+            LOG.debug("pickedSample:");
+            if (pickedSample.result != null) {
+                for (int i = 0; i < ((Tuple) pickedSample.result).size(); i++) {
+                    LOG.debug("the " + i + " ele:" + ((Tuple) pickedSample.result).get(i));
+                }
+            }
+        }
+        return pickedSample;
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "PoissonSampleSpark - " + mKey.toString();
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POReduceBySpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POReduceBySpark.java
new file mode 100644
index 0000000..88b1cac
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POReduceBySpark.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.operator;
+
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.plan.OperatorKey;
+
+/**
+ * ReduceBy operator that maps to Sparks ReduceBy.
+ * Extends ForEach and adds packager, secondary sort and partitioner support.
+ */
+public class POReduceBySpark extends POForEach {
+    private String customPartitioner;
+    protected POLocalRearrange lr;
+    protected POPackage pkg;
+
+    public POReduceBySpark(OperatorKey k, int rp, List<PhysicalPlan> inp, List<Boolean> isToBeFlattened, POPackage
+            pkg, POLocalRearrange lr){
+        super(k, rp, inp, isToBeFlattened);
+        this.pkg = pkg;
+        this.lr = lr;
+        this.addOriginalLocation(lr.getAlias(), lr.getOriginalLocations());
+    }
+
+    public POPackage getPKGOp() {
+        return pkg;
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "Reduce By" + "(" + getFlatStr() + ")" + "["
+                + DataType.findTypeName(resultType) + "]" + " - "
+                + mKey.toString();
+    }
+
+    protected String getFlatStr() {
+        if(isToBeFlattenedArray ==null) {
+            return "";
+        }
+        StringBuilder sb = new StringBuilder();
+        for (Boolean b : isToBeFlattenedArray) {
+            sb.append(b);
+            sb.append(',');
+        }
+        if(sb.length()>0){
+            sb.deleteCharAt(sb.length()-1);
+        }
+        return sb.toString();
+    }
+
+    // Use secondary key
+    private boolean useSecondaryKey;
+    // Sort order for secondary keys;
+    private boolean[] secondarySortOrder;
+
+    public boolean isUseSecondaryKey() {
+        return useSecondaryKey;
+    }
+
+    public void setUseSecondaryKey(boolean useSecondaryKey) {
+        this.useSecondaryKey = useSecondaryKey;
+    }
+
+    public boolean[] getSecondarySortOrder() {
+        return secondarySortOrder;
+    }
+
+    public void setSecondarySortOrder(boolean[] secondarySortOrder) {
+        this.secondarySortOrder = secondarySortOrder;
+    }
+
+    public String getCustomPartitioner() {
+        return customPartitioner;
+    }
+
+    public void setCustomPartitioner(String customPartitioner) {
+        this.customPartitioner = customPartitioner;
+    }
+
+    public POLocalRearrange getLROp() {
+        return lr;
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POSampleSortSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POSampleSortSpark.java
new file mode 100644
index 0000000..2545e6b
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/operator/POSampleSortSpark.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.operator;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
+import org.apache.pig.data.DataType;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class POSampleSortSpark extends POSort {
+
+    public POSampleSortSpark(POSort sort){
+        super(sort.getOperatorKey(), sort.getRequestedParallelism(), null, sort.getSortPlans(), sort.getMAscCols(), sort
+                .getMSortFunc());
+    }
+
+    @Override
+    public void visit(PhyPlanVisitor v) throws VisitorException {
+        v.visit(this);
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return false;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return false;
+    }
+
+    @Override
+    public String name() {
+        return getAliasString() + "POSparkSort" + "["
+                + DataType.findTypeName(resultType) + "]" + "("
+                + (super.getMSortFunc() != null ? super.getMSortFunc().getFuncSpec() : "") + ")"
+                + " - " + mKey.toString();
+    }
+
+    @Override
+    public Tuple illustratorMarkup(Object in, Object out, int eqClassIndex) {
+        return null;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/AccumulatorOptimizer.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/AccumulatorOptimizer.java
new file mode 100644
index 0000000..e12c534
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/AccumulatorOptimizer.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.util.AccumulatorOptimizerUtil;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+import java.util.List;
+
+/**
+ * A visitor to optimize plans that determines if a vertex plan can run in
+ * accumulative mode.
+ */
+public class AccumulatorOptimizer extends SparkOpPlanVisitor {
+
+    public AccumulatorOptimizer(SparkOperPlan plan) {
+        super(plan, new DepthFirstWalker<SparkOperator, SparkOperPlan>(plan));
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOperator) throws
+            VisitorException {
+        PhysicalPlan plan = sparkOperator.physicalPlan;
+        List<PhysicalOperator> pos = plan.getRoots();
+        if (pos == null || pos.size() == 0) {
+            return;
+        }
+
+        List<POGlobalRearrange> glrs = PlanHelper.getPhysicalOperators(plan,
+                POGlobalRearrange.class);
+
+        for (POGlobalRearrange glr : glrs) {
+            List<PhysicalOperator> successors = plan.getSuccessors(glr);
+            AccumulatorOptimizerUtil.addAccumulator(plan, successors);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/CombinerOptimizer.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/CombinerOptimizer.java
new file mode 100644
index 0000000..8c0e648
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/CombinerOptimizer.java
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POProject;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.CombinerPackager;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODistinct;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPreCombinerLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.Packager;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POReduceBySpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.util.CombinerOptimizerUtil;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.plan.optimizer.OptimizerException;
+import org.apache.pig.impl.util.Pair;
+
+import com.google.common.collect.Maps;
+
+/**
+ * This class goes through the physical plan are replaces GlobalRearrange with ReduceBy
+ * where there are algebraic operations.
+ */
+public class CombinerOptimizer extends SparkOpPlanVisitor {
+
+    private static Log LOG = LogFactory.getLog(CombinerOptimizer.class);
+
+    public CombinerOptimizer(SparkOperPlan plan) {
+        super(plan, new DepthFirstWalker<>(plan));
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        try {
+            addCombiner(sparkOp.physicalPlan);
+        } catch (Exception e) {
+            throw new VisitorException(e);
+        }
+    }
+
+    // Checks for algebraic operations and if they exist.
+    // Replaces global rearrange (cogroup) with reduceBy as follows:
+    // Input:
+    // foreach (using algebraicOp)
+    //   -> packager
+    //      -> globalRearrange
+    //          -> localRearrange
+    // Output:
+    // foreach (using algebraicOp.Final)
+    //   -> reduceBy (uses algebraicOp.Intermediate)
+    //         -> foreach (using algebraicOp.Initial)
+    //             -> CombinerRearrange
+    private void addCombiner(PhysicalPlan phyPlan) throws VisitorException, PlanException, CloneNotSupportedException {
+
+        List<PhysicalOperator> leaves = phyPlan.getLeaves();
+        if (leaves == null || leaves.size() != 1) {
+            return;
+        }
+
+        // Ensure there is grouping.
+        List<POGlobalRearrange> glrs = PlanHelper.getPhysicalOperators(phyPlan, POGlobalRearrange.class);
+        if (glrs == null || glrs.size() == 0) {
+            return;
+        }
+        for (POGlobalRearrange glr : glrs) {
+            List<PhysicalOperator> glrSuccessors = phyPlan.getSuccessors(glr);
+            if (glrSuccessors == null || glrSuccessors.isEmpty()) {
+                continue;
+            }
+
+            if (!(glrSuccessors.get(0) instanceof POPackage)) {
+                continue;
+            }
+            POPackage poPackage = (POPackage) glrSuccessors.get(0);
+
+            List<PhysicalOperator> poPackageSuccessors = phyPlan.getSuccessors(poPackage);
+            if (poPackageSuccessors == null || poPackageSuccessors.size() != 1) {
+                continue;
+            }
+            PhysicalOperator successor = poPackageSuccessors.get(0);
+
+            // Retaining the original successor to be used later in modifying the plan.
+            PhysicalOperator packageSuccessor = successor;
+
+            if (successor instanceof POLimit) {
+                // POLimit is acceptable, as long as it has a single foreach as
+                // successor
+                List<PhysicalOperator> limitSucs = phyPlan.getSuccessors(successor);
+                if (limitSucs != null && limitSucs.size() == 1 &&
+                        limitSucs.get(0) instanceof POForEach) {
+                    // the code below will now further examine the foreach
+                    successor = limitSucs.get(0);
+                }
+            }
+            if (successor instanceof POForEach) {
+                POForEach foreach = (POForEach) successor;
+                List<PhysicalOperator> foreachSuccessors = phyPlan.getSuccessors(foreach);
+                // multi-query
+                if (foreachSuccessors == null || foreachSuccessors.size() != 1) {
+                    continue;
+                }
+                // Clone foreach so it can be modified to a post-reduce foreach.
+                POForEach postReduceFE = foreach.clone();
+                List<PhysicalPlan> feInners = postReduceFE.getInputPlans();
+
+                // find algebraic operators and also check if the foreach statement
+                // is suitable for combiner use
+                List<Pair<PhysicalOperator, PhysicalPlan>> algebraicOps = CombinerOptimizerUtil.findAlgebraicOps
+                        (feInners);
+                if (algebraicOps == null || algebraicOps.size() == 0) {
+                    // the plan is not combinable or there is nothing to combine
+                    // we're done
+                    continue;
+                }
+                try {
+                    List<PhysicalOperator> glrPredecessors = phyPlan.getPredecessors(glr);
+                    // Exclude co-group from optimization
+                    if (glrPredecessors == null || glrPredecessors.size() != 1) {
+                        continue;
+                    }
+
+                    if (!(glrPredecessors.get(0) instanceof POLocalRearrange)) {
+                        continue;
+                    }
+
+                    POLocalRearrange rearrange = (POLocalRearrange) glrPredecessors.get(0);
+
+                    LOG.info("Algebraic operations found. Optimizing plan to use combiner.");
+
+                    // Trim the global rearrange and the preceeding package.
+                    convertToMapSideForEach(phyPlan, poPackage);
+
+                    // replace PODistinct->Project[*] with distinct udf (which is Algebraic)
+                    for (Pair<PhysicalOperator, PhysicalPlan> op2plan : algebraicOps) {
+                        if (!(op2plan.first instanceof PODistinct)) {
+                            continue;
+                        }
+                        CombinerOptimizerUtil.DistinctPatcher distinctPatcher
+                                = new CombinerOptimizerUtil.DistinctPatcher(op2plan.second);
+                        distinctPatcher.visit();
+                        if (distinctPatcher.getDistinct() == null) {
+                            int errCode = 2073;
+                            String msg = "Problem with replacing distinct operator with distinct built-in function.";
+                            throw new PlanException(msg, errCode, PigException.BUG);
+                        }
+                        op2plan.first = distinctPatcher.getDistinct();
+                    }
+
+                    // create new map foreach -
+                    POForEach mfe = CombinerOptimizerUtil.createForEachWithGrpProj(postReduceFE, poPackage.getPkgr()
+                            .getKeyType());
+                    Map<PhysicalOperator, Integer> op2newpos = Maps.newHashMap();
+                    Integer pos = 1;
+                    // create plan for each algebraic udf and add as inner plan in map-foreach
+                    for (Pair<PhysicalOperator, PhysicalPlan> op2plan : algebraicOps) {
+                        PhysicalPlan udfPlan = CombinerOptimizerUtil.createPlanWithPredecessors(op2plan.first,
+                                op2plan.second);
+                        mfe.addInputPlan(udfPlan, false);
+                        op2newpos.put(op2plan.first, pos++);
+                    }
+                    CombinerOptimizerUtil.changeFunc(mfe, POUserFunc.INITIAL);
+
+                    // since we will only be creating SingleTupleBag as input to
+                    // the map foreach, we should flag the POProjects in the map
+                    // foreach inner plans to also use SingleTupleBag
+                    for (PhysicalPlan mpl : mfe.getInputPlans()) {
+                        try {
+                            new CombinerOptimizerUtil.fixMapProjects(mpl).visit();
+                        } catch (VisitorException e) {
+                            int errCode = 2089;
+                            String msg = "Unable to flag project operator to use single tuple bag.";
+                            throw new PlanException(msg, errCode, PigException.BUG, e);
+                        }
+                    }
+
+                    // create new combine foreach
+                    POForEach cfe = CombinerOptimizerUtil.createForEachWithGrpProj(postReduceFE, poPackage.getPkgr()
+                            .getKeyType());
+                    // add algebraic functions with appropriate projection
+                    CombinerOptimizerUtil.addAlgebraicFuncToCombineFE(cfe, op2newpos);
+
+                    // we have modified the foreach inner plans - so set them again
+                    // for the foreach so that foreach can do any re-initialization
+                    // around them.
+                    mfe.setInputPlans(mfe.getInputPlans());
+                    cfe.setInputPlans(cfe.getInputPlans());
+
+                    // tell POCombinerPackage which fields need projected and which
+                    // placed in bags. First field is simple project rest need to go
+                    // into bags
+                    int numFields = algebraicOps.size() + 1; // algebraic funcs + group key
+                    boolean[] bags = new boolean[numFields];
+                    bags[0] = false;
+                    for (int i = 1; i < numFields; i++) {
+                        bags[i] = true;
+                    }
+
+                    // Use the POCombiner package in the combine plan
+                    // as it needs to act differently than the regular
+                    // package operator.
+                    CombinerPackager pkgr = new CombinerPackager(poPackage.getPkgr(), bags);
+                    POPackage combinePack = poPackage.clone();
+                    combinePack.setPkgr(pkgr);
+
+                    // A specialized local rearrange operator will replace
+                    // the normal local rearrange in the map plan.
+                    POLocalRearrange newRearrange = CombinerOptimizerUtil.getNewRearrange(rearrange);
+                    POPreCombinerLocalRearrange combinerLocalRearrange = CombinerOptimizerUtil.getPreCombinerLR
+                            (rearrange);
+                    phyPlan.replace(rearrange, combinerLocalRearrange);
+
+                    // Create a reduceBy operator.
+                    POReduceBySpark reduceOperator = new POReduceBySpark(cfe.getOperatorKey(), combinerLocalRearrange
+                            .getRequestedParallelism(), cfe.getInputPlans(), cfe.getToBeFlattened(), combinePack,
+                            newRearrange);
+                    reduceOperator.setCustomPartitioner(glr.getCustomPartitioner());
+                    fixReduceSideFE(postReduceFE, algebraicOps);
+                    CombinerOptimizerUtil.changeFunc(reduceOperator, POUserFunc.INTERMEDIATE);
+                    updatePackager(reduceOperator, newRearrange);
+
+                    // Add the new operators
+                    phyPlan.add(reduceOperator);
+                    phyPlan.add(mfe);
+                    // Connect the new operators as follows:
+                    // reduceBy (using algebraicOp.Intermediate)
+                    //      -> foreach (using algebraicOp.Initial)
+                     phyPlan.connect(mfe, reduceOperator);
+
+                    // Insert the reduce stage between combiner rearrange and its successor.
+                    phyPlan.disconnect(combinerLocalRearrange, packageSuccessor);
+                    phyPlan.connect(reduceOperator, packageSuccessor);
+                    phyPlan.connect(combinerLocalRearrange, mfe);
+
+                    // Replace foreach with post reduce foreach
+                    phyPlan.add(postReduceFE);
+                    phyPlan.replace(foreach, postReduceFE);
+                } catch (Exception e) {
+                    int errCode = 2018;
+                    String msg = "Internal error. Unable to introduce the combiner for optimization.";
+                    throw new OptimizerException(msg, errCode, PigException.BUG, e);
+                }
+            }
+        }
+    }
+
+    // Modifies the input plans of the post reduce foreach to match the output of reduce stage.
+    private void fixReduceSideFE(POForEach postReduceFE, List<Pair<PhysicalOperator, PhysicalPlan>> algebraicOps)
+            throws ExecException, PlanException {
+        int i=1;
+        for (Pair<PhysicalOperator, PhysicalPlan> algebraicOp : algebraicOps) {
+            POUserFunc combineUdf = (POUserFunc) algebraicOp.first;
+            PhysicalPlan pplan = algebraicOp.second;
+            combineUdf.setAlgebraicFunction(POUserFunc.FINAL);
+
+            POProject newProj = new POProject(
+                    CombinerOptimizerUtil.createOperatorKey(postReduceFE.getOperatorKey().getScope()),
+                    1, i
+            );
+            newProj.setResultType(DataType.BAG);
+
+            PhysicalOperator udfInput = pplan.getPredecessors(combineUdf).get(0);
+            pplan.disconnect(udfInput, combineUdf);
+            pplan.add(newProj);
+            pplan.connect(newProj, combineUdf);
+            i++;
+        }
+        postReduceFE.setResultType(DataType.TUPLE);
+    }
+
+    // Modifies the map side of foreach (before reduce).
+    private void convertToMapSideForEach(PhysicalPlan physicalPlan, POPackage poPackage)
+            throws PlanException {
+        LinkedList<PhysicalOperator> operatorsToRemove = new LinkedList<>();
+        for (PhysicalOperator physicalOperator : physicalPlan.getPredecessors(poPackage)) {
+            if (physicalOperator instanceof POGlobalRearrangeSpark) {
+                operatorsToRemove.add(physicalOperator);
+                break;
+            }
+        }
+        // Remove global rearranges preceeding POPackage
+        for (PhysicalOperator po : operatorsToRemove) {
+            physicalPlan.removeAndReconnect(po);
+        }
+        // Remove POPackage itself.
+        physicalPlan.removeAndReconnect(poPackage);
+    }
+
+    // Update the ReduceBy Operator with the packaging used by Local rearrange.
+    private void updatePackager(POReduceBySpark reduceOperator, POLocalRearrange lrearrange) throws OptimizerException {
+        Packager pkgr = reduceOperator.getPKGOp().getPkgr();
+        // annotate the package with information from the LORearrange
+        // update the keyInfo information if already present in the POPackage
+        Map<Integer, Pair<Boolean, Map<Integer, Integer>>> keyInfo = pkgr.getKeyInfo();
+        if (keyInfo == null)
+            keyInfo = new HashMap<>();
+
+        if (keyInfo.get(Integer.valueOf(lrearrange.getIndex())) != null) {
+            // something is wrong - we should not be getting key info
+            // for the same index from two different Local Rearranges
+            int errCode = 2087;
+            String msg = "Unexpected problem during optimization." +
+                    " Found index:" + lrearrange.getIndex() +
+                    " in multiple LocalRearrange operators.";
+            throw new OptimizerException(msg, errCode, PigException.BUG);
+
+        }
+        keyInfo.put(Integer.valueOf(lrearrange.getIndex()),
+                new Pair<Boolean, Map<Integer, Integer>>(
+                        lrearrange.isProjectStar(), lrearrange.getProjectedColsMap()));
+        pkgr.setKeyInfo(keyInfo);
+        pkgr.setKeyTuple(lrearrange.isKeyTuple());
+        pkgr.setKeyCompound(lrearrange.isKeyCompound());
+    }
+
+    /**
+     * Look for a algebraic POUserFunc that is the leaf of an input plan.
+     *
+     * @param pplan physical plan
+     * @return null if any operator other POProject or non-algebraic POUserFunc is
+     * found while going down the plan, otherwise algebraic POUserFunc is returned
+     */
+    private static POUserFunc getAlgebraicSuccessor(PhysicalPlan pplan) {
+        // check if it ends in an UDF
+        List<PhysicalOperator> leaves = pplan.getLeaves();
+        if (leaves == null || leaves.size() != 1) {
+            return null;
+        }
+
+        PhysicalOperator succ = leaves.get(0);
+        if (succ instanceof POUserFunc && ((POUserFunc) succ).combinable()) {
+            return (POUserFunc) succ;
+        }
+
+        // some other operator ? can't combine
+        return null;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/JoinGroupOptimizerSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/JoinGroupOptimizerSpark.java
new file mode 100644
index 0000000..aa7d6b2
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/JoinGroupOptimizerSpark.java
@@ -0,0 +1,214 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POJoinGroupSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.util.MultiMap;
+
+/**
+ * Collapse LocalRearrange,GlobalRearrange,Package to POJoinGroupSpark to reduce unnecessary
+ * map operations to optimize join/group. Detail see PIG-4797
+ */
+public class JoinGroupOptimizerSpark extends SparkOpPlanVisitor {
+    private static final Log LOG = LogFactory.getLog(JoinGroupOptimizerSpark.class);
+
+    public JoinGroupOptimizerSpark(SparkOperPlan plan) {
+        super(plan, new DependencyOrderWalker<SparkOperator, SparkOperPlan>(plan, true));
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        if (sparkOp.physicalPlan != null) {
+            GlobalRearrangeDiscover glrDiscover = new GlobalRearrangeDiscover(sparkOp.physicalPlan);
+            glrDiscover.visit();
+            List<PhysicalPlan> plans = glrDiscover.getPlansWithJoinAndGroup();
+            handlePlans(plans, sparkOp);
+        }
+
+    }
+
+    private void handlePlans(List<PhysicalPlan> plans, SparkOperator sparkOp) throws VisitorException {
+        for(int i=0;i<plans.size();i++){
+            PhysicalPlan planWithJoinAndGroup = plans.get(i);
+            POGlobalRearrangeSpark glrSpark = PlanHelper.getPhysicalOperators(planWithJoinAndGroup,POGlobalRearrangeSpark.class).get(0);
+            if (verifyJoinOrGroupCase(plans.get(i), glrSpark)) {
+                try {
+                    restructSparkOp(planWithJoinAndGroup, glrSpark, sparkOp);
+                } catch (PlanException e) {
+                    throw new VisitorException("GlobalRearrangeDiscover#visitSparkOp fails: ", e);
+                }
+            }
+        }
+    }
+
+    static class GlobalRearrangeDiscover extends PhyPlanVisitor {
+        private List<PhysicalPlan> plansWithJoinAndGroup = new ArrayList<PhysicalPlan>();
+        public GlobalRearrangeDiscover(PhysicalPlan plan) {
+            super(plan, new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(
+                    plan));
+        }
+
+        @Override
+        public void visitGlobalRearrange(POGlobalRearrange glr) throws VisitorException {
+            PhysicalPlan currentPlan = this.mCurrentWalker.getPlan();//If there are POSplit, we need traverse the POSplit.getPlans(), so use mCurrentWalker.getPlan()
+            if( currentPlan != null) {
+                plansWithJoinAndGroup.add(currentPlan);
+            }
+        }
+
+        public List<PhysicalPlan> getPlansWithJoinAndGroup() {
+            return plansWithJoinAndGroup;
+        }
+    }
+
+    //collapse LRA,GRA,PKG to POJoinGroupSpark
+    private void restructSparkOp(PhysicalPlan plan,POGlobalRearrangeSpark glaOp, SparkOperator sparkOp) throws PlanException {
+
+        List<PhysicalOperator> predes = plan.getPredecessors(glaOp);
+        if (predes != null) {
+            List<POLocalRearrange> lraOps = new ArrayList<POLocalRearrange>();
+            List<PhysicalOperator> allPredsOfLRA = new ArrayList<PhysicalOperator>();
+
+            //Get the predecessors of POJoinGroupSpark with correct order after JoinOptimizationSpark
+            //For other PhysicalOperator, we usually use OperatorPlan#getPredecessor(op) to get predecessors and sort predecessors[JobGraphBuilder#getPredecessors] to
+            //get the predecessor with correct order(in common case, PhysicalOperator
+            //with small OperatorKey must be executed before that with bigger OperatorKey),but this is not suitable for POJoinGroupSpark
+            //Give an example to explain this:
+            //original:
+            //POLOAD(scope-1)                                POLOAD(scope-2)
+            //               \                                   /
+            //   POFOREach(scope-3)                              POLocalRearrange(scope-5)
+            //                  \                                /
+            //              POLocalRearrange(scope-4)       POLocalRearrange(scope-5)
+            //                      \                           /
+            //                              POGlobalRearrange(scope-6)
+            //                                      |
+            //                              POPackage(scope-7)
+            //after JoinOptimizationSpark:
+            //POLOAD(scope-1)                                POLOAD(scope-2)
+            //               \                                   /
+            //   POFOREach(scope-3)                             /
+            //                     \                           /
+            //                        POJoinGroupSpark(scope-8)
+
+            //the predecessor of POJoinGroupSpark(scope-8) is POForEach(scope-3) and POLoad(scope-2) because they are
+            //the predecessor of POLocalRearrange(scope-4) and POLocalRearrange(scope-5) while we will get
+            //will be POLoad(scope-2) and POForEach(scope-3) if use OperatorPlan#getPredecessor(op)to gain predecessors and sort predecessors
+            Collections.sort(predes);
+            for (PhysicalOperator lra : predes) {
+                lraOps.add((POLocalRearrange) lra);
+                List<PhysicalOperator> predOfLRAList = plan.getPredecessors(lra);
+                if( predOfLRAList != null && predOfLRAList.size() ==1) {
+                    PhysicalOperator predOfLRA = predOfLRAList.get(0);
+                    plan.disconnect(predOfLRA, lra);
+                    allPredsOfLRA.add(predOfLRA);
+                }
+            }
+
+            POPackage pkgOp = (POPackage) plan.getSuccessors(glaOp).get(0);
+            PhysicalOperator pkgSuccessor = plan.getSuccessors(pkgOp).get(0);
+            POJoinGroupSpark joinSpark = new POJoinGroupSpark(lraOps, glaOp, pkgOp);
+            if(allPredsOfLRA.size()>0) {
+                joinSpark.setPredecessors(allPredsOfLRA);
+            }
+            plan.add(joinSpark);
+
+            for (PhysicalOperator predOfLRA : allPredsOfLRA) {
+                plan.connect(predOfLRA, joinSpark);
+            }
+
+            plan.disconnect(pkgOp, pkgSuccessor);
+            plan.connect(joinSpark, pkgSuccessor);
+            for (POLocalRearrange lra : lraOps) {
+                replaceMultiqueryMapping(sparkOp, lra, joinSpark);
+                plan.remove(lra);
+            }
+            plan.remove(glaOp);
+            plan.remove(pkgOp);
+        }
+    }
+
+    private void replaceMultiqueryMapping(SparkOperator sparkOperator, PhysicalOperator from, PhysicalOperator to) {
+        MultiMap<OperatorKey, OperatorKey> multiQueryOptimizeConnectionItems = sparkOperator.getMultiQueryOptimizeConnectionItem();
+        if (multiQueryOptimizeConnectionItems.containsKey(from.getOperatorKey())) {
+            List<OperatorKey> value = multiQueryOptimizeConnectionItems.get(from.getOperatorKey());
+            multiQueryOptimizeConnectionItems.removeKey(from.getOperatorKey());
+            multiQueryOptimizeConnectionItems.put(to.getOperatorKey(), value);
+        }
+    }
+
+    private boolean verifyJoinOrGroupCase(PhysicalPlan plan, POGlobalRearrangeSpark glaOp) {
+        List<PhysicalOperator> lraOps = plan.getPredecessors(glaOp);
+        List<PhysicalOperator> pkgOps = plan.getSuccessors(glaOp);
+        boolean isAllPredecessorLRA = isAllPredecessorLRA(lraOps);
+        boolean isSuccessorPKG = isSuccessorPKG(pkgOps);
+        return isAllPredecessorLRA && isSuccessorPKG;
+    }
+
+    private boolean isSuccessorPKG(List<PhysicalOperator> pkgOps) {
+        boolean result = false;
+        if (pkgOps != null && (pkgOps.size() == 1)) {
+            if (pkgOps.get(0) instanceof POPackage) {
+                result = true;
+            }
+        } else {
+            result = false;
+        }
+
+
+        return result;
+    }
+
+    private boolean isAllPredecessorLRA(List<PhysicalOperator> lraOps) {
+        boolean result = true;
+        if (lraOps != null) {
+            for (PhysicalOperator lraOp : lraOps) {
+                if (!(lraOp instanceof POLocalRearrange)) {
+                    result = false;
+                    break;
+                }
+            }
+        } else {
+            result = false;
+        }
+
+        return result;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/MultiQueryOptimizerSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/MultiQueryOptimizerSpark.java
new file mode 100644
index 0000000..1ec186c
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/MultiQueryOptimizerSpark.java
@@ -0,0 +1,225 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.plan.NodeIdGenerator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.ReverseDependencyOrderWalker;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.plan.optimizer.OptimizerException;
+
+
+/**
+ * MultiQueryOptimizer for spark
+ */
+public class MultiQueryOptimizerSpark extends SparkOpPlanVisitor {
+
+    private static final Log LOG = LogFactory.getLog(MultiQueryOptimizerSpark.class);
+
+    private String scope;
+    private NodeIdGenerator nig;
+
+    public MultiQueryOptimizerSpark(SparkOperPlan plan) {
+        super(plan, new ReverseDependencyOrderWalker<SparkOperator, SparkOperPlan>(plan));
+        nig = NodeIdGenerator.getGenerator();
+        List<SparkOperator> roots = plan.getRoots();
+        scope = roots.get(0).getOperatorKey().getScope();
+    }
+
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        try {
+            if (!sparkOp.isSplitter()) {
+                return;
+            }
+
+            List<SparkOperator> splittees = getPlan().getSuccessors(sparkOp);
+
+            if (splittees == null) {
+                return;
+            }
+
+            //If the size of predecessors of splittee is more than 1, then not do multiquery optimization
+            //@see TestMultiQueryBasic#testMultiQueryWithFJ_2
+            for (SparkOperator splittee : splittees) {
+                if (getPlan().getPredecessors(splittee).size() > 1) {
+                    return;
+                }
+            }
+
+            if (splittees.size() == 1) {
+                // We don't need a POSplit here, we can merge the splittee into spliter
+                SparkOperator spliter = sparkOp;
+                SparkOperator singleSplitee = splittees.get(0);
+                List<PhysicalOperator> roots = singleSplitee.physicalPlan.getRoots();
+                List<PhysicalOperator> rootCopys = new ArrayList<PhysicalOperator>(roots);
+                //sort the roots by OperatorKey
+                //for the first element of roots, merge the physical plan of spliter and splittee
+                //for the other elements of roots,merge the clone physical plan of spliter and splittee
+                //the clone physical plan will have same type of physical operators but have more bigger OperatorKey
+                //thus physical operator with bigger OperatorKey will be executed later than those have small OperatorKey(see JobGraphBuilder.sortPredecessorRDDs())
+                Collections.sort(rootCopys);
+                List<PhysicalPlan> spliterPhysicalPlan = getPhysicalPlans(spliter.physicalPlan, rootCopys.size());
+                int i = 0;
+                for (PhysicalOperator root : rootCopys) {
+                    if (root instanceof POLoad) {
+                        POLoad load = (POLoad) root;
+                        PhysicalPlan plClone = spliterPhysicalPlan.get(i);
+                        POStore store = (POStore) plClone.getLeaves().get(0);
+                        if (load.getLFile().getFileName().equals(store.getSFile().getFileName())) {
+                            plClone.remove(store);
+                            PhysicalOperator succOfload = singleSplitee.physicalPlan.getSuccessors(load).get(0);
+                            singleSplitee.physicalPlan.remove(load);
+                            mergePlanAWithPlanB(singleSplitee.physicalPlan, plClone, succOfload);
+                            i++;
+                        }
+                    }
+                }
+
+                addSubPlanPropertiesToParent(singleSplitee, spliter);
+                removeSpliter(getPlan(), spliter, singleSplitee);
+            } else {
+                //If the size of splittee is more than 1, we need create a split which type is POSplit, merge all the physical plans
+                // of splittees to the physical plan of split and remove the splittees.
+                List<PhysicalOperator> firstNodeLeaves = sparkOp.physicalPlan.getLeaves();
+                PhysicalOperator firstNodeLeaf = firstNodeLeaves.size() > 0 ? firstNodeLeaves.get(0) : null;
+                POStore poStore = null;
+                if (firstNodeLeaf != null && firstNodeLeaf instanceof POStore) {
+                    poStore = (POStore) firstNodeLeaf;
+                    PhysicalOperator predOfPoStore = sparkOp.physicalPlan.getPredecessors(poStore).get(0);
+                    sparkOp.physicalPlan.remove(poStore); // remove  unnecessary store
+                    POSplit poSplit = createSplit();
+                    ArrayList<SparkOperator> spliteesCopy = new ArrayList
+                            <SparkOperator>(splittees);
+                    for (SparkOperator splitee : spliteesCopy) {
+                        List<PhysicalOperator> rootsOfSplitee = new ArrayList(splitee.physicalPlan.getRoots());
+                        for (int i = 0; i < rootsOfSplitee.size(); i++) {
+                            if (rootsOfSplitee.get(i) instanceof POLoad) {
+                                POLoad poLoad = (POLoad) rootsOfSplitee.get(i);
+                                if (poLoad.getLFile().getFileName().equals(poStore.getSFile().getFileName())) {
+                                    List<PhysicalOperator> successorsOfPoLoad = splitee.physicalPlan.getSuccessors(poLoad);
+                                    List<PhysicalOperator> successorofPoLoadsCopy = new ArrayList<PhysicalOperator>(successorsOfPoLoad);
+                                    splitee.physicalPlan.remove(poLoad);  // remove  unnecessary load
+                                    for (PhysicalOperator successorOfPoLoad : successorofPoLoadsCopy) {
+                                        //we store from to relationship in SparkOperator#multiQueryOptimizeConnectionMap
+                                        sparkOp.addMultiQueryOptimizeConnectionItem(successorOfPoLoad.getOperatorKey(), predOfPoStore.getOperatorKey());
+                                        LOG.debug(String.format("add multiQueryOptimize connection item: to:%s, from:%s for %s",
+                                                successorOfPoLoad.toString(), predOfPoStore.getOperatorKey().toString(), splitee.getOperatorKey()));
+                                    }
+                                }
+                            }
+                        }
+                        poSplit.addPlan(splitee.physicalPlan);
+                        addSubPlanPropertiesToParent(sparkOp, splitee);
+                        removeSplittee(getPlan(), sparkOp, splitee);
+                    }
+                    sparkOp.physicalPlan.addAsLeaf(poSplit);
+                }
+            }
+        } catch (PlanException e) {
+            throw new VisitorException(e);
+        }
+    }
+
+    private List<PhysicalPlan> getPhysicalPlans(PhysicalPlan physicalPlan, int size) throws OptimizerException {
+        List<PhysicalPlan> ppList = new ArrayList<PhysicalPlan>();
+        try {
+            ppList.add(physicalPlan);
+            for (int i = 1; i < size; i++) {
+                ppList.add(physicalPlan.clone());
+            }
+        } catch (CloneNotSupportedException e) {
+            int errCode = 2127;
+            String msg = "Internal Error: Cloning of plan failed for optimization.";
+            throw new OptimizerException(msg, errCode, PigException.BUG, e);
+        }
+        return ppList;
+    }
+
+    //Merge every operators in planB to operator "to" of planA
+    private void mergePlanAWithPlanB(PhysicalPlan planA, PhysicalPlan planB, PhysicalOperator to) throws PlanException {
+        PhysicalOperator predOfStore = planB.getLeaves().get(0);
+        planA.merge(planB);
+        planA.connect(predOfStore, to);
+    }
+
+    private void removeSpliter(SparkOperPlan plan, SparkOperator spliter, SparkOperator splittee) throws PlanException {
+        if (plan.getPredecessors(spliter) != null) {
+            List<SparkOperator> preds = new ArrayList(plan.getPredecessors(spliter));
+            plan.disconnect(spliter, splittee);
+            for (SparkOperator pred : preds) {
+                plan.disconnect(pred, spliter);
+                plan.connect(pred, splittee);
+            }
+        }
+        plan.remove(spliter);
+    }
+
+    private void removeSplittee(SparkOperPlan plan, SparkOperator splitter,
+                                SparkOperator splittee) throws PlanException {
+        if (plan.getSuccessors(splittee) != null) {
+            List<SparkOperator> succs = new ArrayList();
+            succs.addAll(plan.getSuccessors(splittee));
+            plan.disconnect(splitter, splittee);
+            for (SparkOperator succSparkOperator : succs) {
+                plan.disconnect(splittee, succSparkOperator);
+                plan.connect(splitter, succSparkOperator);
+            }
+        }
+        getPlan().remove(splittee);
+    }
+
+    private POSplit createSplit() {
+        return new POSplit(new OperatorKey(scope, nig.getNextNodeId(scope)));
+    }
+
+    static public void addSubPlanPropertiesToParent(SparkOperator parentOper, SparkOperator subPlanOper) {
+        // Copy only map side properties. For eg: crossKeys.
+        // Do not copy reduce side specific properties. For eg: useSecondaryKey, segmentBelow, sortOrder, etc
+        if (subPlanOper.getCrossKeys() != null) {
+            for (String key : subPlanOper.getCrossKeys()) {
+                parentOper.addCrossKey(key);
+            }
+        }
+        parentOper.copyFeatures(subPlanOper, null);
+
+        if (subPlanOper.getRequestedParallelism() > parentOper.getRequestedParallelism()) {
+            parentOper.setRequestedParallelism(subPlanOper.getRequestedParallelism());
+        }
+        subPlanOper.setRequestedParallelismByReference(parentOper);
+        parentOper.UDFs.addAll(subPlanOper.UDFs);
+        parentOper.scalars.addAll(subPlanOper.scalars);
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/NoopFilterRemover.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/NoopFilterRemover.java
new file mode 100644
index 0000000..d73fdb6
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/NoopFilterRemover.java
@@ -0,0 +1,71 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import java.util.List;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.NoopFilterRemoverUtil;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * For historical reasons splits will always produce filters that pass
+ * everything through unchanged. This optimizer removes these.
+ * <p/>
+ * The condition we look for is POFilters with a constant boolean
+ * (true) expression as it's plan.
+ */
+public class NoopFilterRemover extends SparkOpPlanVisitor {
+    private Log log = LogFactory.getLog(NoopFilterRemover.class);
+
+    public NoopFilterRemover(SparkOperPlan plan) {
+        super(plan, new DependencyOrderWalker<SparkOperator, SparkOperPlan>(plan));
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        List<POFilter> filters = PlanHelper.getPhysicalOperators(sparkOp
+                .physicalPlan, POFilter.class);
+        for (POFilter filter : filters) {
+            PhysicalPlan filterPlan = filter.getPlan();
+            if (filterPlan.size() == 1) {
+                PhysicalOperator fp = filterPlan.getRoots().get(0);
+                if (fp instanceof ConstantExpression) {
+                    ConstantExpression exp = (ConstantExpression) fp;
+                    Object value = exp.getValue();
+                    if (value instanceof Boolean) {
+                        Boolean filterValue = (Boolean) value;
+                        if (filterValue) {
+                            NoopFilterRemoverUtil.removeFilter(filter, sparkOp.physicalPlan);
+                        }
+                    }
+                }
+            }
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/ParallelismSetter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/ParallelismSetter.java
new file mode 100644
index 0000000..740a0ba
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/ParallelismSetter.java
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.PigImplConstants;
+import org.apache.pig.impl.builtin.GFCross;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+public class ParallelismSetter extends SparkOpPlanVisitor {
+    private JobConf jobConf;
+
+    public ParallelismSetter(SparkOperPlan plan, JobConf jobConf) {
+        super(plan, new DependencyOrderWalker<SparkOperator, SparkOperPlan>(plan));
+        this.jobConf = jobConf;
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        if (sparkOp instanceof NativeSparkOperator) {
+            return;
+        }
+
+        if (sparkOp.getCrossKeys() != null) {
+            for (String key : sparkOp.getCrossKeys()) {
+                jobConf.set(PigImplConstants.PIG_CROSS_PARALLELISM + "." + key,
+                        // TODO: Estimate parallelism. For now we are hard-coding GFCross.DEFAULT_PARALLELISM
+                        Integer.toString(96));
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SecondaryKeyOptimizerSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SecondaryKeyOptimizerSpark.java
new file mode 100644
index 0000000..53d6770
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SecondaryKeyOptimizerSpark.java
@@ -0,0 +1,217 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you 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.
+*/
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.optimizer.SecondaryKeyOptimizer;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.util.SecondaryKeyOptimizerUtil;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * Secondary key sort optimization for spark mode
+ */
+public class SecondaryKeyOptimizerSpark extends SparkOpPlanVisitor implements SecondaryKeyOptimizer {
+    private static final Log LOG = LogFactory
+            .getLog(SecondaryKeyOptimizerSpark.class);
+
+    private int numSortRemoved = 0;
+    private int numDistinctChanged = 0;
+    private int numUseSecondaryKey = 0;
+
+    public SecondaryKeyOptimizerSpark(SparkOperPlan plan) {
+        super(plan, new DepthFirstWalker<SparkOperator, SparkOperPlan>(plan));
+    }
+
+    /**
+     * Secondary key sort optimization is enabled in group + foreach nested situation, like TestAccumlator#testAccumWithSort
+     * After calling SecondaryKeyOptimizerUtil.applySecondaryKeySort, the POSort in the POForeach will be deleted in the spark plan.
+     * Sort function can be implemented in secondary key sort even though POSort is deleted in the spark plan.
+     *
+     * @param sparkOperator
+     * @throws VisitorException
+     */
+    @Override
+    public void visitSparkOp(SparkOperator sparkOperator) throws VisitorException {
+        List<POLocalRearrange> rearranges = PlanHelper.getPhysicalOperators(sparkOperator.physicalPlan, POLocalRearrange.class);
+        if (rearranges.isEmpty()) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("No POLocalRearranges found in the spark operator" + sparkOperator.getOperatorKey() + ". Skipping secondary key optimization.");
+            }
+            return;
+        }
+
+        /**
+         * When ever POLocalRearrange is encountered in the sparkOperator.physicalPlan,
+         * the sub-physicalplan between the previousLR(or root) to currentLR is considered as mapPlan(like what
+         * we call in mapreduce) and the sub-physicalplan between the POGlobalRearrange(the successor of currentLR) and
+         * nextLR(or leaf) is considered as reducePlan(like what we call in mapreduce).  After mapPlan and reducePlan are got,
+         * use SecondaryKeyOptimizerUtil.applySecondaryKeySort(mapPlan,reducePlan) to enable secondary key optimization.
+         * SecondaryKeyOptimizerUtil.applySecondaryKeySort will remove POSort in the foreach in the reducePlan or
+         * change PODistinct to POSortedDistinct in the foreach in the reducePlan.
+         */
+        for (POLocalRearrange currentLR : rearranges) {
+            PhysicalPlan mapPlan = null;
+            PhysicalPlan reducePlan = null;
+            try {
+                mapPlan = getMapPlan(sparkOperator.physicalPlan, currentLR);
+            } catch (PlanException e) {
+                throw new VisitorException(e);
+            }
+            try {
+                reducePlan = getReducePlan(sparkOperator.physicalPlan, currentLR);
+            } catch (PlanException e) {
+                throw new VisitorException(e);
+            }
+
+            // Current code does not enable secondarykey optimization when join case is encounted
+            List<PhysicalOperator> rootsOfReducePlan = reducePlan.getRoots();
+            if (rootsOfReducePlan.get(0) instanceof POGlobalRearrangeSpark) {
+                PhysicalOperator glr = rootsOfReducePlan.get(0);
+                List<PhysicalOperator> predecessors = sparkOperator.physicalPlan.getPredecessors(glr);
+                if (predecessors != null && predecessors.size() >= 2) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("Current code does not enable secondarykey optimization when  join case is encounted");
+                    }
+                    return;
+                }
+            }
+
+            if (mapPlan.getOperator(currentLR.getOperatorKey()) == null) {
+                // The POLocalRearrange is sub-plan of a POSplit
+                mapPlan = PlanHelper.getLocalRearrangePlanFromSplit(mapPlan, currentLR.getOperatorKey());
+            }
+            SparkSecondaryKeyOptimizerUtil sparkSecondaryKeyOptUtil = new SparkSecondaryKeyOptimizerUtil();
+            SecondaryKeyOptimizerUtil.SecondaryKeyOptimizerInfo info = sparkSecondaryKeyOptUtil.applySecondaryKeySort(mapPlan, reducePlan);
+            if (info != null) {
+                numSortRemoved += info.getNumSortRemoved();
+                numDistinctChanged += info.getNumDistinctChanged();
+                numUseSecondaryKey += info.getNumUseSecondaryKey();
+            }
+        }
+    }
+
+    /**
+     * Find the MRPlan of the physicalPlan which containing currentLR
+     * Backward search all the physicalOperators which precede currentLR until the previous POLocalRearrange
+     * is found or the root of physicalPlan is found.
+     *
+     * @param physicalPlan
+     * @param currentLR
+     * @return
+     * @throws VisitorException
+     * @throws PlanException
+     */
+    private PhysicalPlan getMapPlan(PhysicalPlan physicalPlan, POLocalRearrange currentLR) throws VisitorException, PlanException {
+        PhysicalPlan mapPlan = new PhysicalPlan();
+        mapPlan.addAsRoot(currentLR);
+        List<PhysicalOperator> preList = physicalPlan.getPredecessors(currentLR);
+        while (true) {
+            if (preList == null) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("there is nothing to backward search");
+                }
+                break;
+            }
+            if (preList.size() != 1) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("the size of predecessor of currentLR should be 1 but now it is not 1,physicalPlan:" + physicalPlan);
+                }
+                break;
+            }
+            PhysicalOperator pre = preList.get(0);
+            if (pre instanceof POLocalRearrange) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Finishing to find the mapPlan between preLR and currentLR.");
+                }
+                break;
+            }
+            mapPlan.addAsRoot(pre);
+            preList = physicalPlan.getPredecessors(pre);
+
+        }
+        return mapPlan;
+    }
+
+    /**
+     * Find the ReducePlan of the physicalPlan which containing currentLR
+     * Forward search all the physicalOperators which succeed currentLR until the next POLocalRearrange
+     * is found or the leave of physicalPlan is found.
+     *
+     * @param physicalPlan
+     * @param currentLR
+     * @return
+     * @throws PlanException
+     */
+    private PhysicalPlan getReducePlan(PhysicalPlan physicalPlan, POLocalRearrange currentLR) throws PlanException {
+        PhysicalPlan reducePlan = new PhysicalPlan();
+        List<PhysicalOperator> succList = physicalPlan.getSuccessors(currentLR);
+        while (true) {
+            if (succList == null) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("there is nothing to forward search");
+                }
+                break;
+            }
+            if (succList.size() != 1) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("the size of successors of currentLR should be 1 but now it is not 1,physicalPlan:" + physicalPlan);
+                }
+                break;
+            }
+            PhysicalOperator succ = succList.get(0);
+            if (succ instanceof POLocalRearrange) {
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("Finishing to find the ReducePlan between currentLR and netxtLR.");
+                }
+                break;
+            }
+            reducePlan.addAsLeaf(succ);
+            succList = physicalPlan.getSuccessors(succ);
+        }
+        return reducePlan;
+    }
+
+    @Override
+    public int getNumSortRemoved() {
+        return numSortRemoved;
+    }
+
+    @Override
+    public int getNumDistinctChanged() {
+        return numDistinctChanged;
+    }
+
+    @Override
+    public int getNumUseSecondaryKey() {
+        return numUseSecondaryKey;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SparkSecondaryKeyOptimizerUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SparkSecondaryKeyOptimizerUtil.java
new file mode 100644
index 0000000..634710e
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/optimizer/SparkSecondaryKeyOptimizerUtil.java
@@ -0,0 +1,51 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one
+* or more contributor license agreements.  See the NOTICE file
+* distributed with this work for additional information
+* regarding copyright ownership.  The ASF licenses this file
+* to you 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.
+*/
+package org.apache.pig.backend.hadoop.executionengine.spark.optimizer;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.util.SecondaryKeyOptimizerUtil;
+
+import java.util.List;
+
+public class SparkSecondaryKeyOptimizerUtil extends SecondaryKeyOptimizerUtil{
+    private static Log log = LogFactory.getLog(SparkSecondaryKeyOptimizerUtil.class);
+
+    @Override
+    protected PhysicalOperator getCurrentNode(PhysicalOperator root, PhysicalPlan reducePlan) {
+        PhysicalOperator currentNode = null;
+
+        if (!(root instanceof POGlobalRearrange)) {
+            log.debug("Expected reduce root to be a POGlobalRearrange, skip secondary key optimizing");
+            currentNode = null;
+        } else {
+            List<PhysicalOperator> globalRearrangeSuccs = reducePlan
+                    .getSuccessors(root);
+            if (globalRearrangeSuccs.size() == 1) {
+                currentNode = globalRearrangeSuccs.get(0);
+            } else {
+                log.debug("Expected successor of a POGlobalRearrange is POPackage, skip secondary key optimizing");
+                currentNode = null;
+            }
+        }
+
+        return currentNode;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/DotSparkPrinter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/DotSparkPrinter.java
new file mode 100644
index 0000000..87baf4a
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/DotSparkPrinter.java
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.io.PrintStream;
+import java.util.LinkedList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.pig.impl.plan.DotPlanDumper;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.DotPOPrinter;
+import org.apache.pig.impl.plan.OperatorPlan;
+import org.apache.pig.impl.plan.Operator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.PlanVisitor;
+
+/**
+ * This class can print Spark plan in the DOT format. It uses
+ * clusters to illustrate nesting. If "verbose" is off, it will skip
+ * any nesting in the associated physical plans.
+ */
+public class DotSparkPrinter extends DotPlanDumper<SparkOperator, SparkOperPlan,
+        DotSparkPrinter.InnerOperator,
+        DotSparkPrinter.InnerPlan> {
+
+    private static int counter = 0;
+    private boolean isVerboseNesting = true;
+
+    public DotSparkPrinter(SparkOperPlan plan, PrintStream ps) {
+        this(plan, ps, false, new HashSet<Operator>(), new HashSet<Operator>(),
+                new HashSet<Operator>());
+    }
+
+    private DotSparkPrinter(SparkOperPlan plan, PrintStream ps, boolean isSubGraph,
+                         Set<Operator> subgraphs,
+                         Set<Operator> multiInputSubgraphs,
+                         Set<Operator> multiOutputSubgraphs) {
+        super(plan, ps, isSubGraph, subgraphs,
+                multiInputSubgraphs, multiOutputSubgraphs);
+    }
+
+    @Override
+    public void setVerbose(boolean verbose) {
+        // leave the parents verbose set to true
+        isVerboseNesting = verbose;
+    }
+
+    @Override
+    protected DotPlanDumper makeDumper(InnerPlan plan, PrintStream ps) {
+        return new InnerPrinter(plan, ps, mSubgraphs, mMultiInputSubgraphs,
+                mMultiOutputSubgraphs);
+    }
+
+    @Override
+    protected String getName(SparkOperator op) {
+        String name = op.name();
+        // Cut of the part of the name specifying scope.
+        String delimiter = " - ";
+        String[] temp;
+        temp = name.split(delimiter);
+        return temp[0];
+    }
+
+    @Override
+    protected Collection<InnerPlan> getNestedPlans(SparkOperator op) {
+        Collection<InnerPlan> plans = new LinkedList<InnerPlan>();
+        plans.add(new InnerPlan(op.physicalPlan));
+        return plans;
+    }
+
+    @Override
+    protected String[] getAttributes(SparkOperator op) {
+        String[] attributes = new String[3];
+        attributes[0] = "label=\""+getName(op)+"\"";
+        attributes[1] = "style=\"filled\"";
+        attributes[2] = "fillcolor=\"#EEEEEE\"";
+        return attributes;
+    }
+
+
+    /**
+     * Helper class to represent the relationship of inner operators
+     */
+    public static class InnerOperator extends Operator<PlanVisitor> {
+
+        private static final long serialVersionUID = 1L;
+        String name;
+        PhysicalPlan plan;
+        int code;
+
+        public InnerOperator(PhysicalPlan plan, String name) {
+            super(new OperatorKey());
+            this.name = name;
+            this.plan = plan;
+            this.code = counter++;
+        }
+
+        @Override public void visit(PlanVisitor v) {}
+        @Override public boolean supportsMultipleInputs() {return false;}
+        @Override public boolean supportsMultipleOutputs() {return false;}
+        @Override public String name() {return name;}
+        public PhysicalPlan getPlan() {return plan;}
+        @Override public int hashCode() {return code;}
+    }
+
+    /**
+     * Each spark operator will have and an inner plan of inner
+     * operators. The inner operators contain the physical plan of the
+     * execution phase.
+     */
+    public static class InnerPlan extends OperatorPlan<InnerOperator> {
+
+        private static final long serialVersionUID = 1L;
+
+        public InnerPlan(PhysicalPlan plan) {
+            InnerOperator sparkInnerOp = new InnerOperator(plan, "spark");
+            this.add(sparkInnerOp);
+        }
+    }
+
+    private class InnerPrinter extends DotPlanDumper<InnerOperator, InnerPlan,
+            PhysicalOperator, PhysicalPlan> {
+
+        public InnerPrinter(InnerPlan plan, PrintStream ps,
+                            Set<Operator> subgraphs,
+                            Set<Operator> multiInputSubgraphs,
+                            Set<Operator> multiOutputSubgraphs) {
+            super(plan, ps, true, subgraphs, multiInputSubgraphs,
+                    multiOutputSubgraphs);
+        }
+
+        @Override
+        protected String[] getAttributes(InnerOperator op) {
+            String[] attributes = new String[3];
+            attributes[0] = "label=\""+super.getName(op)+"\"";
+            attributes[1] = "style=\"filled\"";
+            attributes[2] = "fillcolor=\"white\"";
+            return attributes;
+        }
+
+        @Override
+        protected Collection<PhysicalPlan> getNestedPlans(InnerOperator op) {
+            Collection<PhysicalPlan> l = new LinkedList<PhysicalPlan>();
+            l.add(op.getPlan());
+            return l;
+        }
+
+        @Override
+        protected DotPOPrinter makeDumper(PhysicalPlan plan, PrintStream ps) {
+            DotPOPrinter printer = new DotPOPrinter(plan, ps, true,
+                    mSubgraphs,
+                    mMultiInputSubgraphs,
+                    mMultiOutputSubgraphs);
+            printer.setVerbose(isVerboseNesting);
+            return printer;
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java
new file mode 100644
index 0000000..f31d341
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompiler.java
@@ -0,0 +1,1565 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.io.IOException;
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.pig.CollectableLoadFunc;
+import org.apache.pig.FuncSpec;
+import org.apache.pig.IndexableLoadFunc;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.OrderedLoadFunc;
+import org.apache.pig.PigConfiguration;
+import org.apache.pig.PigException;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MergeJoinIndexer;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.ScalarPhyFinder;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.plans.UDFFinder;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.POStatus;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.ConstantExpression;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POProject;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.POUserFunc;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POBroadcastSpark;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCollectedGroup;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCounter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POCross;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODistinct;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFRJoinSpark;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeCogroup;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POMergeJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PONative;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PORank;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSkewedJoin;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStream;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.Packager;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkUtil;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POPoissonSampleSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POSampleSortSpark;
+import org.apache.pig.data.DataType;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.builtin.DefaultIndexableLoader;
+import org.apache.pig.impl.builtin.GetMemNumRows;
+import org.apache.pig.impl.builtin.PartitionSkewedKeys;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.io.FileSpec;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.NodeIdGenerator;
+import org.apache.pig.impl.plan.Operator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.OperatorPlan;
+import org.apache.pig.impl.plan.PlanException;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.util.MultiMap;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.Pair;
+import org.apache.pig.impl.util.Utils;
+import org.apache.pig.newplan.logical.relational.LOJoin;
+
+
+/**
+ * The compiler that compiles a given physical physicalPlan into a DAG of Spark
+ * operators
+ */
+public class SparkCompiler extends PhyPlanVisitor {
+    private static final Log LOG = LogFactory.getLog(SparkCompiler.class);
+
+    private PigContext pigContext;
+    private Properties pigProperties;
+
+    // The physicalPlan that is being compiled
+    private PhysicalPlan physicalPlan;
+
+    // The physicalPlan of Spark Operators
+    private SparkOperPlan sparkPlan;
+
+    private SparkOperator curSparkOp;
+
+    private String scope;
+
+    private SparkOperator[] compiledInputs = null;
+
+    private Map<OperatorKey, SparkOperator> splitsSeen;
+
+    private NodeIdGenerator nig;
+
+    private Map<PhysicalOperator, SparkOperator> phyToSparkOpMap;
+    private UDFFinder udfFinder;
+
+    public SparkCompiler(PhysicalPlan physicalPlan, PigContext pigContext) {
+        super(physicalPlan,
+                new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(
+                        physicalPlan));
+        this.physicalPlan = physicalPlan;
+        this.pigContext = pigContext;
+        this.pigProperties = pigContext.getProperties();
+        this.sparkPlan = new SparkOperPlan();
+        this.phyToSparkOpMap = new HashMap<PhysicalOperator, SparkOperator>();
+        this.udfFinder = new UDFFinder();
+        this.nig = NodeIdGenerator.getGenerator();
+        this.splitsSeen = new HashMap<OperatorKey, SparkOperator>();
+
+    }
+
+    public void compile() throws IOException, PlanException, VisitorException {
+        List<PhysicalOperator> roots = physicalPlan.getRoots();
+        if ((roots == null) || (roots.size() <= 0)) {
+            int errCode = 2053;
+            String msg = "Internal error. Did not find roots in the physical physicalPlan.";
+            throw new SparkCompilerException(msg, errCode, PigException.BUG);
+        }
+        scope = roots.get(0).getOperatorKey().getScope();
+        List<PhysicalOperator> leaves = physicalPlan.getLeaves();
+
+        if (!pigContext.inIllustrator) {
+            for (PhysicalOperator op : leaves) {
+                if (!(op instanceof POStore)) {
+                    int errCode = 2025;
+                    String msg = "Expected leaf of reduce physicalPlan to "
+                            + "always be POStore. Found "
+                            + op.getClass().getSimpleName();
+                    throw new SparkCompilerException(msg, errCode,
+                            PigException.BUG);
+                }
+            }
+        }
+
+        // get all stores and nativeSpark operators, sort them in order(operator
+        // id)
+        // and compile their plans
+        List<POStore> stores = PlanHelper.getPhysicalOperators(physicalPlan,
+                POStore.class);
+        List<PONative> nativeSparks = PlanHelper.getPhysicalOperators(
+                physicalPlan, PONative.class);
+        List<PhysicalOperator> ops;
+        if (!pigContext.inIllustrator) {
+            ops = new ArrayList<PhysicalOperator>(stores.size()
+                    + nativeSparks.size());
+            ops.addAll(stores);
+        } else {
+            ops = new ArrayList<PhysicalOperator>(leaves.size()
+                    + nativeSparks.size());
+            ops.addAll(leaves);
+        }
+        ops.addAll(nativeSparks);
+        Collections.sort(ops);
+
+        for (PhysicalOperator op : ops) {
+            if (LOG.isDebugEnabled())
+                LOG.debug("Starting compile of leaf-level operator " + op);
+            compile(op);
+        }
+    }
+
+    /**
+     * Compiles the physicalPlan below op into a Spark Operator and stores it in
+     * curSparkOp.
+     * 
+     * @param op
+     * @throws IOException
+     * @throws PlanException
+     * @throws VisitorException
+     */
+    private void compile(PhysicalOperator op) throws IOException,
+            PlanException, VisitorException {
+        SparkOperator[] prevCompInp = compiledInputs;
+
+        if (LOG.isDebugEnabled())
+            LOG.debug("Compiling physical operator " + op +
+                ". Current spark operator is " + curSparkOp);
+
+        List<PhysicalOperator> predecessors = physicalPlan.getPredecessors(op);
+        if (op instanceof PONative) {
+            // the predecessor (store) has already been processed
+            // don't process it again
+        } else if (predecessors != null && predecessors.size() > 0) {
+            // When processing an entire script (multiquery), we can
+            // get into a situation where a load has
+            // predecessors. This means that it depends on some store
+            // earlier in the physicalPlan. We need to take that dependency
+            // and connect the respective Spark operators, while at the
+            // same time removing the connection between the Physical
+            // operators. That way the jobs will run in the right
+            // order.
+            if (op instanceof POLoad) {
+
+                if (predecessors.size() != 1) {
+                    int errCode = 2125;
+                    String msg = "Expected at most one predecessor of load. Got "
+                            + predecessors.size();
+                    throw new PlanException(msg, errCode, PigException.BUG);
+                }
+
+                PhysicalOperator p = predecessors.get(0);
+                SparkOperator oper = null;
+                if (p instanceof POStore || p instanceof PONative) {
+                    oper = phyToSparkOpMap.get(p);
+                } else {
+                    int errCode = 2126;
+                    String msg = "Predecessor of load should be a store or spark operator. Got "
+                            + p.getClass();
+                    throw new PlanException(msg, errCode, PigException.BUG);
+                }
+
+                // Need new operator
+                curSparkOp = getSparkOp();
+                curSparkOp.add(op);
+                sparkPlan.add(curSparkOp);
+                physicalPlan.disconnect(op, p);
+                sparkPlan.connect(oper, curSparkOp);
+                phyToSparkOpMap.put(op, curSparkOp);
+                return;
+            }
+
+            Collections.sort(predecessors);
+            compiledInputs = new SparkOperator[predecessors.size()];
+            int i = -1;
+            for (PhysicalOperator pred : predecessors) {
+                if (pred instanceof POSplit
+                        && splitsSeen.containsKey(pred.getOperatorKey())) {
+                    POSplit split = (POSplit) pred;
+                    compiledInputs[++i] = startNew(
+                            split.getSplitStore(),
+                            splitsSeen.get(pred.getOperatorKey()), null);
+                    continue;
+                }
+                compile(pred);
+                compiledInputs[++i] = curSparkOp;
+            }
+        } else {
+            // No predecessors. Mostly a load. But this is where
+            // we start. We create a new sparkOp and add its first
+            // operator op. Also this should be added to the sparkPlan.
+            curSparkOp = getSparkOp();
+            curSparkOp.add(op);
+            if (op != null && op instanceof POLoad) {
+                if (((POLoad) op).getLFile() != null
+                        && ((POLoad) op).getLFile().getFuncSpec() != null)
+                    curSparkOp.UDFs.add(((POLoad) op).getLFile().getFuncSpec()
+                            .toString());
+            }
+            sparkPlan.add(curSparkOp);
+            phyToSparkOpMap.put(op, curSparkOp);
+            return;
+        }
+        op.visit(this);
+        compiledInputs = prevCompInp;
+    }
+
+    private SparkOperator getSparkOp() {
+        SparkOperator op = new SparkOperator(OperatorKey.genOpKey(scope));
+        if (LOG.isDebugEnabled())
+            LOG.debug("Created new Spark operator " + op);
+        return op;
+    }
+
+    public SparkOperPlan getSparkPlan() {
+        return sparkPlan;
+    }
+
+    public void connectSoftLink() throws PlanException, IOException {
+        for (PhysicalOperator op : physicalPlan) {
+            if (physicalPlan.getSoftLinkPredecessors(op) != null) {
+                for (PhysicalOperator pred : physicalPlan
+                        .getSoftLinkPredecessors(op)) {
+                    SparkOperator from = phyToSparkOpMap.get(pred);
+                    SparkOperator to = phyToSparkOpMap.get(op);
+                    if (from == to)
+                        continue;
+                    if (sparkPlan.getPredecessors(to) == null
+                            || !sparkPlan.getPredecessors(to).contains(from)) {
+                        sparkPlan.connect(from, to);
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * @param fSpec
+     * @param old
+     * @param operatorKey: If operatorKey is not null, we assign the operatorKey to POLoad in the new SparkOperator
+     *                     ,otherwise the operatorKey of POLoad will be created by the program. Detail see PIG-5212
+     * @return
+     * @throws PlanException
+     */
+    private SparkOperator startNew(FileSpec fSpec, SparkOperator old, OperatorKey operatorKey) throws PlanException {
+        POLoad ld = getLoad(operatorKey);
+        ld.setLFile(fSpec);
+        SparkOperator ret = getSparkOp();
+        ret.add(ld);
+        sparkPlan.add(ret);
+        sparkPlan.connect(old, ret);
+        return ret;
+    }
+
+    private POLoad getLoad(OperatorKey operatorKey) {
+        POLoad ld = null;
+        if (operatorKey != null) {
+            ld = new POLoad(operatorKey);
+        } else {
+            ld = new POLoad(new OperatorKey(scope, nig.getNextNodeId(scope)));
+        }
+        ld.setPc(pigContext);
+        ld.setIsTmpLoad(true);
+        return ld;
+    }
+        
+    @Override
+    public void visitSplit(POSplit op) throws VisitorException {
+        try {
+            List<PhysicalOperator> preds = this.physicalPlan.getPredecessors(op);
+            OperatorKey predOperatorKey = null;
+            if (preds != null && preds.size() > 0) {
+                predOperatorKey = preds.get(0).getOperatorKey();
+            }
+            FileSpec fSpec = op.getSplitStore();
+            SparkOperator sparkOp = endSingleInputPlanWithStr(fSpec);
+            sparkOp.setSplitter(true);
+            splitsSeen.put(op.getOperatorKey(), sparkOp);
+            curSparkOp = startNew(fSpec, sparkOp, predOperatorKey);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    public void visitDistinct(PODistinct op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    private SparkOperator endSingleInputPlanWithStr(FileSpec fSpec)
+            throws PlanException {
+        if (compiledInputs.length > 1) {
+            int errCode = 2023;
+            String msg = "Received a multi input physicalPlan when expecting only a single input one.";
+            throw new PlanException(msg, errCode, PigException.BUG);
+        }
+        SparkOperator sparkOp = compiledInputs[0]; // Load
+        POStore str = getStore();
+        str.setSFile(fSpec);
+        sparkOp.physicalPlan.addAsLeaf(str);
+        return sparkOp;
+    }
+
+    private POStore getStore() {
+        POStore st = new POStore(new OperatorKey(scope,
+                nig.getNextNodeId(scope)));
+        // mark store as tmp store. These could be removed by the
+        // optimizer, because it wasn't the user requesting it.
+        st.setIsTmpStore(true);
+        return st;
+    }
+
+    @Override
+    public void visitLoad(POLoad op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitNative(PONative op) throws VisitorException {
+        try {
+            SparkOperator nativesparkOpper = getNativeSparkOp(
+                    op.getNativeMRjar(), op.getParams());
+            nativesparkOpper.markNative();
+            sparkPlan.add(nativesparkOpper);
+            sparkPlan.connect(curSparkOp, nativesparkOpper);
+            phyToSparkOpMap.put(op, nativesparkOpper);
+            curSparkOp = nativesparkOpper;
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    private NativeSparkOperator getNativeSparkOp(String sparkJar,
+            String[] parameters) {
+        return new NativeSparkOperator(new OperatorKey(scope,
+                nig.getNextNodeId(scope)), sparkJar, parameters);
+    }
+
+    @Override
+    public void visitStore(POStore op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+            if (op.getSFile() != null && op.getSFile().getFuncSpec() != null)
+                curSparkOp.UDFs.add(op.getSFile().getFuncSpec().toString());
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitFilter(POFilter op) throws VisitorException {
+        try {
+            addToPlan(op);
+            processUDFs(op.getPlan());
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitCross(POCross op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitStream(POStream op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitSort(POSort op) throws VisitorException {
+        try {
+            addToPlan(op);
+            POSort sort = op;
+            long limit = sort.getLimit();
+            if (limit!=-1) {
+                POLimit pLimit2 = new POLimit(new OperatorKey(scope,nig.getNextNodeId(scope)));
+                pLimit2.setLimit(limit);
+                curSparkOp.physicalPlan.addAsLeaf(pLimit2);
+                curSparkOp.markLimitAfterSort();
+            }
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitLimit(POLimit op) throws VisitorException {
+        try {
+            addToPlan(op);
+            curSparkOp.markLimit();
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitLocalRearrange(POLocalRearrange op)
+            throws VisitorException {
+        try {
+            addToPlan(op);
+            List<PhysicalPlan> plans = op.getPlans();
+            if (plans != null)
+                for (PhysicalPlan ep : plans)
+                    processUDFs(ep);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitCollectedGroup(POCollectedGroup op)
+            throws VisitorException {
+        List<PhysicalOperator> roots = curSparkOp.physicalPlan.getRoots();
+        if (roots.size() != 1) {
+            int errCode = 2171;
+            String errMsg = "Expected one but found more then one root physical operator in physical physicalPlan.";
+            throw new SparkCompilerException(errMsg, errCode, PigException.BUG);
+        }
+
+        PhysicalOperator phyOp = roots.get(0);
+        if (!(phyOp instanceof POLoad)) {
+            int errCode = 2172;
+            String errMsg = "Expected physical operator at root to be POLoad. Found : "
+                    + phyOp.getClass().getCanonicalName();
+            throw new SparkCompilerException(errMsg, errCode, PigException.BUG);
+        }
+
+        LoadFunc loadFunc = ((POLoad) phyOp).getLoadFunc();
+        try {
+            if (!(CollectableLoadFunc.class.isAssignableFrom(loadFunc
+                    .getClass()))) {
+                int errCode = 2249;
+                throw new SparkCompilerException(
+                        "While using 'collected' on group; data must be loaded via loader implementing CollectableLoadFunc.",
+                        errCode);
+            }
+            ((CollectableLoadFunc) loadFunc).ensureAllKeyInstancesInSameSplit();
+        } catch (SparkCompilerException e) {
+            throw (e);
+        } catch (IOException e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitPOForEach(POForEach op) throws VisitorException {
+        try {
+            addToPlan(op);
+            List<PhysicalPlan> plans = op.getInputPlans();
+            if (plans != null) {
+                for (PhysicalPlan ep : plans) {
+                    processUDFs(ep);
+                }
+            }
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitCounter(POCounter op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitRank(PORank op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitGlobalRearrange(POGlobalRearrange op)
+            throws VisitorException {
+        try {
+            POGlobalRearrangeSpark glbOp = new POGlobalRearrangeSpark(op);
+            addToPlan(glbOp);
+            if (op.isCross()) {
+                curSparkOp.addCrossKey(op.getOperatorKey().toString());
+            }
+
+            curSparkOp.customPartitioner = op.getCustomPartitioner();
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitPackage(POPackage op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+            if (op.getPkgr().getPackageType() == Packager.PackageType.JOIN) {
+                curSparkOp.markRegularJoin();
+            } else if (op.getPkgr().getPackageType() == Packager.PackageType.GROUP) {
+                if (op.getNumInps() == 1) {
+                    curSparkOp.markGroupBy();
+                } else if (op.getNumInps() > 1) {
+                    curSparkOp.markCogroup();
+                }
+            }
+
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitUnion(POUnion op) throws VisitorException {
+        try {
+            addToPlan(op);
+            phyToSparkOpMap.put(op, curSparkOp);
+            curSparkOp.markUnion();
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    /**
+     * currently use regular join to replace skewedJoin
+     * Skewed join currently works with two-table inner join.
+     * More info about pig SkewedJoin, See https://wiki.apache.org/pig/PigSkewedJoinSpec
+     *
+     * @param op
+     * @throws VisitorException
+     */
+    @Override
+    public void visitSkewedJoin(POSkewedJoin op) throws VisitorException {
+        try {
+            Random r = new Random();
+            String pigKeyDistFile = "pig.keyDistFile" + r.nextInt();
+            // firstly, build sample job
+            SparkOperator sampleSparkOp = getSkewedJoinSampleJob(op);
+
+            buildBroadcastForSkewedJoin(sampleSparkOp, pigKeyDistFile);
+
+            sampleSparkOp.markSampler();
+            sparkPlan.add(sampleSparkOp);
+
+            // secondly, build the join job.
+            addToPlan(op);
+            curSparkOp.setSkewedJoinPartitionFile(pigKeyDistFile);
+
+            // do sampling job before join job
+            sparkPlan.connect(sampleSparkOp, curSparkOp);
+
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator " +
+                    op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitFRJoin(POFRJoin op) throws VisitorException {
+        try {
+            curSparkOp = phyToSparkOpMap.get(op.getInputs().get(op.getFragment()));
+            for (int i = 0; i < compiledInputs.length; i++) {
+                SparkOperator sparkOperator = compiledInputs[i];
+                if (curSparkOp.equals(sparkOperator)) {
+                    continue;
+                }
+
+                OperatorKey broadcastKey = new OperatorKey(scope, nig.getNextNodeId(scope));
+                POBroadcastSpark poBroadcastSpark = new POBroadcastSpark(broadcastKey);
+                poBroadcastSpark.setBroadcastedVariableName(broadcastKey.toString());
+
+                sparkOperator.physicalPlan.addAsLeaf(poBroadcastSpark);
+            }
+
+            POFRJoinSpark poFRJoinSpark = new POFRJoinSpark(op);
+            addToPlan(poFRJoinSpark);
+            phyToSparkOpMap.put(op, curSparkOp);
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator " + op.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    @Override
+    public void visitMergeJoin(POMergeJoin joinOp) throws VisitorException {
+        try {
+            if (compiledInputs.length != 2 || joinOp.getInputs().size() != 2){
+                int errCode=1101;
+                throw new SparkCompilerException("Merge Join must have exactly two inputs. Found : "+compiledInputs.length, errCode);
+            }
+
+            curSparkOp = phyToSparkOpMap.get(joinOp.getInputs().get(0));
+            SparkOperator rightSparkOp;
+            if(curSparkOp.equals(compiledInputs[0])) {
+                rightSparkOp = compiledInputs[1];
+            } else {
+                rightSparkOp = compiledInputs[0];
+            }
+
+            PhysicalPlan rightPipelinePlan;
+            PhysicalPlan rightPhyPlan = rightSparkOp.physicalPlan;
+            if (rightPhyPlan.getRoots().size() != 1) {
+                int errCode = 2171;
+                String errMsg = "Expected one but found more then one root physical operator in physical plan.";
+                throw new SparkCompilerException(errMsg,errCode);
+            }
+            PhysicalOperator rightPhyLoader = rightPhyPlan.getRoots().get(0);
+            if (!(rightPhyLoader instanceof POLoad)) {
+                int errCode = 2172;
+                String errMsg = "Expected physical operator at root to be POLoad. Found : "+rightPhyLoader.getClass().getCanonicalName();
+                throw new SparkCompilerException(errMsg,errCode);
+            }
+            if (rightPhyPlan.getSuccessors(rightPhyLoader) == null || rightPhyPlan.getSuccessors(rightPhyLoader).isEmpty()) {
+                // Load - Join case.
+                rightPipelinePlan = null;
+            } else{ // We got something on right side. Yank it and set it as inner plan of right input.
+                rightPipelinePlan = rightPhyPlan.clone();
+                PhysicalOperator root = rightPipelinePlan.getRoots().get(0);
+                rightPipelinePlan.disconnect(root, rightPipelinePlan.getSuccessors(root).get(0));
+                rightPipelinePlan.remove(root);
+                rightPhyPlan.trimBelow(rightPhyLoader);
+            }
+
+            joinOp.setupRightPipeline(rightPipelinePlan);
+            rightSparkOp.setRequestedParallelism(1); // for indexing job
+
+            POLoad rightLoader = (POLoad)rightSparkOp.physicalPlan.getRoots().get(0);
+            joinOp.setSignature(rightLoader.getSignature());
+            LoadFunc rightLoadFunc = rightLoader.getLoadFunc();
+
+            if(IndexableLoadFunc.class.isAssignableFrom(rightLoadFunc.getClass())) {
+                joinOp.setRightLoaderFuncSpec(rightLoader.getLFile().getFuncSpec());
+                joinOp.setRightInputFileName(rightLoader.getLFile().getFileName());
+                curSparkOp.UDFs.add(rightLoader.getLFile().getFuncSpec().toString());
+
+                // we don't need the right rightSparkOp since
+                // the right loader is an IndexableLoadFunc which can handle the index itself
+                sparkPlan.remove(rightSparkOp);
+                if(rightSparkOp == compiledInputs[0]) {
+                    compiledInputs[0] = null;
+                } else if(rightSparkOp == compiledInputs[1]) {
+                    compiledInputs[1] = null;
+                }
+
+                // validate that the join keys in merge join are only
+                // simple column projections or '*' and not expression - expressions
+                // cannot be handled when the index is built by the storage layer on the sorted
+                // data when the sorted data (and corresponding index) is written.
+                // So merge join will be restricted not have expressions as join keys
+                int numInputs = mPlan.getPredecessors(joinOp).size(); // should be 2
+                for(int i = 0; i < numInputs; i++) {
+                    List<PhysicalPlan> keyPlans = joinOp.getInnerPlansOf(i);
+                    for (PhysicalPlan keyPlan : keyPlans) {
+                        for(PhysicalOperator op : keyPlan) {
+                            if(!(op instanceof POProject)) {
+                                int errCode = 1106;
+                                String errMsg = "Merge join is possible only for simple column or '*' join keys when using " +
+                                        rightLoader.getLFile().getFuncSpec() + " as the loader";
+                                throw new SparkCompilerException(errMsg, errCode, PigException.INPUT);
+                            }
+                        }
+                    }
+                }
+
+            } else {
+                //Replacing POLoad with indexer is disabled for 'merge-sparse' joins.  While
+                //this feature would be useful, the current implementation of DefaultIndexableLoader
+                //is not designed to handle multiple calls to seekNear.  Specifically, it rereads the entire index
+                //for each call.  Some refactoring of this class is required - and then the check below could be removed.
+                if (joinOp.getJoinType() == LOJoin.JOINTYPE.MERGESPARSE) {
+                    int errCode = 1104;
+                    String errMsg = "Right input of merge-join must implement IndexableLoadFunc. " +
+                            "The specified loader " + rightLoadFunc + " doesn't implement it";
+                    throw new SparkCompilerException(errMsg,errCode);
+                }
+
+                // Replace POLoad with  indexer.
+                if (! (OrderedLoadFunc.class.isAssignableFrom(rightLoadFunc.getClass()))){
+                    int errCode = 1104;
+                    String errMsg = "Right input of merge-join must implement " +
+                            "OrderedLoadFunc interface. The specified loader "
+                            + rightLoadFunc + " doesn't implement it";
+                    throw new SparkCompilerException(errMsg,errCode);
+                }
+
+                String[] indexerArgs = new String[6];
+                List<PhysicalPlan> rightInpPlans = joinOp.getInnerPlansOf(1);
+                FileSpec origRightLoaderFileSpec = rightLoader.getLFile();
+
+                indexerArgs[0] = origRightLoaderFileSpec.getFuncSpec().toString();
+                indexerArgs[1] = ObjectSerializer.serialize((Serializable)rightInpPlans);
+                indexerArgs[2] = ObjectSerializer.serialize(rightPipelinePlan);
+                indexerArgs[3] = rightLoader.getSignature();
+                indexerArgs[4] = rightLoader.getOperatorKey().scope;
+                indexerArgs[5] = Boolean.toString(true);
+
+                FileSpec lFile = new FileSpec(rightLoader.getLFile().getFileName(),new FuncSpec(MergeJoinIndexer.class.getName(), indexerArgs));
+                rightLoader.setLFile(lFile);
+
+                // (keyFirst1, keyFirst2, .. , position, splitIndex) See MergeJoinIndexer
+                rightSparkOp.useTypedComparator(true);
+                POStore idxStore = getStore();
+                FileSpec idxStrFile = getTempFileSpec();
+                idxStore.setSFile(idxStrFile);
+                rightSparkOp.physicalPlan.addAsLeaf(idxStore);
+                rightSparkOp.markIndexer();
+
+                curSparkOp.UDFs.add(origRightLoaderFileSpec.getFuncSpec().toString());
+
+                // We want to ensure indexing job runs prior to actual join job.
+                // So, connect them in order.
+                sparkPlan.connect(rightSparkOp, curSparkOp);
+
+                // set up the DefaultIndexableLoader for the join operator
+                String[] defaultIndexableLoaderArgs = new String[5];
+                defaultIndexableLoaderArgs[0] = origRightLoaderFileSpec.getFuncSpec().toString();
+                defaultIndexableLoaderArgs[1] = idxStrFile.getFileName();
+                defaultIndexableLoaderArgs[2] = idxStrFile.getFuncSpec().toString();
+                defaultIndexableLoaderArgs[3] = joinOp.getOperatorKey().scope;
+                defaultIndexableLoaderArgs[4] = origRightLoaderFileSpec.getFileName();
+                joinOp.setRightLoaderFuncSpec((new FuncSpec(DefaultIndexableLoader.class.getName(), defaultIndexableLoaderArgs)));
+                joinOp.setRightInputFileName(origRightLoaderFileSpec.getFileName());
+
+                joinOp.setIndexFile(idxStrFile.getFileName());
+            }
+
+            curSparkOp.physicalPlan.addAsLeaf(joinOp);
+            phyToSparkOpMap.put(joinOp, curSparkOp);
+
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator "
+                    + joinOp.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+
+    private void processUDFs(PhysicalPlan plan) throws VisitorException {
+        if (plan != null) {
+            // Process Scalars (UDF with referencedOperators)
+            ScalarPhyFinder scalarPhyFinder = new ScalarPhyFinder(plan);
+            scalarPhyFinder.visit();
+            curSparkOp.scalars.addAll(scalarPhyFinder.getScalars());
+
+            // Process UDFs
+            udfFinder.setPlan(plan);
+            udfFinder.visit();
+            curSparkOp.UDFs.addAll(udfFinder.getUDFs());
+        }
+    }
+
+    private void addToPlan(PhysicalOperator op) throws PlanException,
+            IOException {
+        SparkOperator sparkOp = null;
+        if (compiledInputs.length == 1) {
+            sparkOp = compiledInputs[0];
+        } else {
+            sparkOp = merge(compiledInputs);
+        }
+        sparkOp.physicalPlan.addAsLeaf(op);
+        curSparkOp = sparkOp;
+    }
+
+    private SparkOperator merge(SparkOperator[] compiledInputs)
+            throws PlanException {
+        SparkOperator ret = getSparkOp();
+        sparkPlan.add(ret);
+
+        Set<SparkOperator> toBeConnected = new HashSet<SparkOperator>();
+        List<SparkOperator> toBeRemoved = new ArrayList<SparkOperator>();
+
+        List<PhysicalPlan> toBeMerged = new ArrayList<PhysicalPlan>();
+
+        for (SparkOperator sparkOp : compiledInputs) {
+            if (LOG.isDebugEnabled())
+                LOG.debug("Merging Spark operator" + sparkOp);
+            toBeRemoved.add(sparkOp);
+            toBeMerged.add(sparkOp.physicalPlan);
+            List<SparkOperator> predecessors = sparkPlan
+                    .getPredecessors(sparkOp);
+            if (predecessors != null) {
+                for (SparkOperator predecessorSparkOp : predecessors) {
+                    toBeConnected.add(predecessorSparkOp);
+                }
+            }
+        }
+        merge(ret.physicalPlan, toBeMerged);
+
+        Iterator<SparkOperator> it = toBeConnected.iterator();
+        while (it.hasNext())
+            sparkPlan.connect(it.next(), ret);
+        for (SparkOperator removeSparkOp : toBeRemoved) {
+            if (removeSparkOp.requestedParallelism > ret.requestedParallelism)
+                ret.requestedParallelism = removeSparkOp.requestedParallelism;
+            for (String udf : removeSparkOp.UDFs) {
+                if (!ret.UDFs.contains(udf))
+                    ret.UDFs.add(udf);
+            }
+            // We also need to change scalar marking
+            for (PhysicalOperator physOp : removeSparkOp.scalars) {
+                if (!ret.scalars.contains(physOp)) {
+                    ret.scalars.add(physOp);
+                }
+            }
+            
+            if(removeSparkOp.getCrossKeys()!=null){
+                for(String crossKey: removeSparkOp.getCrossKeys())
+                   ret.addCrossKey(crossKey);
+            }
+            
+            
+            Set<PhysicalOperator> opsToChange = new HashSet<PhysicalOperator>();
+            for (Map.Entry<PhysicalOperator, SparkOperator> entry : phyToSparkOpMap
+                    .entrySet()) {
+                if (entry.getValue() == removeSparkOp) {
+                    opsToChange.add(entry.getKey());
+                }
+            }
+            for (PhysicalOperator op : opsToChange) {
+                phyToSparkOpMap.put(op, ret);
+            }
+
+            sparkPlan.remove(removeSparkOp);
+        }
+        return ret;
+    }
+
+    /**
+     * The merge of a list of plans into a single physicalPlan
+     * 
+     * @param <O>
+     * @param <E>
+     * @param finPlan
+     *            - Final Plan into which the list of plans is merged
+     * @param plans
+     *            - list of plans to be merged
+     * @throws PlanException
+     */
+    private <O extends Operator<?>, E extends OperatorPlan<O>> void merge(
+            E finPlan, List<E> plans) throws PlanException {
+        for (E e : plans) {
+            finPlan.merge(e);
+        }
+        Collections.sort(finPlan.getLeaves());
+    }
+
+    @Override
+    public void visitMergeCoGroup(POMergeCogroup poCoGrp) throws VisitorException {
+        if (compiledInputs.length < 2) {
+            int errCode = 2251;
+            String errMsg = "Merge Cogroup work on two or more relations." +
+                    "To use map-side group-by on single relation, use 'collected' qualifier.";
+            throw new SparkCompilerException(errMsg, errCode);
+        }
+
+        List<FuncSpec> funcSpecs = new ArrayList<FuncSpec>(compiledInputs.length - 1);
+        List<String> fileSpecs = new ArrayList<String>(compiledInputs.length - 1);
+        List<String> loaderSigns = new ArrayList<String>(compiledInputs.length - 1);
+
+        try {
+            poCoGrp.setEndOfRecordMark(POStatus.STATUS_NULL);
+
+            // Iterate through all the SparkOpererators, disconnect side SparkOperators from
+            // SparkOperator and collect all the information needed in different lists.
+
+            for (int i = 0; i < compiledInputs.length; i++) {
+                SparkOperator sparkOper = compiledInputs[i];
+                PhysicalPlan plan = sparkOper.physicalPlan;
+                if (plan.getRoots().size() != 1) {
+                    int errCode = 2171;
+                    String errMsg = "Expected one but found more then one root physical operator in physical plan.";
+                    throw new SparkCompilerException(errMsg, errCode, PigException.BUG);
+                }
+
+                PhysicalOperator rootPOOp = plan.getRoots().get(0);
+                if (!(rootPOOp instanceof POLoad)) {
+                    int errCode = 2172;
+                    String errMsg = "Expected physical operator at root to be POLoad. Found : " + rootPOOp.getClass().getCanonicalName();
+                    throw new SparkCompilerException(errMsg, errCode);
+                }
+
+                POLoad sideLoader = (POLoad) rootPOOp;
+                FileSpec loadFileSpec = sideLoader.getLFile();
+                FuncSpec funcSpec = loadFileSpec.getFuncSpec();
+                LoadFunc loadfunc = sideLoader.getLoadFunc();
+                if (i == 0) {
+
+                    if (!(CollectableLoadFunc.class.isAssignableFrom(loadfunc.getClass()))) {
+                        int errCode = 2252;
+                        throw new SparkCompilerException("Base loader in Cogroup must implement CollectableLoadFunc.", errCode);
+                    }
+
+                    ((CollectableLoadFunc) loadfunc).ensureAllKeyInstancesInSameSplit();
+                    continue;
+                }
+                if (!(IndexableLoadFunc.class.isAssignableFrom(loadfunc.getClass()))) {
+                    int errCode = 2253;
+                    throw new SparkCompilerException("Side loaders in cogroup must implement IndexableLoadFunc.", errCode);
+                }
+
+                funcSpecs.add(funcSpec);
+                fileSpecs.add(loadFileSpec.getFileName());
+                loaderSigns.add(sideLoader.getSignature());
+                sparkPlan.remove(sparkOper);
+            }
+
+            poCoGrp.setSideLoadFuncs(funcSpecs);
+            poCoGrp.setSideFileSpecs(fileSpecs);
+            poCoGrp.setLoaderSignatures(loaderSigns);
+
+            // Use spark operator of base relation for the cogroup operation.
+            SparkOperator baseSparkOp = phyToSparkOpMap.get(poCoGrp.getInputs().get(0));
+
+            // Create a spark operator to generate index file for tuples from leftmost relation
+            SparkOperator indexerSparkOp = getSparkOp();
+            FileSpec idxFileSpec = getIndexingJob(indexerSparkOp, baseSparkOp, poCoGrp.getLRInnerPlansOf(0));
+            poCoGrp.setIdxFuncSpec(idxFileSpec.getFuncSpec());
+            poCoGrp.setIndexFileName(idxFileSpec.getFileName());
+
+            baseSparkOp.physicalPlan.addAsLeaf(poCoGrp);
+            for (FuncSpec funcSpec : funcSpecs)
+                baseSparkOp.UDFs.add(funcSpec.toString());
+
+            sparkPlan.add(indexerSparkOp);
+            sparkPlan.connect(indexerSparkOp, baseSparkOp);
+            phyToSparkOpMap.put(poCoGrp, baseSparkOp);
+            curSparkOp = baseSparkOp;
+        } catch (ExecException e) {
+            throw new SparkCompilerException(e.getDetailedMessage(), e.getErrorCode(), e.getErrorSource(), e);
+        } catch (SparkCompilerException mrce) {
+            throw (mrce);
+        } catch (CloneNotSupportedException e) {
+            throw new SparkCompilerException(e);
+        } catch (PlanException e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator " + poCoGrp.getClass().getCanonicalName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        } catch (IOException e) {
+            int errCode = 3000;
+            String errMsg = "IOException caught while compiling POMergeCoGroup";
+            throw new SparkCompilerException(errMsg, errCode, e);
+        }
+    }
+
+    // Sets up the indexing job for single-stage cogroups.
+    private FileSpec getIndexingJob(SparkOperator indexerSparkOp,
+                                    final SparkOperator baseSparkOp, final List<PhysicalPlan> mapperLRInnerPlans)
+            throws SparkCompilerException, PlanException, ExecException, IOException, CloneNotSupportedException {
+
+        // First replace loader with  MergeJoinIndexer.
+        PhysicalPlan baseMapPlan = baseSparkOp.physicalPlan;
+        POLoad baseLoader = (POLoad) baseMapPlan.getRoots().get(0);
+        FileSpec origLoaderFileSpec = baseLoader.getLFile();
+        FuncSpec funcSpec = origLoaderFileSpec.getFuncSpec();
+        LoadFunc loadFunc = baseLoader.getLoadFunc();
+
+        if (!(OrderedLoadFunc.class.isAssignableFrom(loadFunc.getClass()))) {
+            int errCode = 1104;
+            String errMsg = "Base relation of merge-coGroup must implement " +
+                    "OrderedLoadFunc interface. The specified loader "
+                    + funcSpec + " doesn't implement it";
+            throw new SparkCompilerException(errMsg, errCode);
+        }
+
+        String[] indexerArgs = new String[6];
+        indexerArgs[0] = funcSpec.toString();
+        indexerArgs[1] = ObjectSerializer.serialize((Serializable) mapperLRInnerPlans);
+        indexerArgs[3] = baseLoader.getSignature();
+        indexerArgs[4] = baseLoader.getOperatorKey().scope;
+        indexerArgs[5] = Boolean.toString(false); // we care for nulls.
+
+        PhysicalPlan phyPlan;
+        if (baseMapPlan.getSuccessors(baseLoader) == null
+                || baseMapPlan.getSuccessors(baseLoader).isEmpty()) {
+            // Load-Load-Cogroup case.
+            phyPlan = null;
+        } else { // We got something. Yank it and set it as inner plan.
+            phyPlan = baseMapPlan.clone();
+            PhysicalOperator root = phyPlan.getRoots().get(0);
+            phyPlan.disconnect(root, phyPlan.getSuccessors(root).get(0));
+            phyPlan.remove(root);
+
+        }
+        indexerArgs[2] = ObjectSerializer.serialize(phyPlan);
+
+        POLoad idxJobLoader = getLoad(null);
+        idxJobLoader.setLFile(new FileSpec(origLoaderFileSpec.getFileName(),
+                new FuncSpec(MergeJoinIndexer.class.getName(), indexerArgs)));
+        indexerSparkOp.physicalPlan.add(idxJobLoader);
+        indexerSparkOp.UDFs.add(baseLoader.getLFile().getFuncSpec().toString());
+
+        // Loader of sparkOp will return a tuple of form -
+        // (key1, key2, .. , WritableComparable, splitIndex). See MergeJoinIndexer for details.
+        // Create a spark node to retrieve index file by MergeJoinIndexer
+        SparkUtil.createIndexerSparkNode(indexerSparkOp, scope, nig);
+
+        POStore st = getStore();
+        FileSpec strFile = getTempFileSpec();
+        st.setSFile(strFile);
+        indexerSparkOp.physicalPlan.addAsLeaf(st);
+
+        return strFile;
+    }
+
+    /**
+     * Returns a temporary DFS Path
+     *
+     * @return
+     * @throws IOException
+     */
+    private FileSpec getTempFileSpec() throws IOException {
+        return new FileSpec(FileLocalizer.getTemporaryPath(pigContext).toString(),
+                new FuncSpec(Utils.getTmpFileCompressorName(pigContext)));
+    }
+
+    private static class FindKeyTypeVisitor extends PhyPlanVisitor {
+
+        byte keyType = DataType.UNKNOWN;
+
+        FindKeyTypeVisitor(PhysicalPlan plan) {
+            super(plan,
+                    new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(plan));
+        }
+
+        @Override
+        public void visitProject(POProject p) throws VisitorException {
+            keyType = p.getResultType();
+        }
+    }
+
+
+    /**
+     * build a POPoissonSampleSpark operator for SkewedJoin's sampling job
+     */
+    private void addSampleOperatorForSkewedJoin(SparkOperator sampleSparkOp)
+            throws PlanException {
+        Configuration conf = ConfigurationUtil.toConfiguration(pigProperties);
+        int sampleRate = conf.getInt(
+                PigConfiguration.PIG_POISSON_SAMPLER_SAMPLE_RATE,
+                POPoissonSampleSpark.DEFAULT_SAMPLE_RATE);
+        float heapPerc = conf.getFloat(
+                PigConfiguration.PIG_SKEWEDJOIN_REDUCE_MEMUSAGE,
+                PartitionSkewedKeys.DEFAULT_PERCENT_MEMUSAGE);
+        long totalMemory = conf.getLong(
+                PigConfiguration.PIG_SKEWEDJOIN_REDUCE_MEM, -1);
+
+        POPoissonSampleSpark poSample = new POPoissonSampleSpark(
+                new OperatorKey(scope, nig.getNextNodeId(scope)), -1,
+                sampleRate, heapPerc, totalMemory);
+
+        sampleSparkOp.physicalPlan.addAsLeaf(poSample);
+    }
+
+    private SparkOperator getSortJob(
+            POSort sort,
+            SparkOperator quantJob,
+            FileSpec lFile,
+            FileSpec quantFile,
+            int rp, Pair<POProject, Byte>[] fields) throws PlanException {
+        SparkOperator sparkOper = startNew(lFile, quantJob, null);
+        List<PhysicalPlan> eps1 = new ArrayList<PhysicalPlan>();
+        byte keyType = DataType.UNKNOWN;
+        if (fields == null) {
+            // This is project *
+            PhysicalPlan ep = new PhysicalPlan();
+            POProject prj = new POProject(new OperatorKey(scope, nig.getNextNodeId(scope)));
+            prj.setStar(true);
+            prj.setOverloaded(false);
+            prj.setResultType(DataType.TUPLE);
+            ep.add(prj);
+            eps1.add(ep);
+        } else {
+            /*
+            for (int i : fields) {
+                PhysicalPlan ep = new PhysicalPlan();
+                POProject prj = new POProject(new OperatorKey(scope,
+                    nig.getNextNodeId(scope)));
+                prj.setColumn(i);
+                prj.setOverloaded(false);
+                prj.setResultType(DataType.BYTEARRAY);
+                ep.add(prj);
+                eps1.add(ep);
+            }
+            */
+            // Attach the sort plans to the local rearrange to get the
+            // projection.
+            eps1.addAll(sort.getSortPlans());
+
+            // Visit the first sort plan to figure out our key type.  We only
+            // have to visit the first because if we have more than one plan,
+            // then the key type will be tuple.
+            try {
+                FindKeyTypeVisitor fktv =
+                        new FindKeyTypeVisitor(sort.getSortPlans().get(0));
+                fktv.visit();
+                keyType = fktv.keyType;
+            } catch (VisitorException ve) {
+                int errCode = 2035;
+                String msg = "Internal error. Could not compute key type of sort operator.";
+                throw new PlanException(msg, errCode, PigException.BUG, ve);
+            }
+        }
+
+        POLocalRearrange lr = new POLocalRearrange(new OperatorKey(scope, nig.getNextNodeId(scope)));
+        try {
+            lr.setIndex(0);
+        } catch (ExecException e) {
+            int errCode = 2058;
+            String msg = "Unable to set index on newly created POLocalRearrange.";
+            throw new PlanException(msg, errCode, PigException.BUG, e);
+        }
+        lr.setKeyType((fields == null || fields.length > 1) ? DataType.TUPLE :
+                keyType);
+        lr.setPlans(eps1);
+        lr.setResultType(DataType.TUPLE);
+        lr.addOriginalLocation(sort.getAlias(), sort.getOriginalLocations());
+        sparkOper.physicalPlan.addAsLeaf(lr);
+
+        sparkOper.setGlobalSort(true);
+        pigContext.getProperties().setProperty("pig.reduce.keytype", Byte.toString(lr.getKeyType()));
+        sparkOper.requestedParallelism = rp;
+        sparkOper.physicalPlan.addAsLeaf(sort);
+
+        long limit = sort.getLimit();
+        if (limit != -1) {
+            POLimit pLimit2 = new POLimit(new OperatorKey(scope, nig.getNextNodeId(scope)));
+            pLimit2.setLimit(limit);
+            sparkOper.physicalPlan.addAsLeaf(pLimit2);
+            sparkOper.markLimitAfterSort();
+        }
+
+        return sparkOper;
+    }
+
+    /**
+     * Create a sampling job to collect statistics by sampling an input file. The sequence of operations is as
+     * following:
+     * <li>Transform input sample tuples into another tuple.</li>
+     * <li>Add an extra field &quot;all&quot; into the tuple </li>
+     * <li>Package all tuples into one bag </li>
+     * <li>Add constant field for number of reducers. </li>
+     * <li>Sorting the bag </li>
+     * <li>Invoke UDF with the number of reducers and the sorted bag.</li>
+     * <li>Data generated by UDF is stored into a file.</li>
+     *
+     * @param sort           the POSort operator used to sort the bag
+     * @param sampleOperator current sampling job
+     * @param rp             configured parallemism
+     * @param udfClassName   the class name of UDF
+     * @param udfArgs        the arguments of UDF
+     * @return pair<SparkOper,integer>
+     * @throws PlanException
+     * @throws VisitorException
+     */
+    @SuppressWarnings("deprecation")
+    private SparkOperator getSamplingJob(POSort sort, SparkOperator sampleOperator, List<PhysicalPlan>
+            transformPlans,
+                                         int rp,
+                                         String udfClassName, String[] udfArgs) throws PlanException,
+            VisitorException, ExecException {
+        addSampleOperatorForSkewedJoin(sampleOperator);
+        List<Boolean> flat1 = new ArrayList<Boolean>();
+        List<PhysicalPlan> eps1 = new ArrayList<PhysicalPlan>();
+
+        // if transform plans are not specified, project the columns of sorting keys
+        if (transformPlans == null) {
+            Pair<POProject, Byte>[] sortProjs = null;
+            sortProjs = getSortCols(sort.getSortPlans());
+            // Set up the projections of the key columns
+            if (sortProjs == null) {
+                PhysicalPlan ep = new PhysicalPlan();
+                POProject prj = new POProject(new OperatorKey(scope,
+                        nig.getNextNodeId(scope)));
+                prj.setStar(true);
+                prj.setOverloaded(false);
+                prj.setResultType(DataType.TUPLE);
+                ep.add(prj);
+                eps1.add(ep);
+                flat1.add(false);
+            } else {
+                for (Pair<POProject, Byte> sortProj : sortProjs) {
+                    // Check for proj being null, null is used by getSortCols for a non POProject
+                    // operator. Since Order by does not allow expression operators,
+                    //it should never be set to null
+                    if (sortProj == null) {
+                        int errCode = 2174;
+                        String msg = "Internal exception. Could not create a sampler job";
+                        throw new SparkCompilerException(msg, errCode, PigException.BUG);
+                    }
+                    PhysicalPlan ep = new PhysicalPlan();
+                    POProject prj;
+                    try {
+                        prj = sortProj.first.clone();
+                    } catch (CloneNotSupportedException e) {
+                        //should not get here
+                        throw new AssertionError(
+                                "Error cloning project caught exception" + e
+                        );
+                    }
+                    ep.add(prj);
+                    eps1.add(ep);
+                    flat1.add(false);
+                }
+            }
+        } else {
+            for (int i = 0; i < transformPlans.size(); i++) {
+                eps1.add(transformPlans.get(i));
+                flat1.add(i == transformPlans.size() - 1 ? true : false);
+            }
+        }
+        // This foreach will pick the sort key columns from the RandomSampleLoader output
+        POForEach nfe1 = new POForEach(new OperatorKey(scope, nig.getNextNodeId(scope)), -1, eps1, flat1);
+        sampleOperator.physicalPlan.addAsLeaf(nfe1);
+
+        //sort the sample
+        POSampleSortSpark poSparkSampleSort = new POSampleSortSpark(sort);
+        sampleOperator.physicalPlan.addAsLeaf(poSparkSampleSort);
+
+        // for the foreach
+        PhysicalPlan fe2Plan = new PhysicalPlan();
+        POProject topPrj = new POProject(new OperatorKey(scope, nig.getNextNodeId(scope)));
+        topPrj.setColumn(1);
+        topPrj.setResultType(DataType.BAG);
+        topPrj.setOverloaded(true);
+        fe2Plan.add(topPrj);
+
+
+        // The plan which will have a constant representing the
+        // degree of parallelism for the final order by map-reduce job
+        // this will either come from a "order by parallel x" in the script
+        // or will be the default number of reducers for the cluster if
+        // "parallel x" is not used in the script
+        PhysicalPlan rpep = new PhysicalPlan();
+        ConstantExpression rpce = new ConstantExpression(new OperatorKey(scope, nig.getNextNodeId(scope)));
+        rpce.setRequestedParallelism(rp);
+
+        // We temporarily set it to rp and will adjust it at runtime, because the final degree of parallelism
+        // is unknown until we are ready to submit it. See PIG-2779.
+        rpce.setValue(rp);
+
+        rpce.setResultType(DataType.INTEGER);
+        rpep.add(rpce);
+
+        List<PhysicalPlan> genEps = new ArrayList<PhysicalPlan>();
+        genEps.add(rpep);
+        genEps.add(fe2Plan);
+
+        List<Boolean> flattened2 = new ArrayList<Boolean>();
+        flattened2.add(false);
+        flattened2.add(false);
+
+        POForEach nfe2 = new POForEach(new OperatorKey(scope, nig.getNextNodeId(scope)), -1, genEps, flattened2);
+        sampleOperator.physicalPlan.addAsLeaf(nfe2);
+
+        // Let's connect the output from the foreach containing
+        // number of quantiles and the sorted bag of samples to
+        // another foreach with the FindQuantiles udf. The input
+        // to the FindQuantiles udf is a project(*) which takes the
+        // foreach input and gives it to the udf
+        PhysicalPlan ep4 = new PhysicalPlan();
+        POProject prjStar4 = new POProject(new OperatorKey(scope, nig.getNextNodeId(scope)));
+        prjStar4.setResultType(DataType.TUPLE);
+        prjStar4.setStar(true);
+        ep4.add(prjStar4);
+
+        List<PhysicalOperator> ufInps = new ArrayList<PhysicalOperator>();
+        ufInps.add(prjStar4);
+
+        POUserFunc uf = new POUserFunc(new OperatorKey(scope, nig.getNextNodeId(scope)), -1, ufInps,
+                new FuncSpec(udfClassName, udfArgs));
+        ep4.add(uf);
+        ep4.connect(prjStar4, uf);
+
+        List<PhysicalPlan> ep4s = new ArrayList<PhysicalPlan>();
+        ep4s.add(ep4);
+        List<Boolean> flattened3 = new ArrayList<Boolean>();
+        flattened3.add(false);
+        POForEach nfe3 = new POForEach(new OperatorKey(scope, nig.getNextNodeId(scope)), -1, ep4s, flattened3);
+
+        sampleOperator.physicalPlan.addAsLeaf(nfe3);
+
+        sampleOperator.requestedParallelism = 1;
+        sampleOperator.markSampler();
+        return sampleOperator;
+    }
+
+    private Pair<POProject, Byte>[] getSortCols(List<PhysicalPlan> plans) throws PlanException, ExecException {
+        if (plans != null) {
+            @SuppressWarnings("unchecked")
+            Pair<POProject, Byte>[] ret = new Pair[plans.size()];
+            int i = -1;
+            for (PhysicalPlan plan : plans) {
+                PhysicalOperator op = plan.getLeaves().get(0);
+                POProject proj;
+                if (op instanceof POProject) {
+                    if (((POProject) op).isStar()) return null;
+                    proj = (POProject) op;
+                } else {
+                    proj = null;
+                }
+                byte type = op.getResultType();
+                ret[++i] = new Pair<POProject, Byte>(proj, type);
+            }
+            return ret;
+        }
+        int errCode = 2026;
+        String msg = "No expression plan found in POSort.";
+        throw new PlanException(msg, errCode, PigException.BUG);
+    }
+
+    /**
+     * Add POBroadcastSpark operator to broadcast key distribution for SkewedJoin's sampling job
+     * @param sampleSparkOp
+     * @throws PlanException
+     */
+    private void buildBroadcastForSkewedJoin(SparkOperator sampleSparkOp, String pigKeyDistFile) throws PlanException {
+
+        POBroadcastSpark poBroadcast = new POBroadcastSpark(new OperatorKey(scope, nig.getNextNodeId(scope)));
+        poBroadcast.setBroadcastedVariableName(pigKeyDistFile);
+        sampleSparkOp.physicalPlan.addAsLeaf(poBroadcast);
+    }
+
+    /**
+     * Create Sampling job for skewed join.
+     */
+    private SparkOperator getSkewedJoinSampleJob(POSkewedJoin skewedJoin) throws PlanException, VisitorException {
+        try {
+            SparkOperator sampleOperator = new SparkOperator(new OperatorKey(scope, nig.getNextNodeId(scope)));
+            sampleOperator.physicalPlan = compiledInputs[0].physicalPlan.clone();
+            MultiMap<PhysicalOperator, PhysicalPlan> joinPlans = skewedJoin.getJoinPlans();
+
+            List<PhysicalOperator> l = physicalPlan.getPredecessors(skewedJoin);
+            List<PhysicalPlan> groups = joinPlans.get(l.get(0));
+            List<Boolean> ascCol = new ArrayList<Boolean>();
+            for (int i = 0; i < groups.size(); i++) {
+                ascCol.add(false);
+            }
+
+            POSort sort = new POSort(skewedJoin.getOperatorKey(), skewedJoin.getRequestedParallelism(), null, groups,
+                    ascCol, null);
+
+            // set up transform plan to get keys and memory size of input tuples
+            // it first adds all the plans to get key columns,
+            List<PhysicalPlan> transformPlans = new ArrayList<PhysicalPlan>();
+            transformPlans.addAll(groups);
+
+            // then it adds a column for memory size
+            POProject prjStar = new POProject(new OperatorKey(scope, nig.getNextNodeId(scope)));
+            prjStar.setResultType(DataType.TUPLE);
+            prjStar.setStar(true);
+
+            List<PhysicalOperator> ufInps = new ArrayList<PhysicalOperator>();
+            ufInps.add(prjStar);
+
+            PhysicalPlan ep = new PhysicalPlan();
+            POUserFunc uf = new POUserFunc(new OperatorKey(scope, nig.getNextNodeId(scope)), -1, ufInps,
+                    new FuncSpec(GetMemNumRows.class.getName(), (String[]) null));
+            uf.setResultType(DataType.TUPLE);
+            ep.add(uf);
+            ep.add(prjStar);
+            ep.connect(prjStar, uf);
+
+            transformPlans.add(ep);
+            // pass configurations to the User Function
+            String per = pigContext.getProperties().getProperty("pig.skewedjoin.reduce.memusage",
+                    String.valueOf(PartitionSkewedKeys.DEFAULT_PERCENT_MEMUSAGE));
+            String mc = pigContext.getProperties().getProperty("pig.skewedjoin.reduce.maxtuple", "0");
+
+            return getSamplingJob(sort, sampleOperator, transformPlans, skewedJoin.getRequestedParallelism(),
+                    PartitionSkewedKeys.class.getName(), new String[]{per, mc});
+        } catch (Exception e) {
+            int errCode = 2034;
+            String msg = "Error compiling operator " +
+                    skewedJoin.getClass().getSimpleName();
+            throw new SparkCompilerException(msg, errCode, PigException.BUG, e);
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompilerException.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompilerException.java
new file mode 100644
index 0000000..bea6e7f
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkCompilerException.java
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * Create a new SparkCompilerException with null as the error message.
+ */
+public class SparkCompilerException extends VisitorException  {
+    private static final long serialVersionUID = 2L;
+
+    /**
+     * Create a new SparkCompilerException with null as the error message.
+     */
+    public SparkCompilerException() {
+        super();
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     */
+    public SparkCompilerException(String message) {
+        super(message);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified cause.
+     *
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public SparkCompilerException(Throwable cause) {
+        super(cause);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public SparkCompilerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     */
+    public SparkCompilerException(String message, int errCode) {
+        super(message, errCode);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public SparkCompilerException(String message, int errCode, Throwable cause) {
+        super(message, errCode, cause);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     */
+    public SparkCompilerException(String message, int errCode, byte errSrc) {
+        super(message, errCode, errSrc);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown. 
+     */
+    public SparkCompilerException(String message, int errCode, byte errSrc,
+                               Throwable cause) {
+        super(message, errCode, errSrc, cause);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param retry - If the exception is retriable or not
+     */
+    public SparkCompilerException(String message, int errCode, boolean retry) {
+        super(message, errCode, retry);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     */
+    public SparkCompilerException(String message, int errCode, byte errSrc,
+                               boolean retry) {
+        super(message, errCode, errSrc, retry);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     */
+    public SparkCompilerException(String message, int errCode, byte errSrc,
+                               boolean retry, String detailedMsg) {
+        super(message, errCode, errSrc, retry, detailedMsg);
+    }
+
+    /**
+     * Create a new SparkCompilerException with the specified message, error code, error source, retriable or not, detalied message for the developer and cause.
+     *
+     * @param message - The error message (which is saved for later retrieval by the <link>Throwable.getMessage()</link> method) shown to the user 
+     * @param errCode - The error code shown to the user 
+     * @param errSrc - The error source 
+     * @param retry - If the exception is retriable or not
+     * @param detailedMsg - The detailed message shown to the developer 
+     * @param cause - The cause (which is saved for later retrieval by the <link>Throwable.getCause()</link> method) indicating the source of this exception. A null value is permitted, and indicates that the cause is nonexistent or unknown.
+     */
+    public SparkCompilerException(String message, int errCode, byte errSrc,
+                               boolean retry, String detailedMsg, Throwable cause) {
+        super(message, errCode, errSrc, retry, detailedMsg, cause);
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOpPlanVisitor.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOpPlanVisitor.java
new file mode 100644
index 0000000..5fda0d7
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOpPlanVisitor.java
@@ -0,0 +1,39 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import org.apache.pig.impl.plan.PlanVisitor;
+import org.apache.pig.impl.plan.PlanWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * A visitor for the SparkOperPlan class
+ */
+public class SparkOpPlanVisitor extends
+        PlanVisitor<SparkOperator, SparkOperPlan> {
+
+    public SparkOpPlanVisitor(SparkOperPlan plan,
+            PlanWalker<SparkOperator, SparkOperPlan> walker) {
+        super(plan, walker);
+    }
+
+    public void visitSparkOp(SparkOperator sparkOperator)
+            throws VisitorException {
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOperPlan.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOperPlan.java
new file mode 100644
index 0000000..794ed54
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOperPlan.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.io.ByteArrayOutputStream;
+import java.io.PrintStream;
+
+import org.apache.pig.impl.plan.OperatorPlan;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * A Plan used to create the physicalPlan of Spark Operators
+ */
+public class SparkOperPlan extends OperatorPlan<SparkOperator> {
+
+    @Override
+    public String toString() {
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        PrintStream ps = new PrintStream(baos);
+        SparkPrinter printer = new SparkPrinter(ps, this);
+        printer.setVerbose(true);
+        try {
+            printer.visit();
+        } catch (VisitorException e) {
+            // TODO Auto-generated catch block
+            throw new RuntimeException(
+                    "Unable to get String representation of plan:" + e, e);
+        }
+        return baos.toString();
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOperator.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOperator.java
new file mode 100644
index 0000000..abdd7ed
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkOperator.java
@@ -0,0 +1,302 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.impl.plan.Operator;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.util.MultiMap;
+
+/**
+ * An operator model for a Spark job. Acts as a host to the plans that will
+ * execute in spark.
+ */
+public class SparkOperator extends Operator<SparkOpPlanVisitor> {
+    private static enum OPER_FEATURE {
+        NONE,
+        // Indicate if this job is a sampling job
+        SAMPLER,
+        // Indicate if this job is a merge indexer
+        INDEXER,
+        // Indicate if this job is a group by job
+        GROUPBY,
+        // Indicate if this job is a cogroup job
+        COGROUP,
+        // Indicate if this job is a regular join job
+        HASHJOIN,
+        // Indicate if this job is a union job
+        UNION,
+        // Indicate if this job is a native job
+        NATIVE,
+        // Indicate if this job is a limit job
+        LIMIT,
+        // Indicate if this job is a limit job after sort
+        LIMIT_AFTER_SORT;
+    };
+
+    public PhysicalPlan physicalPlan;
+
+    public Set<String> UDFs;
+
+    /* Name of the Custom Partitioner used */
+    public String customPartitioner = null;
+
+    public Set<PhysicalOperator> scalars;
+
+    public int requestedParallelism = -1;
+
+    private BitSet feature = new BitSet();
+
+    private boolean splitter = false;
+
+    // Name of the partition file generated by sampling process,
+    // Used by Skewed Join
+    private String skewedJoinPartitionFile;
+
+    private boolean usingTypedComparator = false;
+
+    private boolean combineSmallSplits = true;
+
+    private List<String> crossKeys = null;
+
+    private MultiMap<OperatorKey, OperatorKey> multiQueryOptimizeConnectionMap = new MultiMap<OperatorKey, OperatorKey>();
+
+    // Indicates if a UDF comparator is used
+    boolean isUDFComparatorUsed = false;
+
+    //The quantiles file name if globalSort is true
+    private String quantFile;
+
+    //Indicates if this job is an order by job
+    private boolean globalSort = false;
+
+    public SparkOperator(OperatorKey k) {
+        super(k);
+        physicalPlan = new PhysicalPlan();
+        UDFs = new HashSet<String>();
+        scalars = new HashSet<PhysicalOperator>();
+    }
+
+    @Override
+    public boolean supportsMultipleInputs() {
+        return true;
+    }
+
+    @Override
+    public boolean supportsMultipleOutputs() {
+        return true;
+    }
+
+    @Override
+    public String name() {
+        String udfStr = getUDFsAsStr();
+        StringBuilder sb = new StringBuilder("Spark" + "("
+                + requestedParallelism + (udfStr.equals("") ? "" : ",")
+                + udfStr + ")" + " - " + mKey.toString());
+        return sb.toString();
+    }
+
+    private String getUDFsAsStr() {
+        StringBuilder sb = new StringBuilder();
+        if (UDFs != null && UDFs.size() > 0) {
+            for (String str : UDFs) {
+                sb.append(str.substring(str.lastIndexOf('.') + 1));
+                sb.append(',');
+            }
+            sb.deleteCharAt(sb.length() - 1);
+        }
+        return sb.toString();
+    }
+
+    public void add(PhysicalOperator physicalOper) {
+        this.physicalPlan.add(physicalOper);
+    }
+
+    @Override
+    public void visit(SparkOpPlanVisitor v) throws VisitorException {
+        v.visitSparkOp(this);
+    }
+
+    public void addCrossKey(String key) {
+        if (crossKeys == null) {
+            crossKeys = new ArrayList<String>();
+        }
+        crossKeys.add(key);
+    }
+
+    public List<String> getCrossKeys() {
+        return crossKeys;
+    }
+
+    public boolean isGroupBy() {
+        return feature.get(OPER_FEATURE.GROUPBY.ordinal());
+    }
+
+    public void markGroupBy() {
+        feature.set(OPER_FEATURE.GROUPBY.ordinal());
+    }
+
+    public boolean isCogroup() {
+        return feature.get(OPER_FEATURE.COGROUP.ordinal());
+    }
+
+    public void markCogroup() {
+        feature.set(OPER_FEATURE.COGROUP.ordinal());
+    }
+
+    public boolean isRegularJoin() {
+        return feature.get(OPER_FEATURE.HASHJOIN.ordinal());
+    }
+
+    public void markRegularJoin() {
+        feature.set(OPER_FEATURE.HASHJOIN.ordinal());
+    }
+
+    public int getRequestedParallelism() {
+        return requestedParallelism;
+    }
+
+    public void setSplitter(boolean spl) {
+        splitter = spl;
+    }
+
+    public boolean isSplitter() {
+        return splitter;
+    }
+
+    public boolean isSampler() {
+        return feature.get(OPER_FEATURE.SAMPLER.ordinal());
+    }
+
+    public void markSampler() {
+        feature.set(OPER_FEATURE.SAMPLER.ordinal());
+    }
+
+    public void setSkewedJoinPartitionFile(String file) {
+        skewedJoinPartitionFile = file;
+    }
+
+    public String getSkewedJoinPartitionFile() {
+        return skewedJoinPartitionFile;
+    }
+
+    protected boolean usingTypedComparator() {
+        return usingTypedComparator;
+    }
+
+    protected void useTypedComparator(boolean useTypedComparator) {
+        this.usingTypedComparator = useTypedComparator;
+    }
+
+    protected void noCombineSmallSplits() {
+        combineSmallSplits = false;
+    }
+
+    public boolean combineSmallSplits() {
+        return combineSmallSplits;
+    }
+
+    public boolean isIndexer() {
+        return feature.get(OPER_FEATURE.INDEXER.ordinal());
+    }
+
+    public void markIndexer() {
+        feature.set(OPER_FEATURE.INDEXER.ordinal());
+    }
+    public boolean isUnion() {
+        return feature.get(OPER_FEATURE.UNION.ordinal());
+    }
+
+    public void markUnion() {
+        feature.set(OPER_FEATURE.UNION.ordinal());
+    }
+
+    public boolean isNative() {
+        return feature.get(OPER_FEATURE.NATIVE.ordinal());
+    }
+
+    public void markNative() {
+        feature.set(OPER_FEATURE.NATIVE.ordinal());
+    }
+
+    public boolean isLimit() {
+        return feature.get(OPER_FEATURE.LIMIT.ordinal());
+    }
+
+    public void markLimit() {
+        feature.set(OPER_FEATURE.LIMIT.ordinal());
+    }
+
+    public boolean isLimitAfterSort() {
+        return feature.get(OPER_FEATURE.LIMIT_AFTER_SORT.ordinal());
+    }
+
+    public void markLimitAfterSort() {
+        feature.set(OPER_FEATURE.LIMIT_AFTER_SORT.ordinal());
+    }
+
+    public void copyFeatures(SparkOperator copyFrom, List<OPER_FEATURE> excludeFeatures) {
+        for (OPER_FEATURE opf : OPER_FEATURE.values()) {
+            if (excludeFeatures != null && excludeFeatures.contains(opf)) {
+                continue;
+            }
+            if (copyFrom.feature.get(opf.ordinal())) {
+                feature.set(opf.ordinal());
+            }
+        }
+    }
+
+    public boolean isSkewedJoin() {
+        return (skewedJoinPartitionFile != null);
+    }
+
+    public void setRequestedParallelism(int requestedParallelism) {
+        this.requestedParallelism = requestedParallelism;
+    }
+
+    public void setRequestedParallelismByReference(SparkOperator oper) {
+        this.requestedParallelism = oper.requestedParallelism;
+    }
+
+    //If enable multiquery optimizer, in some cases, the predecessor(from) of a physicalOp(to) will be the leaf physicalOperator of
+    //previous sparkOperator.More detail see PIG-4675
+    public void addMultiQueryOptimizeConnectionItem(OperatorKey to, OperatorKey from) {
+        multiQueryOptimizeConnectionMap.put(to, from);
+    }
+
+    public MultiMap<OperatorKey, OperatorKey> getMultiQueryOptimizeConnectionItem() {
+        return multiQueryOptimizeConnectionMap;
+    }
+
+    public void setGlobalSort(boolean globalSort) {
+        this.globalSort = globalSort;
+    }
+
+    public boolean isGlobalSort() {
+        return globalSort;
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkPOPackageAnnotator.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkPOPackageAnnotator.java
new file mode 100644
index 0000000..9cfd6aa
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkPOPackageAnnotator.java
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhyPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.LitePackager;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.impl.plan.optimizer.OptimizerException;
+import org.apache.pig.impl.util.Pair;
+
+/**
+ * This visitor visits the SparkPlan and does the following for each
+ * SparkOperator - visits the POPackage in the plan and finds the corresponding
+ * POLocalRearrange(s). It then annotates the POPackage with information about
+ * which columns in the "value" are present in the "key" and will need to
+ * stitched in to the "value"
+ */
+public class SparkPOPackageAnnotator extends SparkOpPlanVisitor {
+    private static final Log LOG = LogFactory.getLog(SparkPOPackageAnnotator.class);
+
+    public SparkPOPackageAnnotator(SparkOperPlan plan) {
+        super(plan, new DepthFirstWalker<SparkOperator, SparkOperPlan>(plan));
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        if (!sparkOp.physicalPlan.isEmpty()) {
+            PackageDiscoverer pkgDiscoverer = new PackageDiscoverer(
+                    sparkOp.physicalPlan);
+            pkgDiscoverer.visit();
+        }
+    }
+
+    static class PackageDiscoverer extends PhyPlanVisitor {
+        private POPackage pkg;
+        private PhysicalPlan plan;
+
+        public PackageDiscoverer(PhysicalPlan plan) {
+            super(plan, new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(
+                    plan));
+            this.plan = plan;
+        }
+
+        @Override
+        public void visitPackage(POPackage pkg) throws VisitorException {
+            this.pkg = pkg;
+
+            // Find POLocalRearrange(s) corresponding to this POPackage
+            PhysicalOperator graOp = plan.getPredecessors(pkg).get(0);
+            if (! (graOp instanceof POGlobalRearrange)) {
+                  throw new OptimizerException("Package operator is not preceded by " +
+                        "GlobalRearrange operator in Spark Plan", 2087, PigException.BUG);
+            }
+
+            List<PhysicalOperator> lraOps = plan.getPredecessors(graOp);
+            if (pkg.getNumInps() != lraOps.size()) {
+          throw new OptimizerException("Unexpected problem during optimization. " +
+                            "Could not find all LocalRearrange operators. Expected " + pkg.getNumInps() +
+                        ". Got " + lraOps.size() + ".", 2086, PigException.BUG);
+            }
+            Collections.sort(lraOps);
+            for (PhysicalOperator op : lraOps) {
+                if (! (op instanceof POLocalRearrange)) {
+                    throw new OptimizerException("GlobalRearrange operator can only be preceded by " +
+                            "LocalRearrange operator(s) in Spark Plan", 2087, PigException.BUG);
+                }
+                annotatePkgWithLRA((POLocalRearrange)op);
+            }
+        };
+
+        private void annotatePkgWithLRA(POLocalRearrange lrearrange)
+                throws VisitorException {
+
+            Map<Integer, Pair<Boolean, Map<Integer, Integer>>> keyInfo;
+            if (LOG.isDebugEnabled())
+                 LOG.debug("Annotating package " + pkg + " with localrearrange operator "
+               + lrearrange + " with index " + lrearrange.getIndex());
+
+            if (pkg.getPkgr() instanceof LitePackager) {
+                if (lrearrange.getIndex() != 0) {
+                    throw new VisitorException(
+                            "POLocalRearrange for POPackageLite cannot have index other than 0, but has index - "
+                                    + lrearrange.getIndex());
+                }
+            }
+
+            // annotate the package with information from the LORearrange
+            // update the keyInfo information if already present in the
+            // POPackage
+            keyInfo = pkg.getPkgr().getKeyInfo();
+            if (keyInfo == null)
+                keyInfo = new HashMap<Integer, Pair<Boolean, Map<Integer, Integer>>>();
+
+            if (keyInfo.get(Integer.valueOf(lrearrange.getIndex())) != null) {
+                // something is wrong - we should not be getting key info
+                // for the same index from two different Local Rearranges
+                int errCode = 2087;
+                String msg = "Unexpected problem during optimization."
+                        + " Found index:" + lrearrange.getIndex()
+                        + " in multiple LocalRearrange operators.";
+                throw new OptimizerException(msg, errCode, PigException.BUG);
+
+            }
+            keyInfo.put(
+                    Integer.valueOf(lrearrange.getIndex()),
+                    new Pair<Boolean, Map<Integer, Integer>>(lrearrange
+                            .isProjectStar(), lrearrange.getProjectedColsMap()));
+            if (LOG.isDebugEnabled())
+          LOG.debug("KeyInfo for packager for package operator " + pkg + " is "
+              + keyInfo );
+            pkg.getPkgr().setKeyInfo(keyInfo);
+            pkg.getPkgr().setKeyTuple(lrearrange.isKeyTuple());
+            pkg.getPkgr().setKeyCompound(lrearrange.isKeyCompound());
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkPrinter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkPrinter.java
new file mode 100644
index 0000000..42ca525
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/SparkPrinter.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.io.PrintStream;
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PlanPrinter;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+/**
+ * A visitor mechanism printing out the logical plan.
+ */
+public class SparkPrinter extends SparkOpPlanVisitor {
+
+    private PrintStream mStream = null;
+    private boolean isVerbose = true;
+
+    public SparkPrinter(PrintStream ps, SparkOperPlan plan) {
+        super(plan, new DepthFirstWalker<SparkOperator, SparkOperPlan>(plan));
+        mStream = ps;
+        mStream.println("#--------------------------------------------------");
+        mStream.println("# Spark Plan");
+        mStream.println("#--------------------------------------------------");
+    }
+
+    public void setVerbose(boolean verbose) {
+        isVerbose = verbose;
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+        mStream.println("");
+        mStream.println("Spark node " + sparkOp.getOperatorKey().toString());
+        if (sparkOp instanceof NativeSparkOperator) {
+            mStream.println(((NativeSparkOperator)sparkOp).getCommandString());
+            mStream.println("--------");
+            mStream.println();
+            return;
+        }
+        if (sparkOp.physicalPlan != null && sparkOp.physicalPlan.size() > 0) {
+            PlanPrinter<PhysicalOperator, PhysicalPlan> printer = new PlanPrinter<PhysicalOperator, PhysicalPlan>(
+                    sparkOp.physicalPlan, mStream);
+            printer.setVerbose(isVerbose);
+            printer.visit();
+            mStream.println("--------");
+        }
+        List<POGlobalRearrangeSpark> glrList = PlanHelper.getPhysicalOperators(sparkOp.physicalPlan, POGlobalRearrangeSpark.class);
+        for (POGlobalRearrangeSpark glr : glrList) {
+            if (glr.isUseSecondaryKey()) {
+                mStream.println("POGlobalRearrange(" + glr.getOperatorKey() + ") uses secondaryKey");
+            }
+        }
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/XMLSparkPrinter.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/XMLSparkPrinter.java
new file mode 100644
index 0000000..48efcb4
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/plan/XMLSparkPrinter.java
@@ -0,0 +1,97 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with this
+ * work for additional information regarding copyright ownership. The ASF
+ * licenses this file to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.plan;
+
+import java.io.PrintStream;
+import java.io.StringWriter;
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.Transformer;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.dom.DOMSource;
+import javax.xml.transform.stream.StreamResult;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.XMLPhysicalPlanPrinter;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.impl.plan.DepthFirstWalker;
+import org.apache.pig.impl.plan.VisitorException;
+
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+
+import static javax.xml.transform.OutputKeys.INDENT;
+import static javax.xml.transform.OutputKeys.OMIT_XML_DECLARATION;
+
+
+public class XMLSparkPrinter extends SparkOpPlanVisitor {
+
+    private PrintStream mStream = null;
+
+    private Document doc = null;
+    private Element root = null;
+
+    public XMLSparkPrinter(PrintStream ps, SparkOperPlan plan) throws ParserConfigurationException {
+        super(plan, new DepthFirstWalker<SparkOperator, SparkOperPlan>(plan));
+        mStream = ps;
+        DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+        DocumentBuilder builder = factory.newDocumentBuilder();
+        this.doc = builder.newDocument();
+        this.root = this.doc.createElement("sparkPlan");
+        this.doc.appendChild(this.root);
+
+    }
+
+
+    public void closePlan() throws TransformerException {
+        TransformerFactory factory = TransformerFactory.newInstance();
+        Transformer transformer = factory.newTransformer();
+        transformer.setOutputProperty(OMIT_XML_DECLARATION, "yes");
+        transformer.setOutputProperty(INDENT, "yes");
+        transformer.setOutputProperty("{http://xml.apache.org/xslt}indent-amount", "2");
+
+        StringWriter sw = new StringWriter();
+        StreamResult result = new StreamResult(sw);
+        DOMSource source = new DOMSource(doc);
+        transformer.transform(source, result);
+        mStream.println(sw.toString());
+    }
+
+    @Override
+    public void visitSparkOp(SparkOperator so) throws VisitorException {
+        Element sparkNode = doc.createElement("sparkNode");
+        sparkNode.setAttribute("scope", "" + so.getOperatorKey().id);
+        if(so instanceof NativeSparkOperator) {
+            Element nativeSparkOper = doc.createElement("nativeSpark");
+            nativeSparkOper.setTextContent(((NativeSparkOperator)so).getCommandString());
+            sparkNode.appendChild(nativeSparkOper);
+            root.appendChild(sparkNode);
+            return;
+        }
+
+        if (so.physicalPlan != null && so.physicalPlan.size() > 0) {
+            XMLPhysicalPlanPrinter<PhysicalPlan> printer = new XMLPhysicalPlanPrinter<>(so.physicalPlan, doc, sparkNode);
+            printer.visit();
+        }
+
+        root.appendChild(sparkNode);
+
+    }
+
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/running/PigInputFormatSpark.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/running/PigInputFormatSpark.java
new file mode 100644
index 0000000..604665e
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/running/PigInputFormatSpark.java
@@ -0,0 +1,134 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.running;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.hadoop.mapreduce.lib.input.FileSplit;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigHadoopLogger;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigInputFormat;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigMapReduce;
+import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.PigSplit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.shims.HadoopShims;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigRecordReader;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkPigSplit;
+import org.apache.pig.backend.hadoop.executionengine.util.MapRedUtil;
+import org.apache.pig.data.SchemaTupleBackend;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.PigImplConstants;
+import org.apache.pig.impl.util.ObjectSerializer;
+import org.apache.pig.impl.util.UDFContext;
+import org.apache.pig.tools.pigstats.spark.SparkCounters;
+
+public class PigInputFormatSpark extends PigInputFormat {
+
+    @Override
+    public RecordReader<Text, Tuple> createRecordReader(InputSplit split, TaskAttemptContext context) throws
+            IOException, InterruptedException {
+        resetUDFContext();
+        //PigSplit#conf is the default hadoop configuration, we need get the configuration
+        //from context.getConfigration() to retrieve pig properties
+        PigSplit pigSplit = ((SparkPigSplit) split).getWrappedPigSplit();
+        Configuration conf = context.getConfiguration();
+        pigSplit.setConf(conf);
+        //Set current splitIndex in PigMapReduce.sJobContext.getConfiguration.get(PigImplConstants.PIG_SPLIT_INDEX)
+        //which will be used in POMergeCogroup#setup
+        if (PigMapReduce.sJobContext == null) {
+            PigMapReduce.sJobContext = HadoopShims.createJobContext(conf, new JobID());
+        }
+        PigMapReduce.sJobContext.getConfiguration().setInt(PigImplConstants.PIG_SPLIT_INDEX, pigSplit.getSplitIndex());
+        // Here JobConf is first available in spark Executor thread, we initialize PigContext,UDFContext and
+        // SchemaTupleBackend by reading properties from JobConf
+        initialize(conf);
+
+        SparkRecordReaderFactory sparkRecordReaderFactory = new SparkRecordReaderFactory(pigSplit, context);
+        return sparkRecordReaderFactory.createRecordReader();
+    }
+
+    /**
+     * This is where we have to wrap PigSplits into SparkPigSplits
+     * @param jobcontext
+     * @return
+     * @throws IOException
+     * @throws InterruptedException
+     */
+    @Override
+    public List<InputSplit> getSplits(JobContext jobcontext) throws IOException, InterruptedException {
+        List<InputSplit> sparkPigSplits = new ArrayList<>();
+        List<InputSplit> originalSplits = super.getSplits(jobcontext);
+
+        boolean isFileSplits = true;
+        for (InputSplit inputSplit : originalSplits) {
+            PigSplit split = (PigSplit)inputSplit;
+            if (!(split.getWrappedSplit() instanceof FileSplit)) {
+                isFileSplits = false;
+                break;
+            }
+        }
+
+        for (InputSplit inputSplit : originalSplits) {
+            PigSplit split = (PigSplit) inputSplit;
+            if (!isFileSplits) {
+                sparkPigSplits.add(new SparkPigSplit.GenericSparkPigSplit(split));
+            } else {
+                sparkPigSplits.add(new SparkPigSplit.FileSparkPigSplit(split));
+            }
+        }
+
+        return sparkPigSplits;
+    }
+
+    private void initialize(Configuration jobConf) throws IOException {
+        MapRedUtil.setupUDFContext(jobConf);
+        PigContext pc = (PigContext) ObjectSerializer.deserialize(jobConf.get("pig.pigContext"));
+        SchemaTupleBackend.initialize(jobConf, pc);
+        PigMapReduce.sJobConfInternal.set(jobConf);
+        PigHadoopLogger pigHadoopLogger = PigHadoopLogger.getInstance();
+        pigHadoopLogger.setAggregate("true".equalsIgnoreCase(jobConf.get("aggregate.warning")));
+        pigHadoopLogger.setReporter((SparkCounters)ObjectSerializer.deserialize(jobConf.get("pig.spark.counters")));
+        PhysicalOperator.setPigLogger(pigHadoopLogger);
+    }
+
+    private void resetUDFContext() {
+		UDFContext.getUDFContext().reset();
+	}
+
+
+    static class SparkRecordReaderFactory extends PigInputFormat.RecordReaderFactory {
+
+        public SparkRecordReaderFactory(InputSplit split, TaskAttemptContext context) throws IOException {
+            super(split, context);
+        }
+
+        @Override
+        public RecordReader<Text, Tuple> createRecordReader() throws IOException, InterruptedException {
+            return new SparkPigRecordReader(inputFormat, pigSplit, loadFunc, context, limit);
+        }
+    }
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/spark/streaming/SparkExecutableManager.java b/src/org/apache/pig/backend/hadoop/executionengine/spark/streaming/SparkExecutableManager.java
new file mode 100644
index 0000000..cba0bd4
--- /dev/null
+++ b/src/org/apache/pig/backend/hadoop/executionengine/spark/streaming/SparkExecutableManager.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.backend.hadoop.executionengine.spark.streaming;
+
+import org.apache.pig.backend.hadoop.streaming.HadoopExecutableManager;
+import org.apache.spark.TaskContext;
+
+public class SparkExecutableManager extends HadoopExecutableManager {
+    @Override
+    protected boolean writeErrorToHDFS(int limit, String taskId) {
+        if (command.getPersistStderr()) {
+            int tipId = TaskContext.get().attemptNumber();
+            return tipId < command.getLogFilesLimit();
+        }
+        return false;
+    }
+}
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/AccumulatorOptimizer.java b/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/AccumulatorOptimizer.java
index eb5b801..b1cf2a7 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/AccumulatorOptimizer.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/AccumulatorOptimizer.java
@@ -37,6 +37,6 @@
 
     @Override
     public void visitTezOp(TezOperator tezOp) throws VisitorException {
-        AccumulatorOptimizerUtil.addAccumulator(tezOp.plan);
+        AccumulatorOptimizerUtil.addAccumulator(tezOp.plan, tezOp.plan.getRoots());
     }
 }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/SecondaryKeyOptimizerTez.java b/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/SecondaryKeyOptimizerTez.java
index caf1786..4bdb565 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/SecondaryKeyOptimizerTez.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/tez/plan/optimizer/SecondaryKeyOptimizerTez.java
@@ -102,7 +102,8 @@
             rearrangePlan = PlanHelper.getLocalRearrangePlanFromSplit(from.plan, connectingLR.getOperatorKey());
         }
 
-        SecondaryKeyOptimizerInfo info = SecondaryKeyOptimizerUtil.applySecondaryKeySort(rearrangePlan, to.plan);
+        SecondaryKeyOptimizerUtil secondaryKeyOptUtil = new SecondaryKeyOptimizerUtil();
+        SecondaryKeyOptimizerInfo info = secondaryKeyOptUtil.applySecondaryKeySort(rearrangePlan, to.plan);
         if (info != null) {
             numSortRemoved += info.getNumSortRemoved();
             numDistinctChanged += info.getNumDistinctChanged();
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java
index c4b44ad..571622a 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/util/AccumulatorOptimizerUtil.java
@@ -37,11 +37,15 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.expressionOperators.UnaryExpressionOperator;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSortedDistinct;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.Packager;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.VisitorException;
 
 public class AccumulatorOptimizerUtil {
     private static final Log LOG = LogFactory.getLog(AccumulatorOptimizerUtil.class);
@@ -57,9 +61,8 @@
         return batchSize;
     }
 
-    public static void addAccumulator(PhysicalPlan plan) {
+    public static void addAccumulator(PhysicalPlan plan, List<PhysicalOperator> pos) {
         // See if this is a map-reduce job
-        List<PhysicalOperator> pos = plan.getRoots();
         if (pos == null || pos.size() == 0) {
             return;
         }
@@ -286,4 +289,4 @@
 
         return false;
     }
-}
+}
\ No newline at end of file
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/util/CombinerOptimizerUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/util/CombinerOptimizerUtil.java
index 889c01b..cc71afe 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/util/CombinerOptimizerUtil.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/util/CombinerOptimizerUtil.java
@@ -333,7 +333,7 @@
      * @return null if plan is not combinable, otherwise list of combinable operators
      * @throws VisitorException
      */
-    private static List<Pair<PhysicalOperator, PhysicalPlan>> findAlgebraicOps(List<PhysicalPlan> feInners)
+    public static List<Pair<PhysicalOperator, PhysicalPlan>> findAlgebraicOps(List<PhysicalPlan> feInners)
             throws VisitorException {
         List<Pair<PhysicalOperator, PhysicalPlan>> algebraicOps = Lists.newArrayList();
 
@@ -447,7 +447,7 @@
      * @param keyType type for group-by key
      * @return new POForeach
      */
-    private static POForEach createForEachWithGrpProj(POForEach foreach, byte keyType) {
+    public static POForEach createForEachWithGrpProj(POForEach foreach, byte keyType) {
         String scope = foreach.getOperatorKey().scope;
         POForEach newFE = new POForEach(createOperatorKey(scope), new ArrayList<PhysicalPlan>());
         newFE.addOriginalLocation(foreach.getAlias(), foreach.getOriginalLocations());
@@ -471,7 +471,7 @@
      * @throws CloneNotSupportedException
      * @throws PlanException
      */
-    private static PhysicalPlan createPlanWithPredecessors(PhysicalOperator algeOp, PhysicalPlan pplan)
+    public static PhysicalPlan createPlanWithPredecessors(PhysicalOperator algeOp, PhysicalPlan pplan)
             throws CloneNotSupportedException, PlanException {
         PhysicalPlan newplan = new PhysicalPlan();
         addPredecessorsToPlan(algeOp, pplan, newplan);
@@ -508,7 +508,7 @@
      * @throws CloneNotSupportedException
      * @throws PlanException
      */
-    private static void addAlgebraicFuncToCombineFE(POForEach cfe, Map<PhysicalOperator, Integer> op2newpos)
+    public static void addAlgebraicFuncToCombineFE(POForEach cfe, Map<PhysicalOperator, Integer> op2newpos)
             throws CloneNotSupportedException, PlanException {
         // an array that we will first populate with physical operators in order
         // of their position in input. Used while adding plans to combine
@@ -578,7 +578,7 @@
      * @param rearrange
      * @return
      */
-    private static POPreCombinerLocalRearrange getPreCombinerLR(POLocalRearrange rearrange) {
+    public static POPreCombinerLocalRearrange getPreCombinerLR(POLocalRearrange rearrange) {
         String scope = rearrange.getOperatorKey().scope;
         POPreCombinerLocalRearrange pclr = new POPreCombinerLocalRearrange(
                 createOperatorKey(scope),
@@ -587,7 +587,7 @@
         return pclr;
     }
 
-    private static OperatorKey createOperatorKey(String scope) {
+    public static OperatorKey createOperatorKey(String scope) {
         return new OperatorKey(scope, NodeIdGenerator.getGenerator().getNextNodeId(scope));
     }
 
@@ -619,7 +619,7 @@
      * @param type
      * @throws PlanException
      */
-    private static void changeFunc(POForEach fe, byte type) throws PlanException {
+    public static void changeFunc(POForEach fe, byte type) throws PlanException {
         for (PhysicalPlan plan : fe.getInputPlans()) {
             List<PhysicalOperator> leaves = plan.getLeaves();
             if (leaves == null || leaves.size() != 1) {
@@ -657,7 +657,7 @@
      * @throws PlanException
      * @throws CloneNotSupportedException
      */
-    private static POLocalRearrange getNewRearrange(POLocalRearrange rearrange)
+    public static POLocalRearrange getNewRearrange(POLocalRearrange rearrange)
             throws PlanException, CloneNotSupportedException {
         POLocalRearrange newRearrange = rearrange.clone();
 
@@ -835,7 +835,7 @@
      * with
      * POUserFunc(org.apache.pig.builtin.Distinct)[DataBag]
      */
-    private static class DistinctPatcher extends PhyPlanVisitor {
+    public static class DistinctPatcher extends PhyPlanVisitor {
         private POUserFunc distinct = null;
         /**
          * @param plan
@@ -901,12 +901,12 @@
             }
         }
 
-        POUserFunc getDistinct() {
+        public POUserFunc getDistinct() {
             return distinct;
         }
     }
 
-    private static class fixMapProjects extends PhyPlanVisitor {
+    public static class fixMapProjects extends PhyPlanVisitor {
         public fixMapProjects(PhysicalPlan plan) {
             this(plan, new DepthFirstWalker<PhysicalOperator, PhysicalPlan>(plan));
         }
diff --git a/src/org/apache/pig/backend/hadoop/executionengine/util/SecondaryKeyOptimizerUtil.java b/src/org/apache/pig/backend/hadoop/executionengine/util/SecondaryKeyOptimizerUtil.java
index fcd316b..c655fcf 100644
--- a/src/org/apache/pig/backend/hadoop/executionengine/util/SecondaryKeyOptimizerUtil.java
+++ b/src/org/apache/pig/backend/hadoop/executionengine/util/SecondaryKeyOptimizerUtil.java
@@ -35,12 +35,15 @@
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.PODistinct;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POFilter;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POForEach;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POGlobalRearrange;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLimit;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLocalRearrange;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POPackage;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSort;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSortedDistinct;
 import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POUnion;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POGlobalRearrangeSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.POReduceBySpark;
 import org.apache.pig.classification.InterfaceAudience;
 import org.apache.pig.data.DataType;
 import org.apache.pig.impl.io.PigNullableWritable;
@@ -54,7 +57,7 @@
 public class SecondaryKeyOptimizerUtil {
     private static Log log = LogFactory.getLog(SecondaryKeyOptimizerUtil.class.getName());
 
-    private SecondaryKeyOptimizerUtil() {
+    public SecondaryKeyOptimizerUtil() {
 
     }
 
@@ -182,7 +185,7 @@
         return result;
     }
 
-    public static SecondaryKeyOptimizerInfo applySecondaryKeySort(PhysicalPlan mapPlan, PhysicalPlan reducePlan) throws VisitorException {
+    public SecondaryKeyOptimizerInfo applySecondaryKeySort(PhysicalPlan mapPlan, PhysicalPlan reducePlan) throws VisitorException {
         log.trace("Entering SecondaryKeyOptimizerUtil.addSecondaryKeySort");
         SecondaryKeyOptimizerInfo secKeyOptimizerInfo = new SecondaryKeyOptimizerInfo();
         List<SortKeyInfo> sortKeyInfos = new ArrayList<SortKeyInfo>();
@@ -241,14 +244,11 @@
         }
 
         PhysicalOperator root = reduceRoots.get(0);
-        if (!(root instanceof POPackage)) {
-            log.debug("Expected reduce root to be a POPackage, skip secondary key optimizing");
-            return null;
-        }
+        PhysicalOperator currentNode = getCurrentNode(root,reducePlan);
 
         // visit the POForEach of the reduce plan. We can have Limit and Filter
         // in the middle
-        PhysicalOperator currentNode = root;
+
         POForEach foreach = null;
         while (currentNode != null) {
             if (currentNode instanceof POPackage
@@ -402,12 +402,33 @@
                     throw new VisitorException("Cannot find POLocalRearrange to set secondary plan", errorCode);
                 }
             }
-            POPackage pack = (POPackage) root;
-            pack.getPkgr().setUseSecondaryKey(true);
+
+            if (root instanceof POGlobalRearrangeSpark) {
+                POGlobalRearrangeSpark plg = (POGlobalRearrangeSpark) root;
+                plg.setUseSecondaryKey(true);
+                plg.setSecondarySortOrder(secondarySortKeyInfo.getAscs());
+            } else if (root instanceof POPackage) {
+                POPackage pack = (POPackage) root;
+                pack.getPkgr().setUseSecondaryKey(true);
+            } else if (root instanceof POReduceBySpark) {
+                POReduceBySpark reduceBySpark = (POReduceBySpark) root;
+                reduceBySpark.setUseSecondaryKey(true);
+                reduceBySpark.setSecondarySortOrder(secondarySortKeyInfo.getAscs());
+            }
         }
         return secKeyOptimizerInfo;
     }
 
+    protected PhysicalOperator getCurrentNode(PhysicalOperator root, PhysicalPlan reducePlan) {
+        PhysicalOperator currentNode = null;
+        if (!(root instanceof POPackage)) {
+            log.debug("Expected reduce root to be a POPackage, skip secondary key optimizing");
+        } else {
+            currentNode = root;
+        }
+        return currentNode;
+    }
+
     private static void setSecondaryPlan(PhysicalPlan plan, POLocalRearrange rearrange,
             SortKeyInfo secondarySortKeyInfo) throws VisitorException {
         // Put plan to project secondary key to the POLocalRearrange
@@ -663,5 +684,4 @@
         }
         return false;
     }
-
 }
diff --git a/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java b/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
index 1826131..7a63b8d 100644
--- a/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
+++ b/src/org/apache/pig/backend/hadoop/hbase/HBaseStorage.java
@@ -301,8 +301,15 @@
         }
 
         columnInfo_ = parseColumnList(columnList, delimiter_, ignoreWhitespace_);
-
-        String defaultCaster = UDFContext.getUDFContext().getClientSystemProps().getProperty(CASTER_PROPERTY, STRING_CASTER);
+        //In mr,  UDFContext.deserialize is first called and then UDFContext.getUDFContext().getClientSystemProps() is called,
+        //the value is not null.
+        //In spark mode, when spark executor first initializes all
+        //the object,UDFContext.getUDFContext().getClientSystemProps() is null and then UDFContext.deserialize is called.
+        //so we need check whether UDFContext.getUDFContext().getClientSystemProps()
+        //is null or not, if is null, defaultCaster =STRING_CASTER, otherwise is
+        //UDFContext.getUDFContext().getClientSystemProps().getProperty(CASTER_PROPERTY, STRING_CASTER)
+        //Detail see PIG-4920
+        String defaultCaster = UDFContext.getUDFContext().getClientSystemProps() != null ? UDFContext.getUDFContext().getClientSystemProps().getProperty(CASTER_PROPERTY, STRING_CASTER) : STRING_CASTER;
         String casterOption = configuredOptions_.getOptionValue("caster", defaultCaster);
         if (STRING_CASTER.equalsIgnoreCase(casterOption)) {
             caster_ = new Utf8StorageConverter();
diff --git a/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java b/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java
index 951146f..07eaf0b 100644
--- a/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java
+++ b/src/org/apache/pig/backend/hadoop/streaming/HadoopExecutableManager.java
@@ -172,7 +172,7 @@
      * @return <code>true</code> if stderr data of task should be persisted on 
      *         HDFS, <code>false</code> otherwise
      */
-    private boolean writeErrorToHDFS(int limit, String taskId) {
+    protected boolean writeErrorToHDFS(int limit, String taskId) {
         if (command.getPersistStderr() && taskId != null) {
             int tipId = TaskAttemptID.forName(taskId).getTaskID().getId();
             return tipId < command.getLogFilesLimit();
diff --git a/src/org/apache/pig/data/SelfSpillBag.java b/src/org/apache/pig/data/SelfSpillBag.java
index d17f0a8..633f39a 100644
--- a/src/org/apache/pig/data/SelfSpillBag.java
+++ b/src/org/apache/pig/data/SelfSpillBag.java
@@ -22,6 +22,8 @@
 import org.apache.pig.classification.InterfaceAudience;
 import org.apache.pig.classification.InterfaceStability;
 
+import java.io.Serializable;
+
 /**
  * Class to hold code common to self spilling bags such as InternalCachedBag
  */
@@ -29,6 +31,7 @@
 @InterfaceStability.Evolving
 public abstract class SelfSpillBag extends DefaultAbstractBag {
     private static final long serialVersionUID = 1L;
+    // SelfSpillBag$MemoryLimits is not serializable
     protected MemoryLimits memLimit;
 
     public SelfSpillBag(int bagCount) {
@@ -47,10 +50,11 @@
      * The number of objects that will fit into this memory limit is computed
      * using the average memory size of the objects whose size is given to this
      * class.
+     * In spark mode, MemoryLimits needs implement Serializable interface otherwise NotSerializableExecption will be thrown (See PIG-4611)
      */
     @InterfaceAudience.Private
     @InterfaceStability.Evolving
-    public static class MemoryLimits {
+    public static class MemoryLimits implements Serializable {
 
         private long maxMemUsage;
         private long cacheLimit = Integer.MAX_VALUE;
diff --git a/src/org/apache/pig/impl/PigContext.java b/src/org/apache/pig/impl/PigContext.java
index d43949f..d142eba 100644
--- a/src/org/apache/pig/impl/PigContext.java
+++ b/src/org/apache/pig/impl/PigContext.java
@@ -17,6 +17,9 @@
  */
 package org.apache.pig.impl;
 
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.File;
@@ -26,6 +29,8 @@
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.io.Serializable;
 import java.io.StringWriter;
 import java.lang.reflect.Constructor;
@@ -906,5 +911,5 @@
         } else {
             classloader = new ContextClassLoader(cl);
         }
-    }
+   }
 }
diff --git a/src/org/apache/pig/impl/builtin/StreamingUDF.java b/src/org/apache/pig/impl/builtin/StreamingUDF.java
index de98539..e1ce0da 100644
--- a/src/org/apache/pig/impl/builtin/StreamingUDF.java
+++ b/src/org/apache/pig/impl/builtin/StreamingUDF.java
@@ -242,8 +242,11 @@
             }
             InputStream udfFileStream = this.getClass().getResourceAsStream(
                     absolutePath + getUserFileExtension());
-            command[PATH_TO_FILE_CACHE] = "\"" + userUdfFile.getParentFile().getAbsolutePath()
-                    + "\"";
+            if (udfFileStream == null) {
+                //Try loading the script from other locally available jars (needed for Spark mode)
+                udfFileStream = Thread.currentThread().getContextClassLoader().getResourceAsStream(filePath+getUserFileExtension());
+            }
+            command[PATH_TO_FILE_CACHE] = "\"" + userUdfFile.getParentFile().getAbsolutePath() + "\"";
 
             try {
                 FileUtils.copyInputStreamToFile(udfFileStream, userUdfFile);
diff --git a/src/org/apache/pig/impl/plan/OperatorPlan.java b/src/org/apache/pig/impl/plan/OperatorPlan.java
index 8b2e2e7..0851613 100644
--- a/src/org/apache/pig/impl/plan/OperatorPlan.java
+++ b/src/org/apache/pig/impl/plan/OperatorPlan.java
@@ -534,6 +534,24 @@
             connect(oper, leaf);
         }
     }
+
+    /**
+     * Adds the root operator to the plan and connects
+     * all existing roots the new root
+     *
+     * @param root
+     * @throws PlanException
+     */
+    public void addAsRoot(E root) throws PlanException {
+        List<E> oldRoots = new ArrayList<E>();
+        for (E operator : getRoots()) {
+            oldRoots.add(operator);
+        }
+        add(root);
+        for (E oper : oldRoots) {
+            connect(root, oper);
+        }
+    }
     
     public boolean isSingleLeafPlan() {
         List<E> tmpList = getLeaves() ;
diff --git a/src/org/apache/pig/impl/util/UDFContext.java b/src/org/apache/pig/impl/util/UDFContext.java
index 09afc0a..4835cb7 100644
--- a/src/org/apache/pig/impl/util/UDFContext.java
+++ b/src/org/apache/pig/impl/util/UDFContext.java
@@ -23,11 +23,14 @@
 import java.util.HashMap;
 import java.util.Properties;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
 
 public class UDFContext {
 
+    private static final Log LOG = LogFactory.getLog(UDFContext.class);
     private Configuration jconf = null;
     private HashMap<UDFContextKey, Properties> udfConfs;
     private Properties clientSysProps;
@@ -76,7 +79,7 @@
     /*
      *  internal pig use only - should NOT be called from user code
      */
-    HashMap<UDFContextKey, Properties> getUdfConfs() {
+    public HashMap<UDFContextKey, Properties> getUdfConfs() {
         return udfConfs;
     }
 
@@ -204,6 +207,17 @@
         conf.set(CLIENT_SYS_PROPS, ObjectSerializer.serialize(clientSysProps));
     }
 
+    /*
+     * Internal pig use
+     */
+    public String serialize() {
+        try {
+            return ObjectSerializer.serialize(udfConfs);
+        } catch (IOException e) {
+            LOG.error("UDFContext#serialize throws error ",e);
+            return null;
+        }
+    }
 
     /**
      * Populate the udfConfs field.  This function is intended to
@@ -218,6 +232,14 @@
                 jconf.get(CLIENT_SYS_PROPS));
     }
 
+    public void deserializeForSpark(String udfConfsStr, String clientSysPropsStr) throws IOException {
+        if( udfConfsStr!= null && clientSysPropsStr!=null) {
+            udfConfs = (HashMap<UDFContextKey, Properties>) ObjectSerializer.deserialize(udfConfsStr);
+            clientSysProps = (Properties) ObjectSerializer.deserialize(
+                    clientSysPropsStr);
+        }
+    }
+
     private UDFContextKey generateKey(Class<?> c, String[] args) {
         return new UDFContextKey(c.getName(), args);
     }
@@ -314,4 +336,8 @@
         }
     }
 
+    public Properties getClientSysProps() {
+        return clientSysProps;
+    }
+
 }
\ No newline at end of file
diff --git a/src/org/apache/pig/impl/util/avro/AvroTupleWrapper.java b/src/org/apache/pig/impl/util/avro/AvroTupleWrapper.java
index 8adbe84..91c46bc 100644
--- a/src/org/apache/pig/impl/util/avro/AvroTupleWrapper.java
+++ b/src/org/apache/pig/impl/util/avro/AvroTupleWrapper.java
@@ -26,8 +26,17 @@
 import org.apache.avro.Schema.Type;
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericDatumWriter;
 import org.apache.avro.generic.GenericEnumSymbol;
 import org.apache.avro.generic.IndexedRecord;
+import org.apache.avro.io.DatumReader;
+import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.Decoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.Encoder;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.util.Utf8;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.pig.backend.executionengine.ExecException;
@@ -38,6 +47,8 @@
 import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
 import java.nio.ByteBuffer;
 import java.util.Iterator;
 import java.util.List;
@@ -50,12 +61,12 @@
 public final class AvroTupleWrapper <T extends IndexedRecord>
     implements Tuple {
     private static final Log LOG = LogFactory.getLog(AvroTupleWrapper.class);
-    private TupleFactory mTupleFactory = TupleFactory.getInstance();
+    private transient TupleFactory mTupleFactory = TupleFactory.getInstance();
 
   /**
    * The Avro object wrapped in the pig Tuple.
    */
-  private T avroObject;
+  private transient T avroObject;
 
   /**
    * Creates a new AvroTupleWrapper object.
@@ -205,7 +216,14 @@
       case NULL:
         break;
       case STRING:
-        total += ((String) r.get(f.pos())).length()
+        Object val = r.get(f.pos());
+        String value;
+        if (val instanceof Utf8) {
+          value = val.toString();
+        } else {
+          value = (String) val;
+        }
+        total += value.length()
            * (Character.SIZE << bitsPerByte);
         break;
       case BYTES:
@@ -291,4 +309,21 @@
         );
   }
 
+  // Required for Java serialization used by Spark: PIG-5134
+  private void writeObject(ObjectOutputStream out) throws IOException {
+    out.writeObject(avroObject.getSchema().toString());
+    DatumWriter<T> writer = new GenericDatumWriter<>();
+    writer.setSchema(avroObject.getSchema());
+    Encoder encoder = EncoderFactory.get().binaryEncoder(out, null);
+    writer.write(avroObject, encoder);
+    encoder.flush();
+  }
+
+  // Required for Java serialization used by Spark: PIG-5134
+  private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException {
+    Schema schema = new Schema.Parser().parse((String) in.readObject());
+    DatumReader<T> reader = new GenericDatumReader<>(schema);
+    Decoder decoder = DecoderFactory.get().binaryDecoder(in, null);
+    avroObject = reader.read(avroObject, decoder);
+  }
 }
diff --git a/src/org/apache/pig/scripting/groovy/GroovyEvalFunc.java b/src/org/apache/pig/scripting/groovy/GroovyEvalFunc.java
index cff59db..c83ca22 100644
--- a/src/org/apache/pig/scripting/groovy/GroovyEvalFunc.java
+++ b/src/org/apache/pig/scripting/groovy/GroovyEvalFunc.java
@@ -74,6 +74,10 @@
               resource = ScriptEngine.class.getResource(File.separator + path);
           }
           if (resource == null) {
+            //Try loading the script from other locally available jars (needed for Spark mode)
+              resource = Thread.currentThread().getContextClassLoader().getResource(path);
+          }
+          if (resource == null) {
               throw new IOException("Cannot find " + path);
           }
       } else {
diff --git a/src/org/apache/pig/tools/pigstats/PigStatsUtil.java b/src/org/apache/pig/tools/pigstats/PigStatsUtil.java
index e97625f..53a0df4 100644
--- a/src/org/apache/pig/tools/pigstats/PigStatsUtil.java
+++ b/src/org/apache/pig/tools/pigstats/PigStatsUtil.java
@@ -165,7 +165,7 @@
     private static final String SEPARATOR = "/";
     private static final String SEMICOLON = ";";
 
-    private static String getShortName(String uri) {
+    public static String getShortName(String uri) {
         int scolon = uri.indexOf(SEMICOLON);
         int slash;
         if (scolon!=-1) {
diff --git a/src/org/apache/pig/tools/pigstats/PigStatusReporter.java b/src/org/apache/pig/tools/pigstats/PigStatusReporter.java
index 5396535..b52bd15 100644
--- a/src/org/apache/pig/tools/pigstats/PigStatusReporter.java
+++ b/src/org/apache/pig/tools/pigstats/PigStatusReporter.java
@@ -27,7 +27,7 @@
 
 @InterfaceAudience.Public
 @InterfaceStability.Evolving
-public class PigStatusReporter extends StatusReporter implements Progressable {
+public class PigStatusReporter extends StatusReporter implements Progressable, PigWarnCounter {
 
     private static PigStatusReporter reporter = null;
 
@@ -86,6 +86,16 @@
     }
 
     @Override
+    public boolean incrWarnCounter(Enum<?> name, Object incr) {
+        return incrCounter(name, (Long)incr);
+    }
+
+    @Override
+    public boolean incrWarnCounter(String group, String name, Object incr) {
+        return incrCounter(group, name, (Long)incr);
+    }
+
+    @Override
     public void progress() {
         if (context != null) {
             context.progress();
diff --git a/src/org/apache/pig/tools/pigstats/PigWarnCounter.java b/src/org/apache/pig/tools/pigstats/PigWarnCounter.java
new file mode 100644
index 0000000..28e2e89
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/PigWarnCounter.java
@@ -0,0 +1,29 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats;
+
+/*
+    Interface for incrementing warning counters
+ */
+public interface PigWarnCounter {
+
+    boolean incrWarnCounter(Enum<?> name, Object incr);
+
+    boolean incrWarnCounter(String group, String name, Object incr);
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkCounter.java b/src/org/apache/pig/tools/pigstats/spark/SparkCounter.java
new file mode 100644
index 0000000..6b6dd34
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkCounter.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats.spark;
+
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.spark.Accumulator;
+import org.apache.spark.AccumulatorParam;
+import org.apache.spark.api.java.JavaSparkContext;
+
+public abstract class SparkCounter<T> implements Serializable {
+
+    private String name;
+    private String displayName;
+    private Accumulator<T> accumulator;
+
+    public SparkCounter() {
+        // For serialization.
+    }
+
+    public SparkCounter(
+            String name,
+            String displayName,
+            String groupName,
+            T initValue,
+            JavaSparkContext sparkContext) {
+
+        this.name = name;
+        this.displayName = displayName;
+
+        String accumulatorName = groupName + "_" + name;
+
+        if (sparkContext == null){
+            //Spark executors can register new Accumulators but they won't make it back to the driver hence the limitation
+            throw new  RuntimeException("Not allowed to create SparkCounter on backend executor.");
+
+        }
+        this.accumulator = sparkContext.accumulator(initValue, accumulatorName,  createAccumulatorParam());
+
+    }
+
+    protected abstract AccumulatorParam<T> createAccumulatorParam();
+
+    public T getValue() {
+        if (accumulator != null) {
+            return accumulator.value();
+        } else {
+            return null;
+        }
+    }
+
+    public void increment(T incr) {
+        accumulator.add(incr);
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public String getDisplayName() {
+        return displayName;
+    }
+
+    public void setDisplayName(String displayName) {
+        this.displayName = displayName;
+    }
+
+    public static class LongSparkCounter extends SparkCounter<Long> {
+
+        public LongSparkCounter(){}
+
+        public LongSparkCounter(
+                String name,
+                String displayName,
+                String groupName,
+                Long initValue,
+                JavaSparkContext sparkContext){
+            super(name, displayName, groupName, initValue, sparkContext);
+        }
+
+        @Override
+        protected AccumulatorParam<Long> createAccumulatorParam() {
+            return new LongAccumulatorParam();
+        }
+
+        private class LongAccumulatorParam implements AccumulatorParam<Long> {
+
+            @Override
+            public Long addAccumulator(Long t1, Long t2) {
+                return t1 + t2;
+            }
+
+            @Override
+            public Long addInPlace(Long r1, Long r2) {
+                return r1 + r2;
+            }
+
+            @Override
+            public Long zero(Long initialValue) {
+                return 0L;
+            }
+        }
+    }
+
+    public static class MapSparkCounter extends SparkCounter<Map<String,Long>> {
+
+        public MapSparkCounter(){}
+
+        public MapSparkCounter(
+                String name,
+                String displayName,
+                String groupName,
+                Map<String,Long> initValue,
+                JavaSparkContext sparkContext){
+            super(name, displayName, groupName, initValue, sparkContext);
+        }
+
+        @Override
+        protected AccumulatorParam<Map<String, Long>> createAccumulatorParam() {
+            return new MapAccumulatorParam();
+        }
+
+        private class MapAccumulatorParam implements AccumulatorParam<Map<String,Long>> {
+
+            @Override
+            public Map<String, Long> addAccumulator(Map<String, Long> t1, Map<String, Long> t2) {
+                return addInPlace(t1, t2);
+            }
+
+            @Override
+            public Map<String, Long> addInPlace(Map<String, Long> r1, Map<String, Long> r2) {
+                for (String key : r2.keySet()){
+                    Long r1val = r1.get(key);
+                    Long r2val = r2.get(key);
+                    r1.put(key,r1val == null ? r2val : r1val+r2val);
+                }
+                return r1;
+            }
+
+            @Override
+            public Map<String, Long> zero(Map<String, Long> initialValue) {
+                return new HashMap<>();
+            }
+        }
+    }
+
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkCounterGroup.java b/src/org/apache/pig/tools/pigstats/spark/SparkCounterGroup.java
new file mode 100644
index 0000000..2adde1a
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkCounterGroup.java
@@ -0,0 +1,98 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats.spark;
+
+
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+public abstract class SparkCounterGroup<T> implements Serializable {
+    protected String groupName;
+    protected String groupDisplayName;
+    protected Map<String, SparkCounter<T>> sparkCounters;
+
+    protected transient JavaSparkContext javaSparkContext;
+
+    private SparkCounterGroup() {
+        // For serialization.
+    }
+
+    public SparkCounterGroup(
+            String groupName,
+            String groupDisplayName,
+            JavaSparkContext javaSparkContext) {
+        this.groupName = groupName;
+        this.groupDisplayName = groupDisplayName;
+        this.javaSparkContext = javaSparkContext;
+        this.sparkCounters = new HashMap<String, SparkCounter<T>>();
+    }
+
+    public abstract void createCounter(String name, T initValue);
+
+    public SparkCounter getCounter(String name) {
+        return sparkCounters.get(name);
+    }
+
+    public String getGroupName() {
+        return groupName;
+    }
+
+    public String getGroupDisplayName() {
+        return groupDisplayName;
+    }
+
+    public void setGroupDisplayName(String groupDisplayName) {
+        this.groupDisplayName = groupDisplayName;
+    }
+
+    public Map<String, SparkCounter<T>> getSparkCounters() {
+        return sparkCounters;
+    }
+
+    public static class LongSparkCounterGroup extends SparkCounterGroup<Long> {
+
+        public LongSparkCounterGroup(
+                String groupName,
+                String groupDisplayName,
+                JavaSparkContext javaSparkContext) {
+            super(groupName,groupDisplayName,javaSparkContext);
+        }
+        public void createCounter(String name, Long initValue){
+            SparkCounter counter = new SparkCounter.LongSparkCounter(name, name, groupName, initValue, javaSparkContext);
+            sparkCounters.put(name,counter);
+        }
+    }
+
+    public static class MapSparkCounterGroup extends SparkCounterGroup<Map<String,Long>> {
+
+        public MapSparkCounterGroup(
+                String groupName,
+                String groupDisplayName,
+                JavaSparkContext javaSparkContext) {
+            super(groupName,groupDisplayName,javaSparkContext);
+        }
+        public void createCounter(String name, Map<String,Long> initValue){
+            SparkCounter counter = new SparkCounter.MapSparkCounter(name, name, groupName, initValue, javaSparkContext);
+            sparkCounters.put(name,counter);
+        }
+    }
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkCounters.java b/src/org/apache/pig/tools/pigstats/spark/SparkCounters.java
new file mode 100644
index 0000000..a61eab9
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkCounters.java
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats.spark;
+
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.PigWarning;
+import org.apache.pig.tools.pigstats.PigWarnCounter;
+import org.apache.spark.api.java.JavaSparkContext;
+
+import java.io.Serializable;
+import java.util.HashMap;
+import java.util.Map;
+
+public class SparkCounters implements Serializable, PigWarnCounter {
+    private static final long serialVersionUID = 1L;
+
+    private static final Log LOG = LogFactory.getLog(SparkCounters.class);
+
+    private Map<String, SparkCounterGroup> sparkCounterGroups;
+
+    private final transient JavaSparkContext javaSparkContext;
+
+    private SparkCounters() {
+        this(null);
+    }
+
+    public SparkCounters(JavaSparkContext javaSparkContext) {
+        this.javaSparkContext = javaSparkContext;
+        this.sparkCounterGroups = new HashMap<String, SparkCounterGroup>();
+    }
+
+    public void createCounter(Enum<?> key) {
+        createCounter(key.getDeclaringClass().getName(), key.name());
+    }
+
+    public void createCounter(String groupName, Enum<?> key) {
+        createCounter(groupName, key.name(), 0L);
+    }
+
+    public void createCounter(String groupName, String counterName) {
+        createCounter(groupName, counterName, 0L);
+    }
+
+    public void createCounter(String groupName, String counterName, Object initValue) {
+        getGroup(groupName).createCounter(counterName, initValue);
+    }
+
+    public void increment(Enum<?> key, long incrValue) {
+        increment(key.getDeclaringClass().getName(), key.name(), incrValue);
+    }
+
+    public void increment(String groupName, String counterName, long value) {
+        SparkCounter counter = getGroup(groupName).getCounter(counterName);
+        if (counter == null) {
+            LOG.error(String.format("counter[%s, %s] has not initialized before.", groupName, counterName));
+        } else {
+            counter.increment(value);
+        }
+    }
+
+    public Object getValue(String groupName, String counterName) {
+        SparkCounter counter = getGroup(groupName).getCounter(counterName);
+        if (counter == null) {
+            LOG.error(String.format("counter[%s, %s] has not initialized before.", groupName, counterName));
+            return null;
+        } else {
+            return counter.getValue();
+        }
+    }
+
+    public SparkCounter getCounter(String groupName, String counterName) {
+        return getGroup(groupName).getCounter(counterName);
+    }
+
+    public SparkCounter getCounter(Enum<?> key) {
+        return getCounter(key.getDeclaringClass().getName(), key.name());
+    }
+
+    private SparkCounterGroup getGroup(String groupName) {
+        SparkCounterGroup group = sparkCounterGroups.get(groupName);
+        if (group == null) {
+            group = new SparkCounterGroup.LongSparkCounterGroup(groupName, groupName, javaSparkContext);
+            sparkCounterGroups.put(groupName, group);
+        }
+        return group;
+    }
+
+    public Map<String, SparkCounterGroup> getSparkCounterGroups() {
+        return sparkCounterGroups;
+    }
+
+
+    @Override
+    public boolean incrWarnCounter(Enum<?> name, Object incr) {
+        SparkCounter counter = getCounter(PigWarning.SPARK_WARN);
+        return _incrWarnCounter(counter, name.name(), (Long) incr);
+    }
+
+    @Override
+    public boolean incrWarnCounter(String group, String name, Object incr) {
+        SparkCounter counter = getCounter(PigWarning.SPARK_CUSTOM_WARN);
+        return _incrWarnCounter(counter, group+"::"+name, (Long) incr);
+    }
+
+    private static boolean _incrWarnCounter(SparkCounter counter, String name, Long incr) {
+        if (counter == null){
+            return false;
+        }
+        Map<String,Long> map = new HashMap<String,Long>();
+        map.put(name, incr);
+        counter.increment(map);
+        return true;
+    }
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkJobStats.java b/src/org/apache/pig/tools/pigstats/spark/SparkJobStats.java
new file mode 100644
index 0000000..c8cc031
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkJobStats.java
@@ -0,0 +1,349 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats.spark;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.pig.tools.pigstats.*;
+import scala.Option;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.Counters;
+import org.apache.pig.PigWarning;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.spark.JobMetricsListener;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.newplan.PlanVisitor;
+import org.apache.spark.executor.ShuffleReadMetrics;
+import org.apache.spark.executor.ShuffleWriteMetrics;
+import org.apache.spark.executor.TaskMetrics;
+
+import com.google.common.collect.Maps;
+
+public class SparkJobStats extends JobStats {
+
+    private int jobId;
+    private Map<String, Long> stats = Maps.newLinkedHashMap();
+    private boolean disableCounter;
+    private Counters counters = null;
+    public static String FS_COUNTER_GROUP = "FS_GROUP";
+    private Map<String, SparkCounter<Map<String, Long>>> warningCounters = null;
+
+    protected SparkJobStats(int jobId, PigStats.JobGraph plan, Configuration conf) {
+        this(String.valueOf(jobId), plan, conf);
+        this.jobId = jobId;
+    }
+
+    protected SparkJobStats(String jobId, PigStats.JobGraph plan, Configuration conf) {
+        super(jobId, plan);
+        setConf(conf);
+    }
+
+    public void setConf(Configuration conf) {
+        super.setConf(conf);
+        disableCounter = conf.getBoolean("pig.disable.counter", false);
+        initializeHadoopCounter();
+    }
+
+    public void addOutputInfo(POStore poStore, boolean success,
+                              JobMetricsListener jobMetricsListener) {
+        if (!poStore.isTmpStore()) {
+            long bytes = getOutputSize(poStore, conf);
+            long recordsCount = -1;
+            if (disableCounter == false) {
+                recordsCount = SparkStatsUtil.getRecordCount(poStore);
+            }
+            OutputStats outputStats = new OutputStats(poStore.getSFile().getFileName(),
+                    bytes, recordsCount, success);
+            outputStats.setPOStore(poStore);
+            outputStats.setConf(conf);
+
+            outputs.add(outputStats);
+        }
+    }
+
+    public void addInputStats(POLoad po, boolean success,
+                              boolean singleInput) {
+
+        long recordsCount = -1;
+        if (disableCounter == false) {
+            recordsCount = SparkStatsUtil.getRecordCount(po);
+        }
+        long bytesRead = -1;
+        if (singleInput && stats.get("BytesRead") != null) {
+            bytesRead = stats.get("BytesRead");
+        }
+        InputStats inputStats = new InputStats(po.getLFile().getFileName(),
+                bytesRead, recordsCount, success);
+        inputStats.setConf(conf);
+
+        inputs.add(inputStats);
+    }
+
+    public void collectStats(JobMetricsListener jobMetricsListener) {
+        if (jobMetricsListener != null) {
+            Map<String, List<TaskMetrics>> taskMetrics = jobMetricsListener.getJobMetric(jobId);
+            if (taskMetrics == null) {
+                throw new RuntimeException("No task metrics available for jobId " + jobId);
+            }
+            stats = combineTaskMetrics(taskMetrics);
+        }
+    }
+
+    public Map<String, Long> getStats() {
+        return stats;
+    }
+
+    private Map<String, Long> combineTaskMetrics(Map<String, List<TaskMetrics>> jobMetric) {
+        Map<String, Long> results = Maps.newLinkedHashMap();
+
+        long executorDeserializeTime = 0;
+        long executorRunTime = 0;
+        long resultSize = 0;
+        long jvmGCTime = 0;
+        long resultSerializationTime = 0;
+        long memoryBytesSpilled = 0;
+        long diskBytesSpilled = 0;
+        long bytesRead = 0;
+        long bytesWritten = 0;
+        long remoteBlocksFetched = 0;
+        long localBlocksFetched = 0;
+        long fetchWaitTime = 0;
+        long remoteBytesRead = 0;
+        long shuffleBytesWritten = 0;
+        long shuffleWriteTime = 0;
+        boolean inputMetricExist = false;
+        boolean outputMetricExist = false;
+        boolean shuffleReadMetricExist = false;
+        boolean shuffleWriteMetricExist = false;
+
+        for (List<TaskMetrics> stageMetric : jobMetric.values()) {
+            if (stageMetric != null) {
+                for (TaskMetrics taskMetrics : stageMetric) {
+                    if (taskMetrics != null) {
+                        executorDeserializeTime += taskMetrics.executorDeserializeTime();
+                        executorRunTime += taskMetrics.executorRunTime();
+                        resultSize += taskMetrics.resultSize();
+                        jvmGCTime += taskMetrics.jvmGCTime();
+                        resultSerializationTime += taskMetrics.resultSerializationTime();
+                        memoryBytesSpilled += taskMetrics.memoryBytesSpilled();
+                        diskBytesSpilled += taskMetrics.diskBytesSpilled();
+                        if (!taskMetrics.inputMetrics().isEmpty()) {
+                            inputMetricExist = true;
+                            bytesRead += taskMetrics.inputMetrics().get().bytesRead();
+                        }
+
+                        if (!taskMetrics.outputMetrics().isEmpty()) {
+                            outputMetricExist = true;
+                            bytesWritten += taskMetrics.outputMetrics().get().bytesWritten();
+                        }
+
+                        Option<ShuffleReadMetrics> shuffleReadMetricsOption = taskMetrics.shuffleReadMetrics();
+                        if (!shuffleReadMetricsOption.isEmpty()) {
+                            shuffleReadMetricExist = true;
+                            remoteBlocksFetched += shuffleReadMetricsOption.get().remoteBlocksFetched();
+                            localBlocksFetched += shuffleReadMetricsOption.get().localBlocksFetched();
+                            fetchWaitTime += shuffleReadMetricsOption.get().fetchWaitTime();
+                            remoteBytesRead += shuffleReadMetricsOption.get().remoteBytesRead();
+                        }
+
+                        Option<ShuffleWriteMetrics> shuffleWriteMetricsOption = taskMetrics.shuffleWriteMetrics();
+                        if (!shuffleWriteMetricsOption.isEmpty()) {
+                            shuffleWriteMetricExist = true;
+                            shuffleBytesWritten += shuffleWriteMetricsOption.get().shuffleBytesWritten();
+                            shuffleWriteTime += shuffleWriteMetricsOption.get().shuffleWriteTime();
+                        }
+
+                    }
+                }
+            }
+        }
+
+        results.put("EexcutorDeserializeTime", executorDeserializeTime);
+        results.put("ExecutorRunTime", executorRunTime);
+        results.put("ResultSize", resultSize);
+        results.put("JvmGCTime", jvmGCTime);
+        results.put("ResultSerializationTime", resultSerializationTime);
+        results.put("MemoryBytesSpilled", memoryBytesSpilled);
+        results.put("DiskBytesSpilled", diskBytesSpilled);
+        if (inputMetricExist) {
+            results.put("BytesRead", bytesRead);
+            hdfsBytesRead = bytesRead;
+            counters.incrCounter(FS_COUNTER_GROUP, PigStatsUtil.HDFS_BYTES_READ, hdfsBytesRead);
+        }
+
+        if (outputMetricExist) {
+            results.put("BytesWritten", bytesWritten);
+            hdfsBytesWritten = bytesWritten;
+            counters.incrCounter(FS_COUNTER_GROUP, PigStatsUtil.HDFS_BYTES_WRITTEN, hdfsBytesWritten);
+        }
+
+        if (shuffleReadMetricExist) {
+            results.put("RemoteBlocksFetched", remoteBlocksFetched);
+            results.put("LocalBlocksFetched", localBlocksFetched);
+            results.put("TotalBlocksFetched", localBlocksFetched + remoteBlocksFetched);
+            results.put("FetchWaitTime", fetchWaitTime);
+            results.put("RemoteBytesRead", remoteBytesRead);
+        }
+
+        if (shuffleWriteMetricExist) {
+            results.put("ShuffleBytesWritten", shuffleBytesWritten);
+            results.put("ShuffleWriteTime", shuffleWriteTime);
+        }
+
+        return results;
+    }
+
+    @Override
+    public String getJobId() {
+        return String.valueOf(jobId);
+    }
+
+    @Override
+    public void accept(PlanVisitor v) throws FrontendException {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public String getDisplayString() {
+        return null;
+    }
+
+    @Override
+    public int getNumberMaps() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getNumberReduces() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getMaxMapTime() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getMinMapTime() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getAvgMapTime() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getMaxReduceTime() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getMinReduceTime() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getAvgREduceTime() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getMapInputRecords() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getMapOutputRecords() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getReduceInputRecords() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getReduceOutputRecords() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getSMMSpillCount() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getProactiveSpillCountObjects() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getProactiveSpillCountRecs() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Counters getHadoopCounters() {
+        return counters;
+    }
+
+    @Override
+    public Map<String, Long> getMultiStoreCounters() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Map<String, Long> getMultiInputCounters() {
+        throw new UnsupportedOperationException();
+    }
+
+    public void setAlias(SparkOperator sparkOperator) {
+        SparkScriptState ss = (SparkScriptState) SparkScriptState.get();
+        SparkScriptState.SparkScriptInfo sparkScriptInfo = ss.getScriptInfo();
+        annotate(ALIAS, sparkScriptInfo.getAlias(sparkOperator));
+        annotate(ALIAS_LOCATION, sparkScriptInfo.getAliasLocation(sparkOperator));
+        annotate(FEATURE, sparkScriptInfo.getPigFeatures(sparkOperator));
+    }
+
+    private void initializeHadoopCounter() {
+        counters = new Counters();
+        Counters.Group fsGrp = counters.addGroup(FS_COUNTER_GROUP, FS_COUNTER_GROUP);
+        fsGrp.addCounter(PigStatsUtil.HDFS_BYTES_READ, PigStatsUtil.HDFS_BYTES_READ, 0);
+        fsGrp.addCounter(PigStatsUtil.HDFS_BYTES_WRITTEN, PigStatsUtil.HDFS_BYTES_WRITTEN, 0);
+    }
+
+
+    public Map<String, SparkCounter<Map<String, Long>>> getWarningCounters() {
+        return warningCounters;
+    }
+
+    public void initWarningCounters() {
+        SparkCounters counters = SparkPigStatusReporter.getInstance().getCounters();
+        SparkCounterGroup<Map<String, Long>> sparkCounterGroup = counters.getSparkCounterGroups().get(
+                PigWarning.class.getCanonicalName());
+        if (sparkCounterGroup != null) {
+            this.warningCounters = sparkCounterGroup.getSparkCounters();
+        }
+    }
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkPigStats.java b/src/org/apache/pig/tools/pigstats/spark/SparkPigStats.java
new file mode 100644
index 0000000..61ccbcc
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkPigStats.java
@@ -0,0 +1,252 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.tools.pigstats.spark;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapred.JobClient;
+import org.apache.pig.PigWarning;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.util.PlanHelper;
+import org.apache.pig.backend.hadoop.executionengine.spark.JobMetricsListener;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.plan.CompilationMessageCollector;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.tools.pigstats.InputStats;
+import org.apache.pig.tools.pigstats.JobStats;
+import org.apache.pig.tools.pigstats.PigStats;
+import org.apache.pig.tools.pigstats.ScriptState;
+import org.apache.spark.api.java.JavaSparkContext;
+
+public class SparkPigStats extends PigStats {
+
+    private Map<SparkJobStats,SparkOperator> jobSparkOperatorMap = new HashMap<SparkJobStats, SparkOperator>();
+    private static final Log LOG = LogFactory.getLog(SparkPigStats.class);
+
+    private Set<SparkOperator> sparkOperatorsSet = new HashSet<SparkOperator>();
+
+    private SparkScriptState sparkScriptState;
+
+    private Configuration conf;
+
+    public SparkPigStats() {
+        jobPlan = new JobGraph();
+        this.sparkScriptState = (SparkScriptState) ScriptState.get();
+    }
+
+    public void initialize(PigContext pigContext, SparkOperPlan sparkPlan, Configuration conf) {
+        super.start();
+        this.pigContext = pigContext;
+        this.conf = conf;
+        sparkScriptState.setScriptInfo(sparkPlan);
+    }
+
+    public void addJobStats(POStore poStore, SparkOperator sparkOperator, int jobId,
+                            JobMetricsListener jobMetricsListener,
+                            JavaSparkContext sparkContext) {
+        boolean isSuccess = SparkStatsUtil.isJobSuccess(jobId, sparkContext);
+        SparkJobStats jobStats = new SparkJobStats(jobId, jobPlan, conf);
+        jobStats.setSuccessful(isSuccess);
+        jobStats.collectStats(jobMetricsListener);
+        jobStats.addOutputInfo(poStore, isSuccess, jobMetricsListener);
+        addInputInfoForSparkOper(sparkOperator, jobStats, isSuccess, jobMetricsListener, conf);
+        jobStats.initWarningCounters();
+        jobSparkOperatorMap.put(jobStats, sparkOperator);
+
+        jobPlan.add(jobStats);
+    }
+
+
+    public void addFailJobStats(POStore poStore, SparkOperator sparkOperator, String jobId,
+                                JobMetricsListener jobMetricsListener,
+                                JavaSparkContext sparkContext,
+                                Exception e) {
+        boolean isSuccess = false;
+        SparkJobStats jobStats = new SparkJobStats(jobId, jobPlan, conf);
+        jobStats.setSuccessful(isSuccess);
+        jobStats.collectStats(jobMetricsListener);
+        jobStats.addOutputInfo(poStore, isSuccess, jobMetricsListener);
+        addInputInfoForSparkOper(sparkOperator, jobStats, isSuccess, jobMetricsListener, conf);
+        jobSparkOperatorMap.put(jobStats, sparkOperator);
+        jobPlan.add(jobStats);
+        jobStats.setBackendException(e);
+    }
+
+    public void addNativeJobStats(NativeSparkOperator sparkOperator, String jobId, boolean isSuccess, Exception e) {
+        SparkJobStats jobStats = new SparkJobStats(jobId, jobPlan, conf);
+        jobStats.setSuccessful(isSuccess);
+        jobSparkOperatorMap.put(jobStats, sparkOperator);
+        jobPlan.add(jobStats);
+        jobStats.setBackendException(e);
+    }
+
+    public void finish() {
+        super.stop();
+        display();
+    }
+
+    private void display() {
+        LOG.info(getDisplayString());
+        handleAggregateWarnings();
+    }
+
+    private void handleAggregateWarnings() {
+        Map<Enum, Long> warningAggMap = new HashMap<Enum, Long>();
+
+        Iterator<JobStats> iter = jobPlan.iterator();
+        while (iter.hasNext()) {
+            SparkJobStats js = (SparkJobStats) iter.next();
+            Map<String, SparkCounter<Map<String,Long>>> counterMap = js.getWarningCounters();
+            if (counterMap == null) {
+                continue;
+            }
+            Map<String, Long> warningCounters = counterMap.get(PigWarning.SPARK_WARN.name()).getValue();
+            if (warningCounters == null) {
+                continue;
+            }
+            for (String warnKey : warningCounters.keySet()) {
+                Long val = warningAggMap.get(warnKey);
+                if (val != null) {
+                    val += (Long)warningCounters.get(warnKey);
+                } else {
+                    val = (Long)warningCounters.get(warnKey);
+                }
+                warningAggMap.put(PigWarning.valueOf(warnKey), val);
+            }
+        }
+        CompilationMessageCollector.logAggregate(warningAggMap, CompilationMessageCollector.MessageType.Warning, LOG);
+    }
+
+    @Override
+    public String getDisplayString() {
+        StringBuilder sb = new StringBuilder();
+        Iterator<JobStats> iter = jobPlan.iterator();
+        while (iter.hasNext()) {
+            SparkJobStats js = (SparkJobStats)iter.next();
+            if (jobSparkOperatorMap.containsKey(js)) {
+                SparkOperator sparkOperator = jobSparkOperatorMap.get(js);
+                js.setAlias(sparkOperator);
+            }
+            sb.append("Spark Job [" + js.getJobId() + "] Metrics");
+            Map<String, Long> stats = js.getStats();
+            if (stats == null) {
+                sb.append("No statistics found for job " + js.getJobId());
+                return sb.toString();
+            }
+
+            Iterator statIt = stats.entrySet().iterator();
+            while (statIt.hasNext()) {
+                Map.Entry pairs = (Map.Entry)statIt.next();
+                sb.append("\t" + pairs.getKey() + " : " + pairs.getValue());
+            }
+            for (InputStats inputStat : js.getInputs()){
+                sb.append("\t"+inputStat.getDisplayString());
+            }
+        }
+        return sb.toString();
+    }
+
+    @Override
+    public JobClient getJobClient() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean isEmbedded() {
+        return false;
+    }
+
+    @Override
+    public Map<String, List<PigStats>> getAllStats() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public List<String> getAllErrorMessages() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getSMMSpillCount() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getProactiveSpillCountObjects() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public long getProactiveSpillCountRecords() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public int getNumberJobs() {
+        return jobPlan.size();
+    }
+
+    /**
+     * SparkPlan can have many SparkOperators.
+     * Each SparkOperator can have multiple POStores
+     * We currently collect stats once for every POStore,
+     * But do not want to collect input stats for every POStore
+     *
+     * e.g. After multiQuery optimization, the sparkOperator may look like this:
+     * POLoad_1             (PhysicalPlan) ...POStore_A
+     *         \          /
+     *          ...POSplit
+     *         /          \
+     * POLoad_2            (PhysicalPlan) ...POStore_B
+     */
+    private void addInputInfoForSparkOper(SparkOperator sparkOperator,
+                                          SparkJobStats jobStats,
+                                          boolean isSuccess,
+                                          JobMetricsListener jobMetricsListener,
+                                          Configuration conf) {
+        //to avoid repetition
+        if (sparkOperatorsSet.contains(sparkOperator)) {
+            return;
+        }
+
+        try {
+            List<POLoad> poLoads = PlanHelper.getPhysicalOperators(sparkOperator.physicalPlan, POLoad.class);
+            for (POLoad load : poLoads) {
+                if (!load.isTmpLoad()) {
+                    jobStats.addInputStats(load, isSuccess, (poLoads.size() == 1));
+                }
+            }
+        } catch (VisitorException ve) {
+            LOG.warn(ve);
+        }
+
+        sparkOperatorsSet.add(sparkOperator);
+    }
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkPigStatusReporter.java b/src/org/apache/pig/tools/pigstats/spark/SparkPigStatusReporter.java
new file mode 100644
index 0000000..d91f416
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkPigStatusReporter.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats.spark;
+
+import org.apache.pig.JVMReuseManager;
+import org.apache.pig.StaticDataCleanup;
+
+/**
+ * Just like PigStatusReporter which will create/reset Hadoop counters, SparkPigStatusReporter will
+ * create/reset Spark counters.
+ * Note that, it is not suitable to make SparkCounters as a Singleton, it will be created/reset for
+ * a given pig script or a Dump/Store action in Grunt mode.
+ */
+public class SparkPigStatusReporter {
+    private static SparkPigStatusReporter reporter;
+    private SparkCounters counters;
+
+    static {
+        JVMReuseManager.getInstance().registerForStaticDataCleanup(SparkPigStatusReporter.class);
+    }
+
+    @StaticDataCleanup
+    public static void staticDataCleanup() {
+        reporter = null;
+    }
+
+    private SparkPigStatusReporter() {
+    }
+
+    public static SparkPigStatusReporter getInstance() {
+        if (reporter == null) {
+            reporter = new SparkPigStatusReporter();
+        }
+        return reporter;
+    }
+
+    public void createCounter(String groupName, String counterName) {
+        if (counters != null) {
+            counters.createCounter(groupName, counterName);
+        }
+    }
+
+    public SparkCounters getCounters() {
+        return counters;
+    }
+
+    public void setCounters(SparkCounters counters) {
+        this.counters = counters;
+    }
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkScriptState.java b/src/org/apache/pig/tools/pigstats/spark/SparkScriptState.java
new file mode 100644
index 0000000..ccb1c9b
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkScriptState.java
@@ -0,0 +1,197 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.tools.pigstats.spark;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.pig.LoadFunc;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOpPlanVisitor;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.impl.plan.DependencyOrderWalker;
+import org.apache.pig.impl.plan.OperatorKey;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.tools.pigstats.ScriptState;
+
+import com.google.common.collect.Maps;
+
+/**
+ * ScriptStates encapsulates settings for a Pig script that runs on a hadoop
+ * cluster. These settings are added to all Spark jobs spawned by the script and
+ * in turn are persisted in the hadoop job xml. With the properties already in
+ * the job xml, users who want to know the relations between the script and Spark
+ * jobs can derive them from the job xmls.
+ */
+public class SparkScriptState extends ScriptState {
+    public SparkScriptState(String id) {
+        super(id);
+    }
+
+    private SparkScriptInfo scriptInfo = null;
+
+    public void setScriptInfo(SparkOperPlan plan) {
+        this.scriptInfo = new SparkScriptInfo(plan);
+    }
+
+    public SparkScriptInfo getScriptInfo() {
+        return scriptInfo;
+    }
+
+    public static class SparkScriptInfo {
+
+        private static final Log LOG = LogFactory.getLog(SparkScriptInfo.class);
+        private SparkOperPlan sparkPlan;
+        private String alias;
+        private String aliasLocation;
+        private String features;
+
+        private Map<OperatorKey, String> featuresMap = Maps.newHashMap();
+        private Map<OperatorKey, String> aliasMap = Maps.newHashMap();
+        private Map<OperatorKey, String> aliasLocationMap = Maps.newHashMap();
+
+        public SparkScriptInfo(SparkOperPlan sparkPlan) {
+            this.sparkPlan = sparkPlan;
+            initialize();
+        }
+
+        private void initialize() {
+            try {
+                new DAGAliasVisitor(sparkPlan).visit();
+            } catch (VisitorException e) {
+                LOG.warn("Cannot calculate alias information for DAG", e);
+            }
+        }
+
+        public String getAlias(SparkOperator sparkOp) {
+            return aliasMap.get(sparkOp.getOperatorKey());
+        }
+
+        public String getAliasLocation(SparkOperator sparkOp) {
+            return aliasLocationMap.get(sparkOp.getOperatorKey());
+        }
+
+        public String getPigFeatures(SparkOperator sparkOp) {
+            return featuresMap.get(sparkOp.getOperatorKey());
+        }
+
+        class DAGAliasVisitor extends SparkOpPlanVisitor {
+
+            private Set<String> aliases;
+            private Set<String> aliasLocations;
+            private BitSet featureSet;
+
+            public DAGAliasVisitor(SparkOperPlan plan) {
+                super(plan, new DependencyOrderWalker<SparkOperator, SparkOperPlan>(plan));
+                this.aliases = new HashSet<String>();
+                this.aliasLocations = new HashSet<String>();
+                this.featureSet = new BitSet();
+            }
+
+            @Override
+            public void visitSparkOp(SparkOperator sparkOp) throws VisitorException {
+
+                ArrayList<String> aliasList = new ArrayList<String>();
+                String aliasLocationStr = "";
+                try {
+                    ArrayList<String> aliasLocationList = new ArrayList<String>();
+                    new AliasVisitor(sparkOp.physicalPlan, aliasList, aliasLocationList).visit();
+                    aliasLocationStr += LoadFunc.join(aliasLocationList, ",");
+                    if (!aliasList.isEmpty()) {
+                        Collections.sort(aliasList);
+                        aliases.addAll(aliasList);
+                        aliasLocations.addAll(aliasLocationList);
+                    }
+                } catch (VisitorException e) {
+                    LOG.warn("unable to get alias", e);
+                }
+                aliasMap.put(sparkOp.getOperatorKey(), LoadFunc.join(aliasList, ","));
+                aliasLocationMap.put(sparkOp.getOperatorKey(), aliasLocationStr);
+
+
+                BitSet feature = new BitSet();
+                feature.clear();
+                if (sparkOp.isSampler()) {
+                    feature.set(PIG_FEATURE.SAMPLER.ordinal());
+                }
+                if (sparkOp.isIndexer()) {
+                    feature.set(PIG_FEATURE.INDEXER.ordinal());
+                }
+                if (sparkOp.isCogroup()) {
+                    feature.set(PIG_FEATURE.COGROUP.ordinal());
+                }
+                if (sparkOp.isGroupBy()) {
+                    feature.set(PIG_FEATURE.GROUP_BY.ordinal());
+                }
+                if (sparkOp.isRegularJoin()) {
+                    feature.set(PIG_FEATURE.HASH_JOIN.ordinal());
+                }
+                if (sparkOp.isUnion()) {
+                    feature.set(PIG_FEATURE.UNION.ordinal());
+                }
+                if (sparkOp.isNative()) {
+                    feature.set(PIG_FEATURE.NATIVE.ordinal());
+                }
+                if (sparkOp.isLimit() || sparkOp.isLimitAfterSort()) {
+                    feature.set(PIG_FEATURE.LIMIT.ordinal());
+                }
+                try {
+                    new FeatureVisitor(sparkOp.physicalPlan, feature).visit();
+                } catch (VisitorException e) {
+                    LOG.warn("Feature visitor failed", e);
+                }
+                StringBuilder sb = new StringBuilder();
+                for (int i = feature.nextSetBit(0); i >= 0; i = feature.nextSetBit(i + 1)) {
+                    if (sb.length() > 0) sb.append(",");
+                    sb.append(PIG_FEATURE.values()[i].name());
+                }
+                featuresMap.put(sparkOp.getOperatorKey(), sb.toString());
+                for (int i = 0; i < feature.length(); i++) {
+                    if (feature.get(i)) {
+                        featureSet.set(i);
+                    }
+                }
+            }
+
+            @Override
+            public void visit() throws VisitorException {
+                super.visit();
+                if (!aliases.isEmpty()) {
+                    ArrayList<String> aliasList = new ArrayList<String>(aliases);
+                    ArrayList<String> aliasLocationList = new ArrayList<String>(aliasLocations);
+                    Collections.sort(aliasList);
+                    Collections.sort(aliasLocationList);
+                    alias = LoadFunc.join(aliasList, ",");
+                    aliasLocation = LoadFunc.join(aliasLocationList, ",");
+                }
+                StringBuilder sb = new StringBuilder();
+                for (int i = featureSet.nextSetBit(0); i >= 0; i = featureSet.nextSetBit(i + 1)) {
+                    if (sb.length() > 0) sb.append(",");
+                    sb.append(PIG_FEATURE.values()[i].name());
+                }
+                features = sb.toString();
+            }
+        }
+    }
+}
diff --git a/src/org/apache/pig/tools/pigstats/spark/SparkStatsUtil.java b/src/org/apache/pig/tools/pigstats/spark/SparkStatsUtil.java
new file mode 100644
index 0000000..1541264
--- /dev/null
+++ b/src/org/apache/pig/tools/pigstats/spark/SparkStatsUtil.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+
+package org.apache.pig.tools.pigstats.spark;
+
+import java.util.List;
+
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.PhysicalOperator;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POLoad;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POSplit;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.relationalOperators.POStore;
+import org.apache.pig.backend.hadoop.executionengine.spark.JobGraphBuilder;
+import org.apache.pig.backend.hadoop.executionengine.spark.JobMetricsListener;
+import org.apache.pig.backend.hadoop.executionengine.spark.operator.NativeSparkOperator;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperator;
+import org.apache.pig.tools.pigstats.PigStatsUtil;
+import org.apache.pig.tools.pigstats.PigStats;
+import org.apache.spark.JobExecutionStatus;
+import org.apache.spark.SparkJobInfo;
+import org.apache.spark.api.java.JavaSparkContext;
+
+public class SparkStatsUtil {
+
+    public static final String SPARK_STORE_COUNTER_GROUP = PigStatsUtil.MULTI_STORE_COUNTER_GROUP;
+    public static final String SPARK_STORE_RECORD_COUNTER = PigStatsUtil.MULTI_STORE_RECORD_COUNTER;
+    public static final String SPARK_INPUT_COUNTER_GROUP = PigStatsUtil.MULTI_INPUTS_COUNTER_GROUP;
+    public static final String SPARK_INPUT_RECORD_COUNTER = PigStatsUtil.MULTI_INPUTS_RECORD_COUNTER;
+
+    public static void waitForJobAddStats(int jobID,
+                                          POStore poStore, SparkOperator sparkOperator,
+                                          JobMetricsListener jobMetricsListener,
+                                          JavaSparkContext sparkContext,
+                                          SparkPigStats sparkPigStats)
+            throws InterruptedException {
+        // Even though we are not making any async calls to spark,
+        // the SparkStatusTracker can still return RUNNING status
+        // for a finished job.
+        // Looks like there is a race condition between spark
+        // "event bus" thread updating it's internal listener and
+        // this driver thread calling SparkStatusTracker.
+        // To workaround this, we will wait for this job to "finish".
+        jobMetricsListener.waitForJobToEnd(jobID);
+        sparkPigStats.addJobStats(poStore, sparkOperator, jobID, jobMetricsListener,
+                sparkContext);
+        jobMetricsListener.cleanup(jobID);
+    }
+
+    public static void addFailJobStats(String jobID,
+                                       POStore poStore, SparkOperator sparkOperator,
+                                       SparkPigStats sparkPigStats,
+                                       Exception e) {
+        JobMetricsListener jobMetricsListener = null;
+        JavaSparkContext sparkContext = null;
+        sparkPigStats.addFailJobStats(poStore, sparkOperator, jobID, jobMetricsListener,
+                sparkContext, e);
+    }
+
+    public static String getCounterName(POStore store) {
+        String shortName = PigStatsUtil.getShortName(store.getSFile().getFileName());
+
+        StringBuffer sb = new StringBuffer(SPARK_STORE_RECORD_COUNTER);
+        sb.append("_");
+        sb.append(store.getIndex());
+        sb.append("_");
+        sb.append(store.getOperatorKey());
+        sb.append("_");
+        sb.append(shortName);
+        return sb.toString();
+    }
+
+    public static String getCounterName(POLoad load) {
+        String shortName = PigStatsUtil.getShortName(load.getLFile().getFileName());
+
+        StringBuffer sb = new StringBuffer(SPARK_INPUT_RECORD_COUNTER);
+        sb.append("_");
+        sb.append(load.getOperatorKey());
+        sb.append("_");
+        sb.append(shortName);
+        return sb.toString();
+    }
+
+    public static long getRecordCount(POStore store) {
+        SparkPigStatusReporter reporter = SparkPigStatusReporter.getInstance();
+        Object value = reporter.getCounters().getValue(SPARK_STORE_COUNTER_GROUP, getCounterName(store));
+        if (value == null) {
+            return 0L;
+        } else {
+            return (Long)value;
+        }
+    }
+
+    public static long getRecordCount(POLoad load) {
+        SparkPigStatusReporter reporter = SparkPigStatusReporter.getInstance();
+        int loadersCount = countCoLoadsIfInSplit(load,load.getParentPlan());
+        Object value = reporter.getCounters().getValue(SPARK_INPUT_COUNTER_GROUP, getCounterName(load));
+        if (value == null) {
+            return 0L;
+        } else {
+            return (Long)value/loadersCount;
+        }
+    }
+
+    private static int countCoLoadsIfInSplit(PhysicalOperator op, PhysicalPlan pp){
+        List<PhysicalOperator> successors = pp.getSuccessors(op);
+        if (successors == null || successors.size()==0) return 1;
+        for (PhysicalOperator successor : successors){
+            if (successor instanceof POSplit){
+                return ((POSplit)successor).getPlans().size();
+            }else{
+                return countCoLoadsIfInSplit(successor,pp);
+            }
+        }
+        return 1;
+    }
+
+    public static boolean isJobSuccess(int jobID,
+                                       JavaSparkContext sparkContext) {
+        if (jobID == JobGraphBuilder.NULLPART_JOB_ID) {
+            return true;
+        }
+        JobExecutionStatus status = getJobInfo(jobID, sparkContext).status();
+        if (status == JobExecutionStatus.SUCCEEDED) {
+            return true;
+        } else if (status != JobExecutionStatus.FAILED) {
+            throw new RuntimeException("Unexpected job execution status " +
+                    status);
+        }
+
+        return false;
+    }
+
+    private static SparkJobInfo getJobInfo(int jobID,
+                                           JavaSparkContext sparkContext) {
+        SparkJobInfo jobInfo = sparkContext.statusTracker().getJobInfo(jobID);
+        if (jobInfo == null) {
+            throw new RuntimeException("No jobInfo available for jobID "
+                    + jobID);
+        }
+
+        return jobInfo;
+    }
+
+    public static void addNativeJobStats(PigStats ps, NativeSparkOperator nativeSparkOperator) {
+        ((SparkPigStats) ps).addNativeJobStats(nativeSparkOperator, nativeSparkOperator.getJobId(), true, null);
+    }
+
+    public static void addFailedNativeJobStats(PigStats ps, NativeSparkOperator nativeSparkOperator, Exception e) {
+        ((SparkPigStats) ps).addNativeJobStats(nativeSparkOperator, nativeSparkOperator.getJobId(), false, e);
+    }
+}
\ No newline at end of file
diff --git a/test/e2e/pig/build.xml b/test/e2e/pig/build.xml
index 1ec9cf6..384639d 100644
--- a/test/e2e/pig/build.xml
+++ b/test/e2e/pig/build.xml
@@ -353,6 +353,12 @@
     </antcall>
   </target>
 
+  <target name="test-spark">
+    <antcall target="test-base">
+      <param name="harness.conf.file" value="${basedir}/conf/spark.conf"/>
+    </antcall>
+  </target>
+
   <target name="deploy-base" depends="property-check, tar, init-test">
     <exec executable="perl" dir="${test.location}" failonerror="true">
       <env key="HARNESS_ROOT" value="."/>
diff --git a/test/e2e/pig/conf/spark.conf b/test/e2e/pig/conf/spark.conf
new file mode 100644
index 0000000..d09c8fa
--- /dev/null
+++ b/test/e2e/pig/conf/spark.conf
@@ -0,0 +1,75 @@
+############################################################################
+#  Licensed to the Apache Software Foundation (ASF) under one or more
+#  contributor license agreements.  See the NOTICE file distributed with
+#  this work for additional information regarding copyright ownership.
+#  The ASF licenses this file to You 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.
+
+my $me = `whoami`;
+chomp $me;
+
+# The contents of this file can be rewritten to fit your installation.
+# Also, you can define the following environment variables and set things up as in the test setup
+# PH_ROOT           Root directory where test harness is installed
+# PH_LOCAL    	    Root directory for input and output for local mode tests
+# PH_OUT       	    Root directory where output data will be stored (on local disk, not HDFS)
+# PH_CLUSTER_BIN    Conf directory for cluster being used
+# HADOOP_CONF_DIR   Binary executable for cluster being used
+# PH_PIG      	    Root directory for Pig version being used
+
+my $hdfsBase = $ENV{PH_HDFS_BASE} || "/user/pig";
+
+$cfg = {
+    #HDFS
+      'inpathbase'     => "$hdfsBase/tests/data"
+    , 'outpathbase'    => "$hdfsBase/out"
+
+   #LOCAL
+    , 'localinpathbase'   => "$ENV{PH_LOCAL}/in"
+    , 'localoutpathbase'  => "$ENV{PH_LOCAL}/out/log"
+    , 'localxmlpathbase'  => "$ENV{PH_LOCAL}/out/xml"
+    , 'localpathbase'     => "$ENV{PH_LOCAL}/out/pigtest/$me"
+    , 'benchmarkcachepath'=> "$ENV{PH_BENCHMARK_CACHE_PATH}"
+
+    #TEST
+    , 'benchmarkPath'    => "$ENV{PH_OUT}/benchmarks"
+    , 'scriptPath'       => "$ENV{PH_ROOT}/libexec"
+    , 'tmpPath'          => '/tmp/pigtest'
+
+    #PIG
+    , 'testconfigpath'   => "$ENV{HADOOP_CONF_DIR}"
+    , 'funcjarPath'      => "$ENV{PH_ROOT}/lib/java"
+    , 'paramPath'        => "$ENV{PH_ROOT}/paramfiles"
+    , 'piggybankjarPath' => "$ENV{PH_PIGGYBANK_JAR}"
+    , 'pigpath'          => "$ENV{PH_PIG}"
+    , 'oldpigpath'       => "$ENV{PH_OLDPIG}"
+    , 'hcatbin'          => "$ENV{HCAT_BIN}"
+    , 'usePython'        => "$ENV{PIG_USE_PYTHON}"
+    , 'exectype'         => 'spark'
+    , 'benchmark_exectype'         => 'mapred'
+
+    #HADOOP
+    , 'mapredjars'       => "$ENV{PH_ROOT}/lib"
+
+    #HIVE
+    , 'hivelibdir'       => "$ENV{PH_HIVE_LIB_DIR}"
+    , 'hiveversion'      =>  "$ENV{PH_HIVE_VERSION}"
+    , 'hiveshimsversion' => "$ENV{PH_HIVE_SHIMS_VERSION}"
+
+    , 'userhomePath' => "$ENV{HOME}"
+    ,'local.bin'     => '/usr/bin'
+
+    ,'logDir'                => "$ENV{PH_OUT}/log"
+    ,'propertiesFile'     => "./conf/testpropertiesfile.conf"
+    ,'harness.console.level' => 'ERROR'
+
+};
diff --git a/test/e2e/pig/drivers/TestDriverPig.pm b/test/e2e/pig/drivers/TestDriverPig.pm
index bcec317..0fd0365 100644
--- a/test/e2e/pig/drivers/TestDriverPig.pm
+++ b/test/e2e/pig/drivers/TestDriverPig.pm
@@ -270,7 +270,7 @@
     my @cmd = @baseCmd;
 
     # Add option -l giving location for secondary logs
-    ##!!! Should that even be here? 
+    ##!!! Should that even be here?
     my $locallog = $testCmd->{'localpath'} . $testCmd->{'group'} . "_" . $testCmd->{'num'} . ".log";
     push(@cmd, "-logfile");
     push(@cmd, $locallog);
@@ -425,6 +425,11 @@
         }
         TestDriver::dbg("Additional java parameters: [$additionalJavaParams].\n");
     }
+
+    # Several OutOfMemoryErrors - Perm space issues were seen during running E2E tests, here max Perm size is adjusted
+    if ($testCmd->{'exectype'} eq "spark") {
+        $additionalJavaParams = "-XX:MaxPermSize=512m";
+    }
     
     push(@pigCmd, ("-x", $testCmd->{'exectype'}));
 
@@ -598,7 +603,7 @@
     if (defined $testCmd->{'floatpostprocess'} &&
             defined $testCmd->{'delimiter'}) {
         $fppCmd .= " | perl $toolpath/floatpostprocessor.pl \"" .
-            $testCmd->{'delimiter'} . "\"";
+            $testCmd->{'delimiter'} . "\" " . $testCmd->{'decimals'};
     }
     
     $fppCmd .= " > $localdir/out_original";
diff --git a/test/e2e/pig/tests/bigdata.conf b/test/e2e/pig/tests/bigdata.conf
index c51ae48..e678b90 100644
--- a/test/e2e/pig/tests/bigdata.conf
+++ b/test/e2e/pig/tests/bigdata.conf
@@ -24,7 +24,7 @@
 
 $cfg = {
 	'driver' => 'Pig',
-    'execonly' => 'mapred,tez',
+    'execonly' => 'mapred,tez,spark',
 
 	'groups' => [
 		{
diff --git a/test/e2e/pig/tests/cmdline.conf b/test/e2e/pig/tests/cmdline.conf
index d72c8c0..1459373 100644
--- a/test/e2e/pig/tests/cmdline.conf
+++ b/test/e2e/pig/tests/cmdline.conf
@@ -254,7 +254,7 @@
 		{
 		'name' => 'Warning',
 		'floatpostprocess' => 0,
-		'execonly' => 'mapred,tez', # Warnings use counters, which don't work in local mode
+		'execonly' => 'mapred,tez,spark', # Warnings use counters, which don't work in local mode
 		'delimiter' => '	',
 		'tests' => [
 		
diff --git a/test/e2e/pig/tests/grunt.conf b/test/e2e/pig/tests/grunt.conf
index 9c794d4..e081778 100644
--- a/test/e2e/pig/tests/grunt.conf
+++ b/test/e2e/pig/tests/grunt.conf
@@ -43,13 +43,13 @@
                       },{
                         'num' => 2,
                         'pig' => "pwd",
-                        'execonly' => 'mapred,tez', # don't have a clue what their cwd will be for local mode
+                        'execonly' => 'mapred,tez,spark', # don't have a clue what their cwd will be for local mode
                         'expected_out_regex' => "/user",
                         'rc' => 0
                       },{
                         'num' => 3,
                         'pig' => "ls .",
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'expected_out_regex' => "/user",
                         'rc' => 0
                       },{
diff --git a/test/e2e/pig/tests/hcat.conf b/test/e2e/pig/tests/hcat.conf
index 5910b2b..1294767 100644
--- a/test/e2e/pig/tests/hcat.conf
+++ b/test/e2e/pig/tests/hcat.conf
@@ -24,7 +24,7 @@
 
 $cfg = {
 	'driver' => 'Pig',
-    'execonly' => 'mapred,tez',
+    'execonly' => 'mapred,tez,spark',
 
 	'groups' => [
 		{
diff --git a/test/e2e/pig/tests/multiquery.conf b/test/e2e/pig/tests/multiquery.conf
index 667659a..ac927ef 100644
--- a/test/e2e/pig/tests/multiquery.conf
+++ b/test/e2e/pig/tests/multiquery.conf
@@ -529,7 +529,7 @@
             # Streaming in demux

             {

             'num' => 2,

-            'execonly' => 'mapred,tez',

+            'execonly' => 'mapred,tez,spark',

             'pig' => q#

                         define CMD `perl GroupBy.pl '\t' 0` ship(':SCRIPTHOMEPATH:/GroupBy.pl');

                         A = load ':INPATH:/singlefile/studenttab10k';

@@ -547,7 +547,7 @@
             # Streaming in nested demux

             {

             'num' => 3,

-            'execonly' => 'mapred,tez',

+            'execonly' => 'mapred,tez,spark',

             'pig' => q#

                         define CMD `perl GroupBy.pl '\t' 0` ship(':SCRIPTHOMEPATH:/GroupBy.pl');

                         A = load ':INPATH:/singlefile/studenttab10k';

diff --git a/test/e2e/pig/tests/negative.conf b/test/e2e/pig/tests/negative.conf
index 7b65e13..e5d00f5 100644
--- a/test/e2e/pig/tests/negative.conf
+++ b/test/e2e/pig/tests/negative.conf
@@ -312,7 +312,7 @@
                         {
 			# Define uses using non-existent command (autoship)
                         'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'java_params' => ['-Dopt.fetch=false'],
                         'pig' => q\
 define CMD `perl PigStreamingNotThere.pl`;
@@ -411,7 +411,7 @@
 			# Streaming application fails in the beginning of processing
 			# NEED TO CHECK STDERR MANUALLY FOR NOW
                         'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q\
 define CMD `perl PigStreamingBad.pl start` ship(':SCRIPTHOMEPATH:/PigStreamingBad.pl')  stderr('CMD' limit 1);
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -423,7 +423,7 @@
 			# Streaming application fails in the middle of processing
 			# NEED TO CHECK STDERR MANUALLY FOR NOW
                         'num' => 2,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q\
 define CMD `perl PigStreamingBad.pl middle` ship(':SCRIPTHOMEPATH:/PigStreamingBad.pl')  stderr('CMD' limit 1); 
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -436,7 +436,7 @@
 			# bring logs to dfs
 			# NEED TO CHECK STDERR MANUALLY FOR NOW
                         'num' => 3,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q\
 define CMD `perl PigStreamingBad.pl end` ship(':SCRIPTHOMEPATH:/PigStreamingBad.pl') stderr('CMD' limit 1);
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -449,7 +449,7 @@
 			# bring logs to dfs
 			# NEED TO CHECK STDERR MANUALLY FOR NOW
                         'num' => 4,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q\
 define CMD `perl DieRandomly.pl 10000 2` ship(':SCRIPTHOMEPATH:/DieRandomly.pl') stderr('CMD' limit 1);
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -480,7 +480,7 @@
                         {
                         # Invalid deserializer - throws exception
                         'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q\
 register :FUNCPATH:/testudf.jar;
 define CMD `perl PigStreaming.pl` input(stdin) output(stdout using org.apache.pig.test.udf.streaming.DumpStreamerBad) ship(':SCRIPTHOMEPATH:/PigStreaming.pl');
@@ -492,7 +492,7 @@
 			{
 			# Invalid serializer - throws exception
                         'num' => 2,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q\
 define CMD `perl PigStreamingDepend.pl` input(stdin using StringStoreBad) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
 A = load ':INPATH:/singlefile/studenttab10k';
diff --git a/test/e2e/pig/tests/nightly.conf b/test/e2e/pig/tests/nightly.conf
index 2194946..8c91b4e 100644
--- a/test/e2e/pig/tests/nightly.conf
+++ b/test/e2e/pig/tests/nightly.conf
@@ -573,6 +573,7 @@
 store c into ':OUTPATH:';\,
                         'floatpostprocess' => 1,
                         'delimiter' => '	',
+                        'decimals' => 6,
 			},
 			{
 			'num' => 10,
@@ -2202,6 +2203,7 @@
 			},
 			{
 				'num' => 4,
+				'execonly' => 'mapred,local,tez', # Spark doesn't do implicit ordering in distinct
 				'pig' =>q\a = load ':INPATH:/singlefile/studentnulltab10k';
 b = distinct a;
 c = limit b 100;
@@ -2292,7 +2294,7 @@
                         },
 			{
 				'num' => 12,
-				'execonly' => 'tez', #Limit_5 was not able to test on tez. 
+				'execonly' => 'tez,spark', #Limit_5 was not able to test on tez.
 				'pig' =>q\a = load ':INPATH:/singlefile/studenttab10k';
 b = load ':INPATH:/singlefile/studenttab10k';
 a1 = foreach a generate $0, $1;
@@ -2303,6 +2305,15 @@
 				'verify_pig_script' => q\a = load ':INPATH:/singlefile/studenttab10k' as (name:chararray, age:int);
 b = limit a 100;
 store b into ':OUTPATH:';\,
+			},
+			{
+				'num' => 13,
+				'execonly' => 'spark', # Limit_4 failed on Spark: distinct doesn't do implicit sort like it does in MR
+				'pig' =>q\a = load ':INPATH:/singlefile/studentnulltab10k';
+b = distinct a;
+c = order b by $0, $1, $2;
+d = limit c 100;
+store d into ':OUTPATH:';\,
 			}
 		]
 		},
@@ -2948,7 +2959,7 @@
 				# Merge-join with one file across multiple blocks
         	    {
                 'num' => 8,
-			    'execonly' => 'mapred,tez', # since this join will run out of memory in local mode
+			    'execonly' => 'mapred,tez,spark', # since this join will run out of memory in local mode
 		        'floatpostprocess' => 1,
 		        'delimiter' => '	',
                 'pig' => q\a = load ':INPATH:/singlefile/votertab10k';
@@ -3616,7 +3627,7 @@
             'tests' => [
                     {
                     'num' => 1,
-                    'execonly' => 'mapred,tez', # studenttab20m not available in local mode
+                    'execonly' => 'mapred,tez,spark', # studenttab20m not available in local mode
                     'pig' => q\
 a = load ':INPATH:/singlefile/studenttab20m' using PigStorage() as (name, age, gpa);
 b = foreach a generate age;
@@ -4361,7 +4372,7 @@
                     {
                     # test group
                     'num' => 1,
-                    'execonly' => 'mapred,tez', # since this join will run out of memory in local mode
+                    'execonly' => 'mapred,tez,spark', # since this join will run out of memory in local mode
                     'pig' => q\register :FUNCPATH:/testudf.jar;
 a = load ':INPATH:/singlefile/studenttab10k' using PigStorage() as (name, age:int, gpa);
 b = group a by age PARTITION BY org.apache.pig.test.utils.SimpleCustomPartitioner2 parallel 2;
@@ -4952,7 +4963,7 @@
                 {
                     'num' => 1,
                     'java_params' => ['-Dopt.fetch=false'],
-                    'execonly' => 'mapred,tez', # since distributed cache is not supported in local mode
+                    'execonly' => 'mapred,tez,spark', # since distributed cache is not supported in local mode
                     'pig' => q?
                         register :FUNCPATH:/testudf.jar;
                         define udfdc org.apache.pig.test.udf.evalfunc.Udfcachetest(':INPATH:/singlefile/votertab10k#foodle');
@@ -5194,7 +5205,7 @@
                     }, {
                         # PIG-2576
                         'num' => 4,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q?register :FUNCPATH:/testudf.jar;
                                 define printconf org.apache.pig.test.udf.evalfunc.UdfContextFrontend('dummy');
                                 a = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
@@ -5249,7 +5260,7 @@
                 ],
             },{
                 'name' => 'Bloom',
-			    'execonly' => 'mapred,tez', # distributed cache does not work in local mode
+			    'execonly' => 'mapred,tez', # distributed cache does not work in local mode, bloom is not implemented for Spark(PIG-5117)
                 'tests' => [
                     {
                         'num' => 1,
@@ -5694,7 +5705,7 @@
                 'tests' => [
                     {
                         'num' => 1,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q\
                                     SET default_parallel 7;
                                     SET mapreduce.input.fileinputformat.split.maxsize '300';
@@ -5711,7 +5722,7 @@
                                 \,
                     }, {
                         'num' =>2,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q\
                                     SET default_parallel 9;
                                     SET mapreduce.input.fileinputformat.split.maxsize '300';
@@ -5728,7 +5739,7 @@
                                 \,
                     }, {
                         'num' =>3,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q\
                                     SET default_parallel 7;
                                     SET mapreduce.input.fileinputformat.split.maxsize '300';
@@ -5745,7 +5756,7 @@
                                 \,
                     }, {
                         'num' => 4,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q\
                                     SET default_parallel 5;
                                     SET mapreduce.input.fileinputformat.split.maxsize '300';
@@ -5763,7 +5774,7 @@
                                 \,
                     }, {
                         'num' => 5,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q\
                                     SET default_parallel 5;
                                     SET mapreduce.input.fileinputformat.split.maxsize '300';
@@ -5786,7 +5797,7 @@
                                 \,
                     }, {
                         'num' => 6,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q\
                                     SET mapreduce.input.fileinputformat.split.maxsize '300';
                                     SET pig.splitCombination false;
@@ -5810,7 +5821,7 @@
                 'tests' => [
                     {
 						'num' => 1,
-						'execonly' => 'mapred,tez',
+						'execonly' => 'mapred,tez,spark',
 						'pig' => q\
                                                                         SET mapreduce.input.fileinputformat.split.maxsize '300';
                                                                         SET pig.splitCombination false;
@@ -5827,7 +5838,7 @@
 								\,
 					}, {
 						'num' => 2,
-						'execonly' => 'mapred,tez',
+						'execonly' => 'mapred,tez,spark',
 						'pig' => q\
                                                                         SET mapreduce.input.fileinputformat.split.maxsize '300';
                                                                         SET pig.splitCombination false;
@@ -5844,7 +5855,7 @@
 								\,
 					}, {
 						'num' => 3,
-						'execonly' => 'mapred,tez',
+						'execonly' => 'mapred,tez,spark',
 						'pig' => q\
                                                                         SET mapreduce.input.fileinputformat.split.maxsize '300';
                                                                         SET pig.splitCombination false;
@@ -5861,7 +5872,7 @@
 								\,
 					}, {
 						'num' => 4,
-						'execonly' => 'mapred,tez',
+						'execonly' => 'mapred,tez,spark',
 						'pig' => q\
                                                                         SET mapreduce.input.fileinputformat.split.maxsize '300';
                                                                         SET pig.splitCombination false;
@@ -5881,7 +5892,7 @@
 								\,
 					}, {
 						'num' => 5,
-						'execonly' => 'mapred,tez',
+						'execonly' => 'mapred,tez,spark',
 						'pig' => q\
 									SET default_parallel 9;
                                                                         SET mapreduce.input.fileinputformat.split.maxsize '300';
@@ -6009,8 +6020,9 @@
                         fs -rm :INPATH:/singlefile/names.txt#
                         },
                         {
-                # Custom Hive UDF and MapredContext
+                # Custom Hive UDF and MapredContext - disabled for Spark: see PIG-5234
                 'num' => 7,
+                'execonly' => 'mapred,tez',
                 'pig' => q\set mapred.max.split.size '100000000'
                         register :FUNCPATH:/testudf.jar;
                         define DummyContextUDF HiveUDF('org.apache.pig.test.udf.evalfunc.DummyContextUDF');
diff --git a/test/e2e/pig/tests/orc.conf b/test/e2e/pig/tests/orc.conf
index 9498d88..a023ea7 100644
--- a/test/e2e/pig/tests/orc.conf
+++ b/test/e2e/pig/tests/orc.conf
@@ -46,6 +46,7 @@
                         {
                         'num' => 2,
                         'notmq' => 1,
+                        'execonly' => 'mapred,tez',
                         'pig' => q\
 a = load ':INPATH:/singlefile/studentcomplextab10k' as (nameagegpamap:map[], nameagegpatuple:tuple(tname:chararray, tage:int, tgpa:float), nameagegpabag:bag{t:tuple(bname:chararray, bage:int, bgpa:float)});
 store a into ':OUTPATH:.intermediate' using OrcStorage();
@@ -113,6 +114,23 @@
                         'verify_pig_script' => q\a = load ':INPATH:/singlefile/studentnulltab10k' as (name:chararray, age);
 b = foreach a generate (name is null ? [] : TOMAP(name, age));
 store b into ':OUTPATH:';\,
+                        },
+# Test 6 : Running for Spark only as a replacement of Test 2: Spark and MR may produce different order of entries in
+# Pig maps, which although is fine, triggers a false failure during comparison
+                        {
+                          'num' => 6,
+                          'notmq' => 1,
+                          'execonly' => 'spark',
+                          'pig' => q\
+a = load ':INPATH:/singlefile/studentcomplextab10k' as (nameagegpamap:map[], nameagegpatuple:tuple(tname:chararray, tage:int, tgpa:float), nameagegpabag:bag{t:tuple(bname:chararray, bage:int, bgpa:float)});
+store a into ':OUTPATH:.intermediate' using OrcStorage();
+exec
+b = load ':OUTPATH:.intermediate' using OrcStorage();
+c = foreach b generate nameagegpamap#'name', nameagegpamap#'age', nameagegpamap#'gpa', nameagegpatuple, nameagegpabag;
+store c into ':OUTPATH:';\,
+                          'verify_pig_script' => q\a = load ':INPATH:/singlefile/studentcomplextab10k' as (nameagegpamap:map[], nameagegpatuple:tuple(tname:chararray, tage:int, tgpa:float), nameagegpabag:bag{t:tuple(bname:chararray, bage:int, bgpa:float)});
+b = foreach a generate nameagegpamap#'name', nameagegpamap#'age', nameagegpamap#'gpa', nameagegpatuple, nameagegpabag;
+store b into ':OUTPATH:';\,
                         }
                         ]
                 },
@@ -139,7 +157,7 @@
                         {
                         'num' => 2,
                         'notmq' => 1,
-                        'execonly' => 'mapred,tez', # studenttab20m not available in local mode
+                        'execonly' => 'mapred,tez,spark', # studenttab20m not available in local mode
                         'pig' => q\
 a = load ':INPATH:/singlefile/studenttab20m' as (name:chararray, age:int, gpa:float);
 b = order a by age desc parallel 4;
diff --git a/test/e2e/pig/tests/streaming.conf b/test/e2e/pig/tests/streaming.conf
index 18f2fb2..ed9d95f 100644
--- a/test/e2e/pig/tests/streaming.conf
+++ b/test/e2e/pig/tests/streaming.conf
@@ -79,7 +79,7 @@
 			{
 			#Section 1.1: perl script, no parameters, autoship(Section 2.1)
                         'num' => 4,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 A = load ':INPATH:/singlefile/studenttab10k';
 B = foreach A generate $0, $1, $2;
@@ -90,7 +90,7 @@
 			{
 			# Section 1.2: perl script that takes parameters; explicit ship of script (Section 2.1)
                         'num' => 5,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl - -` ship(':SCRIPTHOMEPATH:/PigStreaming.pl') stderr('CMD' limit 1);
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -102,7 +102,7 @@
 			{
 			# Section 1.3: define clause; explicit ship of script (Section 2.1)
                         'num' => 6,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl` ship(':SCRIPTHOMEPATH:/PigStreaming.pl') stderr('CMD');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -115,7 +115,7 @@
 			{
 			# Section 1.4: grouped data
                         'num' => 7,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl GroupBy.pl '\t' 0` ship(':SCRIPTHOMEPATH:/GroupBy.pl');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -128,7 +128,7 @@
 			{
 			# Section 1.4: grouped and ordered data
                         'num' => 8,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl GroupBy.pl '\t' 0 1` ship(':SCRIPTHOMEPATH:/GroupBy.pl');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -144,7 +144,7 @@
 			{
 			# Section 1.5: multiple streaming operators - adjacent - map side
                         'num' => 9,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreamingDepend.pl` input(stdin) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -157,7 +157,7 @@
 			{
 			# Section 1.5: multiple streaming operators - not adjacent - map side
                         'num' => 10,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 A = load ':INPATH:/singlefile/studenttab10k';
 define CMD `perl PigStreamingDepend.pl` input(stdin) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
@@ -172,7 +172,7 @@
 			{
 			# Section 1.5: multiple streaming operators - adjacent - reduce side
                         'num' => 11,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD1 `perl GroupBy.pl '\t' 0 1` ship(':SCRIPTHOMEPATH:/GroupBy.pl') stderr('CMD1');
 define CMD2 `perl PigStreamingDepend.pl` input(stdin) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm') stderr('CMD2');
@@ -191,7 +191,7 @@
 			# Section 1.5: multiple streaming operators - one on map and one on reduce side
 			# same alias name
                         'num' => 12,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD1 `perl GroupBy.pl '\t' 0` ship(':SCRIPTHOMEPATH:/GroupBy.pl');
 define CMD2 `perl PigStreamingDepend.pl` input(stdin) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
@@ -206,7 +206,7 @@
 			{
 			# Section 1.5: multiple streaming operators - adjacent - map side
                         'num' => 13,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreamingDepend.pl` input(stdin) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -232,7 +232,7 @@
                         # Section 2.1: perl script and its dependency shipped
 			# Also covers part of section 3.1: custom serializer
 			'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreamingDepend.pl` input(stdin) ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -243,7 +243,7 @@
 			{
 			# Section 2.1: perl script and supported data file is shipped
 			'num' => 2,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl - - nameMap` ship(':SCRIPTHOMEPATH:/PigStreaming.pl', ':SCRIPTHOMEPATH:/nameMap');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -257,7 +257,7 @@
 			{
 			# Section 2.2: script is shipped while the supporting file is cached
 			'num' => 3,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q@
 define CMD `perl PigStreaming.pl - - nameMap` ship(':SCRIPTHOMEPATH:/PigStreaming.pl') cache(':INPATH:/nameMap/part-00000#nameMap');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -283,7 +283,7 @@
 			{
 			# Section 3.1: use of custom deserializer
 			'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl` output(stdout) ship(':SCRIPTHOMEPATH:/PigStreaming.pl');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -294,7 +294,7 @@
 			{
 			# Section 3.1: use of custom serializer and deserializer
 			'num' => 2,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 register :FUNCPATH:/testudf.jar;
 define CMD `perl PigStreaming.pl` input(stdin using org.apache.pig.test.udf.streaming.StreamingDump) output(stdout using org.apache.pig.test.udf.streaming.DumpStreamer) ship(':SCRIPTHOMEPATH:/PigStreaming.pl');
@@ -307,7 +307,7 @@
 			{
 			# Section 3.3: streaming application reads from file rather than stdin
 			'num' => 3,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl foo -` input('foo') ship(':SCRIPTHOMEPATH:/PigStreaming.pl');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -318,7 +318,7 @@
 			{
 			# Section 3.4: streaming application writes single output to a file
 			'num' => 4,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl - foo nameMap` output('foo') ship(':SCRIPTHOMEPATH:/PigStreaming.pl', ':SCRIPTHOMEPATH:/nameMap');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -330,7 +330,7 @@
 			{
 			# Section 3.4: streaming application writes multiple outputs to file
 			'num' => 5,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreamingDepend.pl - sio_5_1 sio_5_2` input(stdin) output('sio_5_1', 'sio_5_2') ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -341,7 +341,7 @@
 			{
 			# Section 3.4: streaming application writes multiple outputs: 1 to file and 1 to stdout
 			'num' => 6,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreamingDepend.pl - - sio_5_2` input(stdin) output(stdout, 'sio_5_2') ship(':SCRIPTHOMEPATH:/PigStreamingDepend.pl', ':SCRIPTHOMEPATH:/PigStreamingModule.pm');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -362,7 +362,7 @@
 			{
 			# Section 4.3: integration with parameter substitition
 			'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig_params' => ['-p', qq(script_name='PigStreaming.pl')],
 			'pig' => q#
 define CMD `perl $script_name - - nameMap` ship(':SCRIPTHOMEPATH:/$script_name', ':SCRIPTHOMEPATH:/nameMap');
@@ -387,7 +387,7 @@
 			{
 			# Section 5.1: load/store optimization
 			'num' => 1,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD `perl PigStreaming.pl` ship(':SCRIPTHOMEPATH:/PigStreaming.pl') stderr('CMD');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -422,7 +422,7 @@
 			{
 			# PIG-272: problem with optimization and intermediate store
 			'num' => 3,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD1 `perl -ne 'print $_;print STDERR "stderr $_";'`; 
 define CMD2 `Split.pl 3` input(stdin using PigStreaming(',')) ship(':SCRIPTHOMEPATH:/Split.pl'); 
@@ -444,7 +444,7 @@
 			{
 			# PIG-272: problem with optimization and intermediate store
 			'num' => 4,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 define CMD1 `perl -ne 'print $_;'`; 
 define CMD2 `Split.pl 3` input(stdin using PigStreaming(',')) ship(':SCRIPTHOMEPATH:/Split.pl'); 
@@ -472,7 +472,7 @@
 			# Make sure join with stream optimization works
             # optimization only on load side
 			'num' => 5,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 A = load ':INPATH:/singlefile/studenttab10k';
 B = stream A through `cat` as (name:chararray, age:int, gpa:double);
@@ -485,7 +485,7 @@
 			# Make sure join with stream optimization works
             # optimization only on store side 
 			'num' => 6,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 A = load ':INPATH:/singlefile/studenttab10k';
 B = filter A by $1 > 25;
@@ -500,7 +500,7 @@
 			# Make sure join with stream optimization works
             # optimization on load and store
 			'num' => 7,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
 			'pig' => q#
 A = load ':INPATH:/singlefile/studenttab10k';
 B = stream A through `cat` as (name:chararray, age:int, gpa:double);
@@ -536,7 +536,7 @@
 		        # case where binary finishes normally
                 # BEFORE all input has been passed to it
                         'num' => 2,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q#
 define CMD `perl DieRandomly.pl 10000 0` ship(':SCRIPTHOMEPATH:/DieRandomly.pl');
 A = load ':INPATH:/singlefile/studenttab10k';
@@ -565,7 +565,7 @@
                 # BEFORE all input has been passed to it
                 # FIXME: in local mode
                         'num' => 4,
-			'execonly' => 'mapred,tez',
+			'execonly' => 'mapred,tez,spark',
                         'pig' => q#
 A = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
 B = stream A through `head -1` as (name, age, gpa);
@@ -581,7 +581,7 @@
                 # BEFORE all input has been passed to it
                 # and emits no output
                         'num' => 5,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q#
 define CMD `perl DieRandomly.pl 10000 0` ship(':SCRIPTHOMEPATH:/DieRandomly.pl');
 A = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
@@ -598,7 +598,7 @@
                 # BEFORE all input has been passed to it
                 # and emits no output
                         'num' => 6,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q#
 define CMD `perl DieRandomly.pl 10000 0` ship(':SCRIPTHOMEPATH:/DieRandomly.pl');
 A = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
@@ -613,7 +613,7 @@
                 # two stream operators one after another where first
                 # one emits no output
                         'num' => 7,
-                        'execonly' => 'mapred,tez',
+                        'execonly' => 'mapred,tez,spark',
                         'pig' => q#
 define CMD `perl DieRandomly.pl 10000 0` ship(':SCRIPTHOMEPATH:/DieRandomly.pl');
 A = load ':INPATH:/singlefile/studenttab10k' as (name, age, gpa);
diff --git a/test/e2e/pig/tests/turing_jython.conf b/test/e2e/pig/tests/turing_jython.conf
index 077231b..414c1de 100644
--- a/test/e2e/pig/tests/turing_jython.conf
+++ b/test/e2e/pig/tests/turing_jython.conf
@@ -452,9 +452,9 @@
                        ,'rc'=> 0
 
 	}, {
-                # illustrate() on a complex query 	 
+                # illustrate() on a complex query
 		'num' => 2
-		,'execonly' => 'mapred,local' #TODO: PIG-3993: Illustrate is yet to be implemented in Tez
+		,'execonly' => 'mapred,local' #TODO: PIG-3993,PIG-5204: Illustrate is yet to be implemented in Tez and in Spark
 		,'pig' => q\#!/usr/bin/python
 from org.apache.pig.scripting import Pig
 
diff --git a/test/e2e/pig/tools/test/floatpostprocessor.pl b/test/e2e/pig/tools/test/floatpostprocessor.pl
index d6631b2..7044842 100644
--- a/test/e2e/pig/tools/test/floatpostprocessor.pl
+++ b/test/e2e/pig/tools/test/floatpostprocessor.pl
@@ -26,6 +26,7 @@
 
 our @floats;
 our $delim;
+our $decimals;
 
 sub parseLine($)
 {
@@ -41,7 +42,7 @@
 	for (my $i = 0; $i < @fields; $i++) {
 		if ($i != 0) { print($delim); }
 		if ($floats[$i]) {
-			printf("%.3f", $fields[$i]);
+			printf("%." . $decimals . "f", $fields[$i]);
 		} else {
 			print($fields[$i]);
 		}
@@ -72,6 +73,10 @@
 	if (!defined($delim)) {
 		die "Usage: $0 delimiter\n";
 	}
+	$decimals = shift;
+	if (!defined($decimals)) {
+		$decimals = 3;
+	}
 
 	my @sampled;
     my $line;
diff --git a/test/excluded-tests-mr b/test/excluded-tests-mr
index f433e8d..5a65242 100644
--- a/test/excluded-tests-mr
+++ b/test/excluded-tests-mr
@@ -1 +1,2 @@
-**/tez/*.java
\ No newline at end of file
+**/tez/*.java
++**/spark/*.java
\ No newline at end of file
diff --git a/test/excluded-tests-spark b/test/excluded-tests-spark
new file mode 100644
index 0000000..38317fe
--- /dev/null
+++ b/test/excluded-tests-spark
@@ -0,0 +1,4 @@
+**/Test*MR.java
+**/tez/*.java
+**/TestNativeMapReduce.java
+**/TestCounters.java
diff --git a/test/excluded-tests-tez b/test/excluded-tests-tez
index 019f1c5..50b7540 100644
--- a/test/excluded-tests-tez
+++ b/test/excluded-tests-tez
@@ -1 +1,2 @@
-**/Test*MR.java
\ No newline at end of file
+**/Test*MR.java
++**/spark/*.java
\ No newline at end of file
diff --git a/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java b/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java
index 0e40b6b..ad6b216 100644
--- a/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java
+++ b/test/org/apache/pig/newplan/logical/relational/TestLocationInPhysicalPlan.java
@@ -62,6 +62,9 @@
         JobStats jStats = (JobStats)job.getStatistics().getJobGraph().getSinks().get(0);
         if (Util.getLocalTestMode().toString().equals("TEZ_LOCAL")) {
             Assert.assertEquals("A[1,4],A[3,4],B[2,4]", jStats.getAliasLocation());
+        } else if (Util.getLocalTestMode().toString().equals("SPARK_LOCAL")) {
+            //TODO PIG-5239:Investigate why there are duplicated A[3,4]
+            Assert.assertEquals("A[1,4],A[3,4],B[2,4],A[3,4]", jStats.getAliasLocation());
         } else {
             Assert.assertEquals("M: A[1,4],A[3,4],B[2,4] C: A[3,4],B[2,4] R: A[3,4]", jStats.getAliasLocation());
         }
diff --git a/test/org/apache/pig/pigunit/PigTest.java b/test/org/apache/pig/pigunit/PigTest.java
index f11c394..82b2c5c 100644
--- a/test/org/apache/pig/pigunit/PigTest.java
+++ b/test/org/apache/pig/pigunit/PigTest.java
@@ -142,6 +142,12 @@
           } else if (System.getProperties().getProperty(EXEC_CLUSTER).equalsIgnoreCase("tez_local")) {
             LOG.info("Using tez local mode");
             execType = ExecTypeProvider.fromString("tez_local");
+          } else if (System.getProperties().getProperty(EXEC_CLUSTER).equalsIgnoreCase("spark")) {
+              LOG.info("Using spark cluster mode");
+              execType = ExecTypeProvider.fromString("spark");
+          } else if (System.getProperties().getProperty(EXEC_CLUSTER).equalsIgnoreCase("spark_local")) {
+              LOG.info("Using spark local cluster mode");
+              execType = ExecTypeProvider.fromString("spark_local");
           } else {
             LOG.info("Using default local mode");
           }
diff --git a/test/org/apache/pig/spark/TestIndexedKey.java b/test/org/apache/pig/spark/TestIndexedKey.java
new file mode 100644
index 0000000..9c0638a
--- /dev/null
+++ b/test/org/apache/pig/spark/TestIndexedKey.java
@@ -0,0 +1,164 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.spark;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.JUnit4;
+
+import org.apache.pig.backend.hadoop.executionengine.spark.converter.IndexedKey;
+import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
+import static org.junit.Assert.assertEquals;
+
+@RunWith(JUnit4.class)
+public class TestIndexedKey {
+
+    /**Case1:Compare IndexedKeys with same index value
+     * key1    key2    equal?  hashCode1        hashCode2
+     * foo     null      N     hashCode(foo)    index
+     * null    foo       N     index            hashCode(foo)
+     * foo     foo       Y     hashCode(foo)    hashCode(foo)
+     * null    null      Y     index            index
+     * (1,1)   (1,1)     Y     hashCode((1,1))  hashCode((1,1))
+     * (1,)    (1,)      Y     hashCode((1,))   hashCode((1,))
+     * (1,1)   (1,2)     N     hashCode((1,1))  hashCode((1,2))
+     */
+    @Test
+    public void testIndexedKeyWithSameIndexValue() throws Exception {
+        IndexedKey a0 = new IndexedKey(new Byte("0"), "foo");
+        IndexedKey a1 = new IndexedKey(new Byte("0"), null);
+        assertEquals(a0.equals(a1), false);
+        assertEquals(a0.hashCode()==a1.hashCode(),false);
+
+        IndexedKey a2 = new IndexedKey(new Byte("0"), null);
+        IndexedKey a3 = new IndexedKey(new Byte("0"), "foo");
+        assertEquals(a2.equals(a3),false);
+        assertEquals(a2.hashCode()==a3.hashCode(),false);
+
+        IndexedKey a4 = new IndexedKey(new Byte("0"), "foo");
+        IndexedKey a5 = new IndexedKey(new Byte("0"), "foo");
+        assertEquals(a4.equals(a5),true);
+        assertEquals(a4.hashCode()==a5.hashCode(),true);
+
+        IndexedKey a6 = new IndexedKey(new Byte("0"), null);
+        IndexedKey a7 = new IndexedKey(new Byte("0"), null);
+        assertEquals(a6.equals(a7),true);
+        assertEquals(a6.hashCode()==a7.hashCode(),true);
+
+        Tuple t1 = TupleFactory.getInstance().newTuple(2);
+        t1.set(0,"1");
+        t1.set(1,"1");
+        Tuple t2 = TupleFactory.getInstance().newTuple(2);
+        t2.set(0,"1");
+        t2.set(1,"1");
+        IndexedKey a8 = new IndexedKey(new Byte("0"), t1);
+        IndexedKey a9 = new IndexedKey(new Byte("0"), t2);
+        assertEquals(a8.equals(a9),true);
+        assertEquals(a8.hashCode()==a9.hashCode(),true);
+
+        Tuple t3 = TupleFactory.getInstance().newTuple(2);
+        t3.set(0,"1");
+        t3.set(1,null);
+        Tuple t4 = TupleFactory.getInstance().newTuple(2);
+        t4.set(0,"1");
+        t4.set(1,null);
+        IndexedKey a10 = new IndexedKey(new Byte("0"), t3);
+        IndexedKey a11 = new IndexedKey(new Byte("0"), t4);
+        assertEquals(a10.equals(a11),true);
+        assertEquals(a10.hashCode()==a11.hashCode(),true);
+
+        Tuple t5 = TupleFactory.getInstance().newTuple(2);
+        t5.set(0,"1");
+        t5.set(1,"1");
+        Tuple t6 = TupleFactory.getInstance().newTuple(2);
+        t6.set(0,"1");
+        t6.set(1,"2");
+        IndexedKey a12 = new IndexedKey(new Byte("0"), t5);
+        IndexedKey a13 = new IndexedKey(new Byte("0"), t6);
+        assertEquals(a12.equals(a13),false);
+        assertEquals(a12.hashCode()==a13.hashCode(),false);
+    }
+
+    /*
+     * Case2:Compare IndexedKeys with different index value
+     * key1    key2    equal?  hashCode1        hashCode2
+     * foo     null     N      hashCode(foo)    index2
+     * null    foo      N      index1           hashCode(foo)
+     * foo     foo      Y      hashCode(foo)    hashCode(foo)
+     * null    null     N      index1           index2
+     * (1,1)   (1,1)    Y      hashCode((1,1))  hashCode((1,1))
+     * (1,)    (1,)     N      hashCode((1,))   hashCode((1,))
+     * (1,1)   (1,2)    N      hashCode((1,1))  hashCode((1,2))
+     */
+    @Test
+    public void testIndexedKeyWithDifferentIndexValue() throws Exception {
+        IndexedKey a0 = new IndexedKey(new Byte("0"), "foo");
+        IndexedKey a1 = new IndexedKey(new Byte("1"), null);
+        assertEquals(a0.equals(a1), false);
+        assertEquals(a0.hashCode() == a1.hashCode(), false);
+
+        IndexedKey a2 = new IndexedKey(new Byte("0"), null);
+        IndexedKey a3 = new IndexedKey(new Byte("1"), "foo");
+        assertEquals(a2.equals(a3), false);
+        assertEquals(a2.hashCode() == a3.hashCode(), false);
+
+        IndexedKey a4 = new IndexedKey(new Byte("0"), "foo");
+        IndexedKey a5 = new IndexedKey(new Byte("1"), "foo");
+        assertEquals(a4.equals(a5), true);
+        assertEquals(a4.hashCode() == a5.hashCode(), true);
+
+        IndexedKey a6 = new IndexedKey(new Byte("0"), null);
+        IndexedKey a7 = new IndexedKey(new Byte("1"), null);
+        assertEquals(a6.equals(a7), false);
+        assertEquals(a6.hashCode() == a7.hashCode(), false);
+
+        Tuple t1 = TupleFactory.getInstance().newTuple(2);
+        t1.set(0, "1");
+        t1.set(1, "1");
+        Tuple t2 = TupleFactory.getInstance().newTuple(2);
+        t2.set(0, "1");
+        t2.set(1, "1");
+        IndexedKey a8 = new IndexedKey(new Byte("0"), t1);
+        IndexedKey a9 = new IndexedKey(new Byte("1"), t2);
+        assertEquals(a8.equals(a9), true);
+        assertEquals(a8.hashCode() == a9.hashCode(), true);
+
+        Tuple t3 = TupleFactory.getInstance().newTuple(2);
+        t3.set(0, "1");
+        t3.set(1, null);
+        Tuple t4 = TupleFactory.getInstance().newTuple(2);
+        t4.set(0, "1");
+        t4.set(1, null);
+        IndexedKey a10 = new IndexedKey(new Byte("0"), t3);
+        IndexedKey a11 = new IndexedKey(new Byte("1"), t4);
+        assertEquals(a10.equals(a11), false);
+        assertEquals(a10.hashCode() == a11.hashCode(), true); //hashcode of a10 and a11 are equal but they are not equal
+
+        Tuple t5 = TupleFactory.getInstance().newTuple(2);
+        t5.set(0, "1");
+        t5.set(1, "1");
+        Tuple t6 = TupleFactory.getInstance().newTuple(2);
+        t6.set(0, "1");
+        t6.set(1, "2");
+        IndexedKey a12 = new IndexedKey(new Byte("0"), t5);
+        IndexedKey a13 = new IndexedKey(new Byte("1"), t6);
+        assertEquals(a12.equals(a13), false);
+        assertEquals(a12.hashCode() == a13.hashCode(), false);
+    }
+}
diff --git a/test/org/apache/pig/spark/TestSecondarySortSpark.java b/test/org/apache/pig/spark/TestSecondarySortSpark.java
new file mode 100644
index 0000000..2bc2f14
--- /dev/null
+++ b/test/org/apache/pig/spark/TestSecondarySortSpark.java
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.spark;
+
+import org.apache.pig.backend.hadoop.executionengine.optimizer.SecondaryKeyOptimizer;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.optimizer.SecondaryKeyOptimizerSpark;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkCompiler;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.test.MiniGenericCluster;
+import org.apache.pig.test.TestSecondarySort;
+import org.apache.pig.test.Util;
+
+/**
+ * TestSecondarySortSpark.
+ */
+public class TestSecondarySortSpark extends TestSecondarySort {
+
+    public TestSecondarySortSpark() {
+        super();
+    }
+
+    @Override
+    public MiniGenericCluster getCluster() {
+        return MiniGenericCluster.buildCluster(MiniGenericCluster.EXECTYPE_SPARK);
+    }
+
+    @Override
+    public SecondaryKeyOptimizer visitSecondaryKeyOptimizer(String query) throws Exception {
+        PhysicalPlan pp = Util.buildPp(pigServer, query);
+        SparkCompiler comp = new SparkCompiler(pp, pc);
+        comp.compile();
+        SparkOperPlan sparkPlan = comp.getSparkPlan();
+        SecondaryKeyOptimizerSpark optimizer = new SecondaryKeyOptimizerSpark(sparkPlan);
+        optimizer.visit();
+        return optimizer;
+    }
+}
diff --git a/test/org/apache/pig/spark/TestSparkCompiler.java b/test/org/apache/pig/spark/TestSparkCompiler.java
new file mode 100644
index 0000000..15f77f6
--- /dev/null
+++ b/test/org/apache/pig/spark/TestSparkCompiler.java
@@ -0,0 +1,182 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.spark;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.PrintStream;
+import java.util.Properties;
+import java.util.Random;
+
+import javax.xml.parsers.ParserConfigurationException;
+import javax.xml.transform.TransformerException;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.pig.PigServer;
+import org.apache.pig.backend.executionengine.ExecException;
+import org.apache.pig.backend.hadoop.executionengine.physicalLayer.plans.PhysicalPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkLocalExecType;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.DotSparkPrinter;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkOperPlan;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.SparkPrinter;
+import org.apache.pig.backend.hadoop.executionengine.spark.plan.XMLSparkPrinter;
+import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.io.FileLocalizer;
+import org.apache.pig.impl.plan.NodeIdGenerator;
+import org.apache.pig.impl.plan.VisitorException;
+import org.apache.pig.test.Util;
+import org.apache.pig.test.utils.TestHelper;
+
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Test cases to test the SparkCompiler. VERY IMPORTANT NOTE: The tests here
+ * compare results with a "golden" set of outputs. In each test case here, the
+ * operators generated have a random operator key which uses Java's Random
+ * class. So if there is a code change which changes the number of operators
+ * created in a plan, then  the "golden" file for that test case
+ * need to be changed.
+ */
+
+public class TestSparkCompiler {
+    private static PigContext pc;
+    private static PigServer pigServer;
+    private static final int MAX_SIZE = 100000;
+
+    private enum PlanPrinter {
+        TEXT,
+        DOT,
+        XML;
+
+        public void doPrint(PrintStream ps, SparkOperPlan plan) throws VisitorException, ParserConfigurationException, TransformerException {
+            switch (this) {
+                case DOT:
+                    throw new RuntimeException("Testing in DOT format not supported yet");
+                    //(new DotSparkPrinter(plan, ps)).dump();
+                    //break;
+                case XML:
+                    XMLSparkPrinter printer = new XMLSparkPrinter(ps, plan);
+                    printer.visit();
+                    printer.closePlan();
+                    break;
+                case TEXT:
+                default:
+                    (new SparkPrinter(ps, plan)).visit();
+                    break;
+            }
+        }
+    }
+
+    // If for some reason, the golden files need to be regenerated, set this to
+    // true - THIS WILL OVERWRITE THE GOLDEN FILES - So use this with caution
+    // and only for the test cases you need and are sure of.
+    private boolean generate = false;
+
+    @BeforeClass
+    public static void setUpBeforeClass() throws Exception {
+        resetFileLocalizer();
+        pc = new PigContext(new SparkLocalExecType(), new Properties());
+        FileUtils.deleteDirectory(new File("/tmp/pigoutput"));
+    }
+
+    @AfterClass
+    public static void tearDownAfterClass() throws Exception {
+        resetFileLocalizer();
+    }
+
+    @Before
+    public void setUp() throws ExecException {
+        resetScope();
+        pigServer = new PigServer(pc);
+    }
+
+    private void resetScope() {
+        NodeIdGenerator.reset();
+        PigServer.resetScope();
+    }
+
+    private static void resetFileLocalizer() {
+        FileLocalizer.deleteTempFiles();
+        FileLocalizer.setInitialized(false);
+        // Set random seed to generate deterministic temporary paths
+        FileLocalizer.setR(new Random(1331L));
+    }
+
+    @Test
+    public void testStoreLoad() throws Exception {
+        String query =
+                "a = load 'file:///tmp/input' as (x:int, y:int);" +
+                "store a into 'file:///tmp/pigoutput';" +
+                "b = load 'file:///tmp/pigoutput' as (x:int, y:int);" +
+                "store b into 'file:///tmp/pigoutput1';";
+
+        run(query, "test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-text.gld", PlanPrinter.TEXT);
+        run(query, "test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-xml.gld", PlanPrinter.XML);
+        //TODO: enable this when DOT file comparison is supported
+        //run(query, "test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-dot.gld", PlanPrinter.DOT);
+    }
+
+    private void run(String query, String expectedFile, PlanPrinter planPrinter) throws Exception {
+        PhysicalPlan pp = Util.buildPp(pigServer, query);
+        SparkLauncher launcher = new SparkLauncher();
+        pc.inExplain = true;
+        SparkOperPlan sparkOperPlan = launcher.compile(pp, pc);
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        PrintStream ps = new PrintStream(baos);
+        planPrinter.doPrint(ps, sparkOperPlan);
+        String compiledPlan = baos.toString();
+        System.out.println();
+        System.out.println("<<<" + compiledPlan + ">>>");
+
+        if (generate) {
+            FileOutputStream fos = new FileOutputStream(expectedFile);
+            fos.write(baos.toByteArray());
+            fos.close();
+            return;
+        }
+        FileInputStream fis = new FileInputStream(expectedFile);
+        byte[] b = new byte[MAX_SIZE];
+        int len = fis.read(b);
+        fis.close();
+        String goldenPlan = new String(b, 0, len);
+        if (goldenPlan.charAt(len-1) == '\n') {
+            goldenPlan = goldenPlan.substring(0, len-1);
+        }
+
+        System.out.println("-------------");
+        System.out.println("Golden");
+        System.out.println("<<<" + goldenPlan + ">>>");
+        System.out.println("-------------");
+
+        String goldenPlanClean = Util.standardizeNewline(goldenPlan).trim();
+        String compiledPlanClean = Util.standardizeNewline(compiledPlan).trim();
+        assertEquals(TestHelper.sortUDFs(Util.removeSignature(goldenPlanClean)),
+                TestHelper.sortUDFs(Util.removeSignature(compiledPlanClean)));
+    }
+
+}
+
diff --git a/test/org/apache/pig/test/MiniGenericCluster.java b/test/org/apache/pig/test/MiniGenericCluster.java
index 9347269..674860f 100644
--- a/test/org/apache/pig/test/MiniGenericCluster.java
+++ b/test/org/apache/pig/test/MiniGenericCluster.java
@@ -48,6 +48,7 @@
 
     public static String EXECTYPE_MR = "mr";
     public static String EXECTYPE_TEZ = "tez";
+    public static String EXECTYPE_SPARK = "spark";
 
     /**
      * Returns the single instance of class MiniGenericCluster that represents
@@ -75,6 +76,8 @@
                 INSTANCE = new MiniCluster();
             } else if (execType.equalsIgnoreCase(EXECTYPE_TEZ)) {
                 INSTANCE = new TezMiniCluster();
+            } else if (execType.equalsIgnoreCase(EXECTYPE_SPARK)) {
+                INSTANCE = new SparkMiniCluster();
             } else {
                 throw new RuntimeException("Unknown test.exec.type: " + execType);
             }
@@ -157,7 +160,9 @@
             return MiniCluster.getLauncher();
         } else if (execType.equalsIgnoreCase(EXECTYPE_TEZ)) {
             return TezMiniCluster.getLauncher();
-        } else {
+        } else if(execType.equalsIgnoreCase(EXECTYPE_SPARK)){
+            return SparkMiniCluster.getLauncher();
+        } else{
             throw new RuntimeException("Unknown test.exec.type: " + execType);
         }
     }
diff --git a/test/org/apache/pig/test/SparkMiniCluster.java b/test/org/apache/pig/test/SparkMiniCluster.java
new file mode 100644
index 0000000..27f7f8c
--- /dev/null
+++ b/test/org/apache/pig/test/SparkMiniCluster.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.pig.ExecType;
+import org.apache.pig.backend.hadoop.executionengine.Launcher;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkExecType;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkLauncher;
+
+public class SparkMiniCluster extends YarnMiniCluster {
+
+    private static final Log LOG = LogFactory
+            .getLog(SparkMiniCluster.class);
+    private ExecType spark = new SparkExecType();
+
+    @Override
+    public ExecType getExecType() {
+        return spark;
+    }
+
+    static public Launcher getLauncher() {
+        return new SparkLauncher();
+    }
+}
diff --git a/test/org/apache/pig/test/TestAssert.java b/test/org/apache/pig/test/TestAssert.java
index 458c0fd..3eef71e 100644
--- a/test/org/apache/pig/test/TestAssert.java
+++ b/test/org/apache/pig/test/TestAssert.java
@@ -116,7 +116,8 @@
       try {
           pigServer.openIterator("A");
       } catch (FrontendException fe) {
-          if (pigServer.getPigContext().getExecType().toString().startsWith("TEZ")) {
+          if (pigServer.getPigContext().getExecType().toString().startsWith("TEZ")
+                  || pigServer.getPigContext().getExecType().toString().startsWith("SPARK")) {
               Assert.assertTrue(fe.getCause().getMessage().contains(
                       "Assertion violated: i should be greater than 1"));
           } else {
@@ -147,7 +148,8 @@
       try {
           pigServer.openIterator("A");
       } catch (FrontendException fe) {
-          if (pigServer.getPigContext().getExecType().toString().startsWith("TEZ")) {
+          if (pigServer.getPigContext().getExecType().toString().startsWith("TEZ")
+                  || pigServer.getPigContext().getExecType().toString().startsWith("SPARK")) {
               Assert.assertTrue(fe.getCause().getMessage().contains(
                       "Assertion violated: i should be greater than 1"));
           } else {
diff --git a/test/org/apache/pig/test/TestCase.java b/test/org/apache/pig/test/TestCase.java
index 6bf684a..63b2144 100644
--- a/test/org/apache/pig/test/TestCase.java
+++ b/test/org/apache/pig/test/TestCase.java
@@ -29,6 +29,7 @@
 import org.apache.pig.builtin.mock.Storage.Data;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.logicalLayer.FrontendException;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.junit.Test;
 
 public class TestCase {
@@ -267,10 +268,15 @@
         pigServer.registerQuery("STORE C INTO 'bar' USING mock.Storage();");
 
         List<Tuple> out = data.get("bar");
-        assertEquals(3, out.size());
-        assertEquals(tuple(1, "3n+1", bag(tuple("a","x"), tuple("a","y"))), out.get(0));
-        assertEquals(tuple(2, "3n+2", bag(tuple("b","x"), tuple("b","y"))), out.get(1));
-        assertEquals(tuple(3, "3n",   bag(tuple("c","x"), tuple("c","y"))), out.get(2));
+
+        String[] expected = new String[] {
+                "(1,3n+1,{(a,x),(a,y)})",
+                "(2,3n+2,{(b,x),(b,y)})",
+                "(3,3n,{(c,x),(c,y)})"
+        };
+        Schema s = pigServer.dumpSchema("C");
+        Util.checkQueryOutputs(out.iterator(), expected, org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(Util.getLocalTestMode()));
     }
 
     /**
diff --git a/test/org/apache/pig/test/TestCollectedGroup.java b/test/org/apache/pig/test/TestCollectedGroup.java
index a958d33..f8b9683 100644
--- a/test/org/apache/pig/test/TestCollectedGroup.java
+++ b/test/org/apache/pig/test/TestCollectedGroup.java
@@ -25,7 +25,6 @@
 import java.util.List;
 
 import org.apache.pig.CollectableLoadFunc;
-import org.apache.pig.ExecType;
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRCompilerException;
@@ -296,13 +295,13 @@
     @Test
     public void testMapsideGroupWithMergeJoin() throws IOException{
         pigServer = new PigServer(cluster.getExecType(), cluster.getProperties());
-        pigServer.registerQuery("A = LOAD '" + INPUT_FILE + "' using "+DummyCollectableLoader.class.getName() +"() as (id, name, grade);");
-        pigServer.registerQuery("B = LOAD '" + INPUT_FILE + "' using "+DummyCollectableLoader.class.getName() +"() as (id, name, grade);");
+        pigServer.registerQuery("A = LOAD '" + INPUT_FILE + "' using " + DummyCollectableLoader.class.getName() + "() as (id, name, grade);");
+        pigServer.registerQuery("B = LOAD '" + INPUT_FILE + "' using " + DummyCollectableLoader.class.getName() + "() as (id, name, grade);");
         try {
             DataBag dbfrj = BagFactory.getInstance().newDefaultBag();
             DataBag dbshj = BagFactory.getInstance().newDefaultBag();
             {
-            	pigServer.registerQuery("C = join A by id, B by id using 'merge';");
+                pigServer.registerQuery("C = join A by id, B by id using 'merge';");
                 pigServer.registerQuery("D = group C by A::id using 'collected';");
                 pigServer.registerQuery("E = foreach D generate group, COUNT(C);");
                 Iterator<Tuple> iter = pigServer.openIterator("E");
@@ -312,7 +311,7 @@
                 }
             }
             {
-            	pigServer.registerQuery("F = join A by id, B by id;");
+                pigServer.registerQuery("F = join A by id, B by id;");
                 pigServer.registerQuery("G = group F by A::id;");
                 pigServer.registerQuery("H = foreach G generate group, COUNT(F);");
                 Iterator<Tuple> iter = pigServer.openIterator("H");
@@ -321,7 +320,7 @@
                     dbshj.add(iter.next());
                 }
             }
-            Assert.assertTrue(dbfrj.size()>0 && dbshj.size()>0);
+            Assert.assertTrue(dbfrj.size() > 0 && dbshj.size() > 0);
             Assert.assertEquals(true, TestHelper.compareBags(dbfrj, dbshj));
 
         } catch (Exception e) {
@@ -337,4 +336,4 @@
         }
 
     }
-}
+}
\ No newline at end of file
diff --git a/test/org/apache/pig/test/TestCombiner.java b/test/org/apache/pig/test/TestCombiner.java
index df44293..a2c0f7a 100644
--- a/test/org/apache/pig/test/TestCombiner.java
+++ b/test/org/apache/pig/test/TestCombiner.java
@@ -116,13 +116,17 @@
         inputLines.add("a,c,1");
         String inputFileName = loadWithTestLoadFunc("A", pig, inputLines);
 
-        pig.registerQuery("B = group A by ($0, $1);");
-        pig.registerQuery("C = foreach B generate flatten(group), COUNT($1);");
-        Iterator<Tuple> resultIterator = pig.openIterator("C");
-        Tuple tuple = resultIterator.next();
-        assertEquals("(a,b,2)", tuple.toString());
-        tuple = resultIterator.next();
-        assertEquals("(a,c,1)", tuple.toString());
+        pig.registerQuery("B = foreach A generate $0 as (c0:chararray), $1 as (c1:chararray), $2 as (c2:int);");
+        pig.registerQuery("C = group B by ($0, $1);");
+        pig.registerQuery("D = foreach C generate flatten(group), COUNT($1) as int;");
+        // Since the input has no schema, using Util.getTuplesFromConstantTupleStrings fails assert.
+        List<Tuple> resultTuples = Util.getTuplesFromConstantTupleStrings(
+            new String[]{
+                "('a','b',2)",
+                "('a','c',1)",
+            });
+        Iterator<Tuple> resultIterator = pig.openIterator("D");
+        Util.checkQueryOutputsAfterSort(resultIterator, resultTuples);
 
         return inputFileName;
     }
@@ -185,7 +189,7 @@
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
         pigServer.explain("c", ps);
-        assertTrue(baos.toString().matches("(?si).*combine plan.*"));
+        checkCombinerUsed(pigServer, "c", true);
 
         Iterator<Tuple> it = pigServer.openIterator("c");
         Tuple t = it.next();
@@ -235,7 +239,7 @@
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
         pigServer.explain("c", ps);
-        assertTrue(baos.toString().matches("(?si).*combine plan.*"));
+        checkCombinerUsed(pigServer, "c", true);
 
         HashMap<String, Object[]> results = new HashMap<String, Object[]>();
         results.put("pig1", new Object[] { "pig1", 3L, 57L, 5.2, 75L, 9.4, 3L, 3L, 57L });
@@ -256,6 +260,56 @@
     }
 
     @Test
+    public void testGroupAndUnion() throws Exception {
+        // test use of combiner when group elements are accessed in the foreach
+        String input1[] = {
+                "ABC\t1\ta\t1",
+                "ABC\t1\tb\t2",
+                "ABC\t1\ta\t3",
+                "ABC\t2\tb\t4",
+        };
+
+        Util.createInputFile(cluster, "testGroupElements1.txt", input1);
+        PigServer pigServer = new PigServer(cluster.getExecType(), properties);
+        pigServer.debugOn();
+        pigServer.registerQuery("a1 = load 'testGroupElements1.txt' " +
+                "as (str:chararray, num1:int, alph : chararray, num2 : int);");
+        pigServer.registerQuery("b1 = group a1 by str;");
+
+        // check if combiner is present or not for various forms of foreach
+        pigServer.registerQuery("c1 = foreach b1  generate flatten(group), COUNT(a1.alph), SUM(a1.num2); ");
+
+        String input2[] = {
+                "DEF\t2\ta\t3",
+                "DEF\t2\td\t5",
+        };
+
+        Util.createInputFile(cluster, "testGroupElements2.txt", input2);
+        pigServer.registerQuery("a2 = load 'testGroupElements2.txt' " +
+                "as (str:chararray, num1:int, alph : chararray, num2 : int);");
+        pigServer.registerQuery("b2 = group a2 by str;");
+
+        // check if combiner is present or not for various forms of foreach
+        pigServer.registerQuery("c2 = foreach b2  generate flatten(group), COUNT(a2.alph), SUM(a2.num2); ");
+
+        pigServer.registerQuery("c = union c1, c2;");
+
+        List<Tuple> expectedRes =
+                Util.getTuplesFromConstantTupleStrings(
+                        new String[]{
+                                "('ABC',4L,10L)",
+                                "('DEF',2L,8L)",
+                        });
+
+        Iterator<Tuple> it = pigServer.openIterator("c");
+        Util.checkQueryOutputsAfterSort(it, expectedRes);
+
+        Util.deleteFile(cluster, "testGroupElements1.txt");
+        Util.deleteFile(cluster, "testGroupElements2.txt");
+        pigServer.shutdown();
+    }
+
+    @Test
     public void testGroupElements() throws Exception {
         // test use of combiner when group elements are accessed in the foreach
         String input[] = {
@@ -352,13 +406,19 @@
         pigServer.shutdown();
     }
 
-    private void checkCombinerUsed(PigServer pigServer, String string, boolean combineExpected)
+    private void checkCombinerUsed(PigServer pigServer, String alias, boolean combineExpected)
             throws IOException {
         // make sure there is a combine plan in the explain output
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
         PrintStream ps = new PrintStream(baos);
-        pigServer.explain("c", ps);
-        boolean combinerFound = baos.toString().matches("(?si).*combine plan.*");
+        pigServer.explain(alias, ps);
+        boolean combinerFound;
+        if (pigServer.getPigContext().getExecType().name().equalsIgnoreCase("spark")) {
+            combinerFound = baos.toString().contains("Reduce By");
+        } else {
+            combinerFound = baos.toString().matches("(?si).*combine plan.*");
+        }
+
         System.out.println(baos.toString());
         assertEquals("is combiner present as expected", combineExpected, combinerFound);
     }
diff --git a/test/org/apache/pig/test/TestCubeOperator.java b/test/org/apache/pig/test/TestCubeOperator.java
index 456f807..253af32 100644
--- a/test/org/apache/pig/test/TestCubeOperator.java
+++ b/test/org/apache/pig/test/TestCubeOperator.java
@@ -566,8 +566,9 @@
     public void testIllustrate() throws Exception {
 	// test for illustrate
         Assume.assumeTrue("illustrate does not work in tez (PIG-3993)", !Util.getLocalTestMode().toString().startsWith("TEZ"));
-	String query = "a = load 'input' USING mock.Storage() as (a1:chararray,b1:chararray,c1:long); "
-	        + "b = cube a by cube(a1,b1);";
+        Assume.assumeTrue("illustrate does not work in spark (PIG-4621)", !Util.getLocalTestMode().toString().startsWith("SPARK"));
+        String query = "a = load 'input' USING mock.Storage() as (a1:chararray,b1:chararray,c1:long); "
+            + "b = cube a by cube(a1,b1);";
 
         Util.registerMultiLineQuery(pigServer, query);
         Map<Operator, DataBag> examples = pigServer.getExamples("b");
diff --git a/test/org/apache/pig/test/TestEmptyInputDir.java b/test/org/apache/pig/test/TestEmptyInputDir.java
index a9a46af..cfcdbc3 100644
--- a/test/org/apache/pig/test/TestEmptyInputDir.java
+++ b/test/org/apache/pig/test/TestEmptyInputDir.java
@@ -33,6 +33,7 @@
 import org.apache.pig.tools.pigstats.PigStats;
 import org.apache.pig.tools.pigstats.mapreduce.MRJobStats;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -85,6 +86,8 @@
                 assertEquals(0, js.getNumberMaps());
             }
 
+            //Spark doesn't create an empty result file part-*, only a _SUCCESS file if input dir was empty
+            Assume.assumeTrue("Skip this test for Spark. See PIG-5140", !Util.isSparkExecType(cluster.getExecType()));
             assertEmptyOutputFile();
         } finally {
             new File(PIG_FILE).delete();
diff --git a/test/org/apache/pig/test/TestEvalPipeline.java b/test/org/apache/pig/test/TestEvalPipeline.java
index 48ece69..6480fbe 100644
--- a/test/org/apache/pig/test/TestEvalPipeline.java
+++ b/test/org/apache/pig/test/TestEvalPipeline.java
@@ -24,6 +24,7 @@
 import java.io.PrintWriter;
 import java.io.StringWriter;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
@@ -422,9 +423,15 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-        int numIdentity = 0;
+        List<Tuple> actualResList = new ArrayList<Tuple>();
         while(iter.hasNext()){
-            Tuple t = iter.next();
+            actualResList.add(iter.next());
+        }
+
+        Util.sortQueryOutputsIfNeed(actualResList, Util.isSparkExecType(cluster.getExecType()));
+
+        int numIdentity = 0;
+        for (Tuple t : actualResList) {
             Assert.assertEquals((Integer)numIdentity, (Integer)t.get(0));
             Assert.assertEquals((Long)5L, (Long)t.get(2));
             Assert.assertEquals(LOOP_COUNT*2.0, (Double)t.get(3), 0.01);
@@ -467,9 +474,15 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-        int numIdentity = 0;
+        List<Tuple> actualResList = new ArrayList<Tuple>();
         while(iter.hasNext()){
-            Tuple t = iter.next();
+            actualResList.add(iter.next());
+        }
+
+        Util.sortQueryOutputsIfNeed(actualResList, Util.isSparkExecType(cluster.getExecType()));
+
+        int numIdentity = 0;
+        for (Tuple t : actualResList) {
             Assert.assertEquals((Integer)numIdentity, (Integer)t.get(0));
             Assert.assertEquals((Long)5L, (Long)t.get(2));
             Assert.assertEquals(LOOP_COUNT*2.0, (Double)t.get(3), 0.01);
@@ -842,9 +855,15 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-        int numIdentity = 0;
+        List<Tuple> actualResList = new ArrayList<Tuple>();
         while(iter.hasNext()){
-            Tuple t = iter.next();
+            actualResList.add(iter.next());
+        }
+
+        Util.sortQueryOutputsIfNeed(actualResList, Util.isSparkExecType(cluster.getExecType()));
+
+        int numIdentity = 0;
+        for (Tuple t : actualResList) {
             Assert.assertEquals((Integer)((numIdentity + 1) * 10), (Integer)t.get(0));
             Assert.assertEquals((Long)10L, (Long)t.get(1));
             Assert.assertEquals((Long)5L, (Long)t.get(2));
@@ -873,6 +892,10 @@
         pigServer.registerQuery("A = LOAD '"
                 + Util.generateURI(tmpFile.toString(), pigContext) + "';");
         pigServer.registerQuery("B = distinct A;");
+        if (Util.isSparkExecType(cluster.getExecType())) {
+            pigServer.registerQuery("B = order B by *;");
+        }
+
         String query = "C = foreach B {"
         + "C1 = $1 - $0;"
         + "C2 = $1%2;"
@@ -883,7 +906,6 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-
         int numRows = 0;
         for(int i = 0; i < LOOP_COUNT; i++) {
             for(int j = 0; j < LOOP_COUNT; j+=2){
@@ -920,6 +942,10 @@
         pigServer.registerQuery("A = LOAD '"
                 + Util.generateURI(tmpFile.toString(), pigContext) + "';");
         pigServer.registerQuery("B = distinct A;");
+        if (Util.isSparkExecType(cluster.getExecType())) {
+            pigServer.registerQuery("B = order B by *;");
+        }
+
         String query = "C = foreach B {"
         + "C1 = $0 + $1;"
         + "C2 = C1 + $0;"
@@ -929,7 +955,6 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-
         int numRows = 0;
         for(int i = 0; i < LOOP_COUNT; i++) {
             for(int j = 0; j < LOOP_COUNT; j+=2){
diff --git a/test/org/apache/pig/test/TestEvalPipeline2.java b/test/org/apache/pig/test/TestEvalPipeline2.java
index c8f51d7..34db8b8 100644
--- a/test/org/apache/pig/test/TestEvalPipeline2.java
+++ b/test/org/apache/pig/test/TestEvalPipeline2.java
@@ -138,9 +138,15 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-        int numIdentity = 0;
+        List<Tuple> actualResList = new ArrayList<Tuple>();
         while(iter.hasNext()){
-            Tuple tuple = iter.next();
+            actualResList.add(iter.next());
+        }
+
+        Util.sortQueryOutputsIfNeed(actualResList,Util.isSparkExecType(cluster.getExecType()));
+
+        int numIdentity = 0;
+        for (Tuple tuple : actualResList) {
             Tuple t = (Tuple)tuple.get(0);
             Assert.assertEquals(DataByteArray.class, t.get(0).getClass());
             int group = Integer.parseInt(new String(((DataByteArray)t.get(0)).get()));
@@ -476,17 +482,24 @@
         pigServer.registerQuery("D = COGROUP C BY b0, A BY a0 PARALLEL 2;");
         Iterator<Tuple> iter = pigServer.openIterator("D");
 
-        Assert.assertTrue(iter.hasNext());
-        Tuple t = iter.next();
+        if (Util.isSparkExecType(cluster.getExecType())) {
+            String[] expectedResults =
+                new String[] {"(2,{(2,2)},{(2,5,2)})", "(1,{(1,1)},{(1,2,3)})" };
+            Util.checkQueryOutputs(iter, expectedResults,
+                org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("D")), Util.isSparkExecType(cluster.getExecType()));
+        } else {
+            Assert.assertTrue(iter.hasNext());
+            Tuple t = iter.next();
 
-        Assert.assertTrue(t.toString().equals("(2,{(2,2)},{(2,5,2)})"));
+            Assert.assertTrue(t.toString().equals("(2,{(2,2)},{(2,5,2)})"));
 
-        Assert.assertTrue(iter.hasNext());
-        t = iter.next();
+            Assert.assertTrue(iter.hasNext());
+            t = iter.next();
 
-        Assert.assertTrue(t.toString().equals("(1,{(1,1)},{(1,2,3)})"));
+            Assert.assertTrue(t.toString().equals("(1,{(1,1)},{(1,2,3)})"));
 
-        Assert.assertFalse(iter.hasNext());
+            Assert.assertFalse(iter.hasNext());
+        }
     }
 
     // See PIG-1195
@@ -739,16 +752,10 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("EventsPerMinute");
 
-        Tuple t = iter.next();
-        Assert.assertTrue( (Long)t.get(0) == 60000 && (Long)t.get(1) == 2 && (Long)t.get(2) == 3 );
+        List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
+            new String[]{"(60000L,2L,3L)", "(120000L,2L,2L)", "(240000L,1L,1L)"});
 
-        t = iter.next();
-        Assert.assertTrue( (Long)t.get(0) == 120000 && (Long)t.get(1) == 2 && (Long)t.get(2) == 2 );
-
-        t = iter.next();
-        Assert.assertTrue( (Long)t.get(0) == 240000 && (Long)t.get(1) == 1 && (Long)t.get(2) == 1 );
-
-        Assert.assertFalse(iter.hasNext());
+        Util.checkQueryOutputs(iter, expectedResults, Util.isSparkExecType(cluster.getExecType()));
     }
 
     // See PIG-1729
@@ -1580,6 +1587,9 @@
 
         pigServer.registerQuery("data = load 'table_testLimitFlatten' as (k,v);");
         pigServer.registerQuery("grouped = GROUP data BY k;");
+        if (Util.isSparkExecType(cluster.getExecType())) {
+            pigServer.registerQuery("grouped = ORDER grouped BY group;");
+        }
         pigServer.registerQuery("selected = LIMIT grouped 2;");
         pigServer.registerQuery("flattened = FOREACH selected GENERATE FLATTEN (data);");
 
@@ -1587,7 +1597,9 @@
 
         String[] expected = new String[] {"(1,A)", "(1,B)", "(2,C)"};
 
-        Util.checkQueryOutputsAfterSortRecursive(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("flattened")));
+        Util.checkQueryOutputs(iter, expected,
+            org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("flattened")), 
+            Util.isSparkExecType(cluster.getExecType()));
     }
 
     // See PIG-2237
@@ -1650,8 +1662,14 @@
                     return false;
                 }
             });
-            // auto-parallelism is 2 in MR, 20 in Tez, so check >=2
-            Assert.assertTrue(partFiles.length >= 2);
+
+            if (Util.isSparkExecType(cluster.getExecType())) {
+                // TODO: Fix this when we implement auto-parallelism in Spark
+                Assert.assertTrue(partFiles.length == 1);
+            } else {
+                // auto-parallelism is 2 in MR, 20 in Tez, so check >=2
+                Assert.assertTrue(partFiles.length >= 2);
+            }
             // Check the output
             Iterator<Tuple> iter = job.getResults();
             List<Tuple> results = new ArrayList<Tuple>();
diff --git a/test/org/apache/pig/test/TestEvalPipelineLocal.java b/test/org/apache/pig/test/TestEvalPipelineLocal.java
index 906d61f..460ae12 100644
--- a/test/org/apache/pig/test/TestEvalPipelineLocal.java
+++ b/test/org/apache/pig/test/TestEvalPipelineLocal.java
@@ -37,6 +37,7 @@
 import java.util.Random;
 import java.util.Set;
 import java.util.StringTokenizer;
+import java.util.Collections;
 
 import org.apache.hadoop.mapreduce.Job;
 import org.apache.log4j.Appender;
@@ -884,7 +885,7 @@
         int LOOP_COUNT = 10;
         File tmpFile = File.createTempFile("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
-        for(int i = 0; i < LOOP_COUNT; i++) {
+        for(int i=0; i<LOOP_COUNT; i++) {
             for(int j=0;j<LOOP_COUNT;j+=2){
                 ps.println(i+"\t"+j);
                 ps.println(i+"\t"+j);
@@ -908,18 +909,29 @@
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
         int numIdentity = 0;
+        // When running with spark, output can be in a different order than that 
+        // when running in mr mode.
+        List<Tuple> resList = new ArrayList<Tuple>();
         while(iter.hasNext()){
-            Tuple t = iter.next();
-            Assert.assertEquals((Integer)((numIdentity + 1) * 10), (Integer)t.get(0));
+            resList.add(iter.next());
+        }
+
+        numIdentity = resList.size();
+        Util.sortQueryOutputsIfNeed(resList, Util.isSparkExecType(Util.getLocalTestMode()));
+        Assert.assertEquals(LOOP_COUNT, numIdentity);
+        // Since delta differences in some cases are allowed, utility function 
+        // to compare tuple-lists cannot be used here.
+        // This loop generates sorted expected data
+        for (int i=0; i<numIdentity; i++) {
+            Tuple t = resList.get(i);
+            Assert.assertEquals((Integer)((i + 1) * 10), (Integer)t.get(0));
             Assert.assertEquals((Long)10L, (Long)t.get(1));
             Assert.assertEquals((Long)5L, (Long)t.get(2));
             Assert.assertEquals(LOOP_COUNT*2.0, (Double)t.get(3), 0.01);
             Assert.assertEquals(8.0, (Double)t.get(5), 0.01);
             Assert.assertEquals(5L, ((DataBag)t.get(6)).size());
             Assert.assertEquals(7, t.size());
-            ++numIdentity;
         }
-        Assert.assertEquals(LOOP_COUNT, numIdentity);
     }
 
     @Test
@@ -927,12 +939,25 @@
         int LOOP_COUNT = 10;
         File tmpFile = File.createTempFile("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
+
+        List<Tuple> expectedList = new ArrayList<Tuple>();
         for(int i = 0; i < LOOP_COUNT; i++) {
             for(int j=0;j<LOOP_COUNT;j+=2){
                 ps.println(i+"\t"+j);
                 ps.println(i+"\t"+j);
+                //  Generating expected data
+                Tuple t = mTf.newTuple();
+                t.append(new Double(j - i));
+                t.append((Integer)(j%2));
+                if(j == 0) {
+                    t.append(0.0);
+                } else {
+                    t.append((Double)((double)i/j));
+                }
+                expectedList.add(t);
             }
         }
+        Util.sortQueryOutputsIfNeed(expectedList, Util.isSparkExecType(Util.getLocalTestMode()));
         ps.close();
 
         pigServer.registerQuery("A = LOAD '"
@@ -949,25 +974,30 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-
-        int numRows = 0;
-        for(int i = 0; i < LOOP_COUNT; i++) {
-            for(int j = 0; j < LOOP_COUNT; j+=2){
-                Tuple t = null;
-                if(iter.hasNext()) t = iter.next();
-                Assert.assertEquals(3, t.size());
-                Assert.assertEquals(new Double(j - i), (Double)t.get(0), 0.01);
-                Assert.assertEquals((Integer)(j%2), (Integer)t.get(1));
-                if(j == 0) {
-                    Assert.assertEquals(0.0, (Double)t.get(2), 0.01);
-                } else {
-                    Assert.assertEquals((Double)((double)i/j), (Double)t.get(2), 0.01);
-                }
-                ++numRows;
-            }
+        // When ruuning with spark, output can be in a different order than when
+        // running in mr mode.
+        List<Tuple> resList = new ArrayList<Tuple>();
+        while(iter.hasNext()){
+            resList.add(iter.next());
         }
 
-        Assert.assertEquals((LOOP_COUNT * LOOP_COUNT)/2, numRows);
+        Util.sortQueryOutputsIfNeed(resList, Util.isSparkExecType(Util.getLocalTestMode()));
+        Assert.assertEquals((LOOP_COUNT * LOOP_COUNT)/2, resList.size());
+
+        // Since delta difference in some cases is allowed, utility function 
+        // to compare tuple-lists cannot be used here.
+        for(int i = 0; i < LOOP_COUNT; i++) {
+            for(int j = 0; j < LOOP_COUNT; j+=2){
+                int k = i*LOOP_COUNT/2 + j/2;
+                Tuple res_t = resList.get(k);
+                Tuple expec_t = expectedList.get(k);
+
+                Assert.assertEquals(expec_t.size(), res_t.size());
+                Assert.assertEquals((Double)expec_t.get(0), (Double)res_t.get(0), 0.01);
+                Assert.assertEquals((Integer)expec_t.get(1), (Integer)res_t.get(1));
+                Assert.assertEquals((Double)expec_t.get(2), (Double)res_t.get(2), 0.01);
+            }
+        }
     }
 
     @Test
@@ -975,10 +1005,16 @@
         int LOOP_COUNT = 10;
         File tmpFile = File.createTempFile("test", "txt");
         PrintStream ps = new PrintStream(new FileOutputStream(tmpFile));
+        List<Tuple> expectedList = new ArrayList<Tuple>();
         for(int i = 0; i < LOOP_COUNT; i++) {
             for(int j=0;j<LOOP_COUNT;j+=2){
                 ps.println(i+"\t"+j);
                 ps.println(i+"\t"+j);
+                // Generating expected data.
+                Tuple t = mTf.newTuple();
+                t.append(new Double(i+j));
+                t.append(new Double(i + j + i));
+                expectedList.add(t);
             }
         }
         ps.close();
@@ -996,20 +1032,9 @@
         pigServer.registerQuery(query);
         Iterator<Tuple> iter = pigServer.openIterator("C");
         if(!iter.hasNext()) Assert.fail("No output found");
-
-        int numRows = 0;
-        for(int i = 0; i < LOOP_COUNT; i++) {
-            for(int j = 0; j < LOOP_COUNT; j+=2){
-                Tuple t = null;
-                if(iter.hasNext()) t = iter.next();
-                Assert.assertEquals(2, t.size());
-                Assert.assertEquals(new Double(i + j), (Double)t.get(0), 0.01);
-                Assert.assertEquals(new Double(i + j + i), (Double)t.get(1));
-                ++numRows;
-            }
-        }
-
-        Assert.assertEquals((LOOP_COUNT * LOOP_COUNT)/2, numRows);
+        // When ruuning with spark, output can be in a different order than that
+        // when running in mr mode.
+        Util.checkQueryOutputs(iter, expectedList, Util.isSparkExecType(Util.getLocalTestMode()));
     }
 
     @Test
@@ -1128,9 +1153,15 @@
         pigServer.registerQuery("b = foreach a generate TOTUPLE(x, y) as t, z;");
         pigServer.registerQuery("c = group b by t;");
         Iterator<Tuple> iter = pigServer.openIterator("c");
-        Assert.assertTrue(iter.next().toString().equals("((1,2),{((1,2),3)})"));
-        Assert.assertTrue(iter.next().toString().equals("((4,5),{((4,5),6)})"));
-        Assert.assertFalse(iter.hasNext());
+        // When ruuning with spark, output can be in a different order than that 
+        // when running in mr mode.
+        List<Tuple> expectedRes =
+                Util.getTuplesFromConstantTupleStrings(
+                        new String[] {
+                                "((1,2),{((1,2),3)})",
+                                "((4,5),{((4,5),6)})"
+                        });
+        Util.checkQueryOutputs(iter, expectedRes, Util.isSparkExecType(Util.getLocalTestMode()));
     }
     
     @Test
@@ -1297,7 +1328,7 @@
 
         logger.removeAppender(appender);
 
-        Assert.assertTrue(bos.toString().contains("New For Each(false,false)[tuple]"));
+        Assert.assertTrue(bos.toString().contains("New For Each"));
     }
 
     @Test
diff --git a/test/org/apache/pig/test/TestFinish.java b/test/org/apache/pig/test/TestFinish.java
index f18c103..7c79976 100644
--- a/test/org/apache/pig/test/TestFinish.java
+++ b/test/org/apache/pig/test/TestFinish.java
@@ -45,7 +45,7 @@
     BagFactory mBf = BagFactory.getInstance();
     File f1;
 
-    static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
+    private static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
 
     static public class MyEvalFunction extends EvalFunc<Tuple> {
         String execType;
diff --git a/test/org/apache/pig/test/TestFlatten.java b/test/org/apache/pig/test/TestFlatten.java
index 1b92be4..f315b84 100644
--- a/test/org/apache/pig/test/TestFlatten.java
+++ b/test/org/apache/pig/test/TestFlatten.java
@@ -95,6 +95,6 @@
         List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
                 new String[] {
                 "('a', 'b', '1', '2')", "('a', 'b', '3', '4')", "('c', 'd', '1', '2')", "('c', 'd', '3', '4')" });
-        Util.checkQueryOutputs(actualResults.iterator(), expectedResults);
+        Util.checkQueryOutputsAfterSort(actualResults.iterator(), expectedResults);
     }
 }
diff --git a/test/org/apache/pig/test/TestForEachNestedPlanLocal.java b/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
index 63d8f67..4bc4815 100644
--- a/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
+++ b/test/org/apache/pig/test/TestForEachNestedPlanLocal.java
@@ -101,14 +101,8 @@
         pig.registerQuery("D = foreach C {"
                 + "crossed = cross user, session;"
                 + "generate crossed;" + "}");
-        Iterator<Tuple> expectedItr = expectedResults.iterator();
         Iterator<Tuple> actualItr = pig.openIterator("D");
-        while (expectedItr.hasNext() && actualItr.hasNext()) {
-            Tuple expectedTuple = expectedItr.next();
-            Tuple actualTuple = actualItr.next();
-            assertEquals(expectedTuple, actualTuple);
-        }
-        assertEquals(expectedItr.hasNext(), actualItr.hasNext());
+        Util.checkQueryOutputsAfterSort(actualItr, expectedResults);
     }
 
     @Test
@@ -156,14 +150,8 @@
                 + "crossed = cross user, distinct_session;"
                 + "filtered = filter crossed by user::region == distinct_session::region;"
                 + "generate filtered;" + "}");
-        Iterator<Tuple> expectedItr = expectedResults.iterator();
         Iterator<Tuple> actualItr = pig.openIterator("D");
-        while (expectedItr.hasNext() && actualItr.hasNext()) {
-            Tuple expectedTuple = expectedItr.next();
-            Tuple actualTuple = actualItr.next();
-            assertEquals(expectedTuple, actualTuple);
-        }
-        assertEquals(expectedItr.hasNext(), actualItr.hasNext());
+        Util.checkQueryOutputsAfterSort(actualItr, expectedResults);
     }
 
     @Test
@@ -187,14 +175,8 @@
         pig.registerQuery("D = foreach C {"
                 + "crossed = cross user, session, profile;"
                 + "generate crossed;" + "}");
-        Iterator<Tuple> expectedItr = expectedResults.iterator();
         Iterator<Tuple> actualItr = pig.openIterator("D");
-        while (expectedItr.hasNext() && actualItr.hasNext()) {
-            Tuple expectedTuple = expectedItr.next();
-            Tuple actualTuple = actualItr.next();
-            assertEquals(expectedTuple, actualTuple);
-        }
-        assertEquals(expectedItr.hasNext(), actualItr.hasNext());
+        Util.checkQueryOutputsAfterSort(actualItr, expectedResults);
     }
 
     /*
diff --git a/test/org/apache/pig/test/TestGrunt.java b/test/org/apache/pig/test/TestGrunt.java
index 99ab31b..5ed629a 100644
--- a/test/org/apache/pig/test/TestGrunt.java
+++ b/test/org/apache/pig/test/TestGrunt.java
@@ -70,7 +70,6 @@
 import org.junit.Test;
 
 public class TestGrunt {
-
     static MiniGenericCluster cluster = MiniGenericCluster.buildCluster();
     private String basedir = "test/org/apache/pig/test/data";
 
@@ -915,6 +914,15 @@
 
     @Test
     public void testKeepGoigFailed() throws Throwable {
+        // in mr mode, the output file 'baz' will be automatically deleted if the mr job fails
+        // when "cat baz;" is executed, it throws "Encountered IOException. Directory baz does not exist"
+        // in GruntParser#processCat() and variable "caught" is true
+        // in spark mode, the output file 'baz' will not be automatically deleted even the job fails(see SPARK-7953)
+        // when "cat baz;" is executed, it does not throw exception and the variable "caught" is false
+        // TODO: Enable this for Spark when SPARK-7953 is resolved
+        Assume.assumeTrue(
+            "Skip this test for Spark until SPARK-7953 is resolved!",
+            !Util.isSparkExecType(cluster.getExecType()));
         PigServer server = new PigServer(cluster.getExecType(), cluster.getProperties());
         PigContext context = server.getPigContext();
         Util.copyFromLocalToCluster(cluster, "test/org/apache/pig/test/data/passwd", "passwd");
@@ -936,7 +944,6 @@
         InputStreamReader reader = new InputStreamReader(cmd);
 
         Grunt grunt = new Grunt(new BufferedReader(reader), context);
-
         boolean caught = false;
         try {
             grunt.exec();
@@ -1004,7 +1011,13 @@
             grunt.exec();
         } catch (PigException e) {
             caught = true;
-            assertTrue(e.getErrorCode() == 6017);
+            if (!Util.isSparkExecType(cluster.getExecType())) {
+                assertTrue(e.getErrorCode() == 6017);
+            } else {
+                //In spark mode, We wrap ExecException to RunTimeException and is thrown out in JobGraphBuilder#sparkOperToRDD,
+                //So unwrap the exception here
+                assertTrue(((ExecException) e.getCause()).getErrorCode() == 6017);
+            }
         }
 
         if (Util.isMapredExecType(cluster.getExecType())) {
@@ -1621,7 +1634,7 @@
         boolean found = false;
         for (String line : lines) {
             if (line.matches(".*Added jar .*" + jarName + ".*")) {
-                // MR mode
+                // MR and Spark mode
                 found = true;
             } else if (line.matches(".*Local resource.*" + jarName + ".*")) {
                 // Tez mode
diff --git a/test/org/apache/pig/test/TestHBaseStorage.java b/test/org/apache/pig/test/TestHBaseStorage.java
index 864985e..246feca 100644
--- a/test/org/apache/pig/test/TestHBaseStorage.java
+++ b/test/org/apache/pig/test/TestHBaseStorage.java
@@ -18,6 +18,7 @@
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -45,6 +46,7 @@
 import org.apache.pig.data.DataByteArray;
 import org.apache.pig.data.DataType;
 import org.apache.pig.data.Tuple;
+import org.apache.pig.data.TupleFactory;
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Assert;
@@ -1443,23 +1445,20 @@
 
         Iterator<Tuple> it = pig.openIterator("c");
         int index = 0;
-        while (it.hasNext()) {
-            Tuple t = it.next();
-            String rowKey = (String) t.get(0);
-            int col_a = (Integer) t.get(1);
-            Assert.assertNotNull(t.get(2));
-            double col_b = (Double) t.get(2);
-            String col_c = (String) t.get(3);
-
-            Assert.assertEquals("00".substring((index + "").length()) + index,
-                    rowKey);
-            Assert.assertEquals(index, col_a);
-            Assert.assertEquals(index + 0.0, col_b, 1e-6);
-            Assert.assertEquals("Text_" + index, col_c);
+        List<Tuple> expected = new ArrayList<Tuple>();
+        while (index<TEST_ROW_COUNT) {
+            Tuple t = TupleFactory.getInstance().newTuple();
+            t.append("00".substring((index + "").length()) + index);
+            t.append(index);
+            t.append(index + 0.0);
+            t.append("Text_" + index);
+            t.append(index);
+            t.append(new DataByteArray("Text_" + index));
             index++;
+            expected.add(t);
         }
-        Assert.assertEquals(index, TEST_ROW_COUNT);
-    }
+        Util.checkQueryOutputsAfterSort(it, expected);
+}
 
     @Test
     // See PIG-4151
diff --git a/test/org/apache/pig/test/TestLimitVariable.java b/test/org/apache/pig/test/TestLimitVariable.java
index 53b9dae..d16600a 100644
--- a/test/org/apache/pig/test/TestLimitVariable.java
+++ b/test/org/apache/pig/test/TestLimitVariable.java
@@ -80,15 +80,17 @@
         Iterator<Tuple> it = pigServer.openIterator("g");
 
         List<Tuple> expectedRes = Util.getTuplesFromConstantTupleStrings(new String[] {
-                "(5.0,36)"});
-        Util.checkQueryOutputs(it, expectedRes);
+                "(5.0,36L)"});
+        Util.checkQueryOutputsAfterSort(it, expectedRes);
         pigServer.getPigContext().getProperties().remove(PigConfiguration.PIG_EXEC_MAP_PARTAGG);
     }
 
     @Test
     public void testLimitVariable2() throws IOException {
+        //add field type here to use  Util.checkQueryOutputsAfterSort comparing the expected and actual
+        //results
         String query =
-            "a = load '" + inputFile.getName() + "' as (id, num);" +
+            "a = load '" + inputFile.getName() + "' as (id:int, num:int);" +
             "b = filter a by id == 2;" + // only 1 tuple returned (2,3)
             "c = order a by id ASC;" +
             "d = limit c b.num;" + // test bytearray to long implicit cast
@@ -99,18 +101,20 @@
         Iterator<Tuple> itD = pigServer.openIterator("d");
         List<Tuple> expectedResD = Util.getTuplesFromConstantTupleStrings(new String[] {
                 "(1,11)", "(2,3)", "(3,10)" });
-        Util.checkQueryOutputs(itD, expectedResD);
+        Util.checkQueryOutputsAfterSort(itD, expectedResD);
 
         Iterator<Tuple> itE = pigServer.openIterator("e");
         List<Tuple> expectedResE = Util.getTuplesFromConstantTupleStrings(new String[] {
                 "(1,11)", "(2,3)", "(3,10)", "(4,11)", "(5,10)", "(6,15)" });
-        Util.checkQueryOutputs(itE, expectedResE);
+        Util.checkQueryOutputsAfterSort(itE, expectedResE);
     }
 
     @Test
     public void testLimitVariable3() throws IOException {
+        //add field type here to use  Util.checkQueryOutputsAfterSort comparing the expected and actual
+        //results
         String query =
-            "a = load '" + inputFile.getName() + "' ;" +
+            "a = load '" + inputFile.getName() + "' as (id:int, num:int);" +
             "b = group a all;" +
             "c = foreach b generate COUNT(a) as sum;" +
             "d = order a by $0 ASC;" +
@@ -121,7 +125,7 @@
         Iterator<Tuple> itE = pigServer.openIterator("e");
         List<Tuple> expectedResE = Util.getTuplesFromConstantTupleStrings(new String[] {
                 "(1,11)", "(2,3)", "(3,10)", "(4,11)", "(5,10)", "(6,15)" });
-        Util.checkQueryOutputs(itE, expectedResE);
+        Util.checkQueryOutputsAfterSort(itE, expectedResE);
     }
 
     @Test
diff --git a/test/org/apache/pig/test/TestLineageFindRelVisitor.java b/test/org/apache/pig/test/TestLineageFindRelVisitor.java
index 49d7d65..cd54065 100644
--- a/test/org/apache/pig/test/TestLineageFindRelVisitor.java
+++ b/test/org/apache/pig/test/TestLineageFindRelVisitor.java
@@ -259,9 +259,10 @@
         pig.registerQuery("E = FOREACH D GENERATE (chararray) tupleD.a1;\n");
         Iterator<Tuple> iter  = pig.openIterator("E");
 
-        Assert.assertEquals("123", iter.next().get(0));
-        Assert.assertEquals("456", iter.next().get(0));
-        Assert.assertEquals("789", iter.next().get(0));
+        Util.checkQueryOutputs(iter,
+                new String[]{"(123)", "(456)", "(789)"},
+                org.apache.pig.newplan.logical.Util.translateSchema(pig.dumpSchema("E")), Util.isSparkExecType(Util
+                .getLocalTestMode()));
     }
 
     @Test
diff --git a/test/org/apache/pig/test/TestMapSideCogroup.java b/test/org/apache/pig/test/TestMapSideCogroup.java
index 2c78b4a..0d723f7 100644
--- a/test/org/apache/pig/test/TestMapSideCogroup.java
+++ b/test/org/apache/pig/test/TestMapSideCogroup.java
@@ -311,12 +311,11 @@
         pigServer.registerQuery("A = LOAD '" + INPUT_FILE1 + "," + INPUT_FILE4 + "' using "+ DummyCollectableLoader.class.getName() +"() as (c1:chararray,c2:int);");
         pigServer.registerQuery("B = LOAD '" + INPUT_FILE5 + "' using "+ DummyIndexableLoader.class.getName()   +"() as (c1:chararray,c2:int);");
 
-        DataBag dbMergeCogrp = BagFactory.getInstance().newDefaultBag();
+        List<Tuple> dbMergeCogrp = new ArrayList<Tuple>();
 
         pigServer.registerQuery("C = cogroup A by c1, B by c1 using 'merge';");
         Iterator<Tuple> iter = pigServer.openIterator("C");
 
-
         while(iter.hasNext()) {
             Tuple t = iter.next();
             dbMergeCogrp.add(t);
@@ -335,12 +334,29 @@
                 "(3,{(3,3),(3,2),(3,1)},{(3,1),(3,2),(3,3)})"
         };
 
-        assertEquals(9, dbMergeCogrp.size());
-        Iterator<Tuple> itr = dbMergeCogrp.iterator();
-        for(int i=0; i<9; i++){
-            assertEquals(itr.next().toString(), results[i]);   
+        List<Tuple> expected = Util.getTuplesFromConstantTupleStrings(results);
+
+        //We need sort dbMergeCogrp because the result is different in sequence between spark and other mode when
+        //multiple files are loaded(LOAD INPUT_FILE1,INPUT_FILE4...)
+        for (Tuple t : dbMergeCogrp) {
+            Util.convertBagToSortedBag(t);
         }
-        assertFalse(itr.hasNext());
+        for (Tuple t : expected) {
+            Util.convertBagToSortedBag(t);
+        }
+
+        Collections.sort(dbMergeCogrp);
+        Collections.sort(expected);
+        assertEquals(dbMergeCogrp.size(), expected.size());
+
+        //Since TestMapSideCogroup.DummyIndexableLoader.getNext() does not
+        //apply schema for each input tuple,Util#checkQueryOutputsAfterSortRecursive fails to assert.
+        // The schema for C is (int,{(chararray,int),(chararray,int),(chararray,int)},{(chararray,int),(chararray,int),(chararray,int)}).
+        //But the schema for result "dbMergeCogrp" is (int,{(chararray,int),(chararray,int),(chararray,int)},{(chararray,chararray),(chararray,chararray),(chararray,chararray)})
+        Iterator<Tuple> itr = dbMergeCogrp.iterator();
+        for (int i = 0; i < dbMergeCogrp.size(); i++) {
+            assertEquals(itr.next().toString(), expected.get(i).toString());
+        }
     }
 
     @Test
diff --git a/test/org/apache/pig/test/TestMergeJoinOuter.java b/test/org/apache/pig/test/TestMergeJoinOuter.java
index 81aee55..a578600 100644
--- a/test/org/apache/pig/test/TestMergeJoinOuter.java
+++ b/test/org/apache/pig/test/TestMergeJoinOuter.java
@@ -168,7 +168,7 @@
     
     @Test
     public void testLeftOuter() throws IOException {
-        
+
         pigServer.registerQuery("A = LOAD '"+INPUT_FILE1+"' using "+ DummyCollectableLoader.class.getName() +"() as (c1:chararray, c2:chararray);");
         pigServer.registerQuery("B = LOAD '"+INPUT_FILE2+"' using "+ DummyIndexableLoader.class.getName() +"() as (c1:chararray, c2:chararray);");
 
@@ -197,7 +197,7 @@
     
     @Test
     public void testRightOuter() throws IOException{
-        
+
         pigServer.registerQuery("A = LOAD '"+INPUT_FILE1+"' using "+ DummyCollectableLoader.class.getName() +"() as (c1:chararray, c2:chararray);");
         pigServer.registerQuery("B = LOAD '"+INPUT_FILE2+"' using "+ DummyIndexableLoader.class.getName() +"() as (c1:chararray, c2:chararray);");
         pigServer.registerQuery("C = join A by c1 right, B by c1 using 'merge';");
@@ -224,7 +224,7 @@
     
     @Test
     public void testFullOuter() throws IOException{
-        
+
         pigServer.registerQuery("A = LOAD '"+INPUT_FILE1+"' using "+ DummyCollectableLoader.class.getName() +"() as (c1:chararray, c2:chararray);");
         pigServer.registerQuery("B = LOAD '"+INPUT_FILE2+"' using "+ DummyIndexableLoader.class.getName() +"() as (c1:chararray, c2:chararray);");
         pigServer.registerQuery("C = join A by c1 full, B by c1 using 'merge';");
diff --git a/test/org/apache/pig/test/TestMultiQuery.java b/test/org/apache/pig/test/TestMultiQuery.java
index c32eab7..47cc8ed 100644
--- a/test/org/apache/pig/test/TestMultiQuery.java
+++ b/test/org/apache/pig/test/TestMultiQuery.java
@@ -31,6 +31,9 @@
 import org.apache.pig.builtin.mock.Storage;
 import org.apache.pig.data.Tuple;
 import org.apache.pig.impl.PigContext;
+import org.apache.pig.impl.logicalLayer.schema.Schema;
+import org.apache.pig.tools.pigstats.InputStats;
+
 import org.junit.After;
 import org.junit.AfterClass;
 import org.junit.Before;
@@ -106,33 +109,25 @@
         myPig.registerQuery("E = load 'output1' as (a:int, b:int);");
         Iterator<Tuple> iter = myPig.openIterator("E");
 
-        List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
-                new String[] {
+        String[] expectedResults = new String[]{
                         "(1,2)",
                         "(2,3)"
-                });
+        };
+        Schema s = myPig.dumpSchema("E");
+        Util.checkQueryOutputs(iter, expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(s), Util
+                .isSparkExecType(Util.getLocalTestMode()));
 
-        int counter = 0;
-        while (iter.hasNext()) {
-            assertEquals(expectedResults.get(counter++).toString(), iter.next().toString());
-        }
-        assertEquals(expectedResults.size(), counter);
 
         myPig.registerQuery("E = load 'output2' as (a:int, b:int);");
         iter = myPig.openIterator("E");
 
-        expectedResults = Util.getTuplesFromConstantTupleStrings(
-                new String[] {
+        expectedResults = new String[]{
                         "(2,3)",
                         "(3,4)"
-                });
-
-        counter = 0;
-        while (iter.hasNext()) {
-            assertEquals(expectedResults.get(counter++).toString(), iter.next().toString());
-        }
-
-        assertEquals(expectedResults.size(), counter);
+        };
+        s = myPig.dumpSchema("E");
+        Util.checkQueryOutputs(iter, expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(s), Util
+                .isSparkExecType(Util.getLocalTestMode()));
     }
 
     @Test
@@ -165,20 +160,15 @@
 
         Iterator<Tuple> iter = myPig.openIterator("F");
 
-        List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
-                new String[] {
+        String[] expectedResults = new String[]{
                         "(1,2)",
                         "(2,3)",
                         "(3,5)",
                         "(5,6)"
-                });
-
-        int counter = 0;
-        while (iter.hasNext()) {
-            assertEquals(expectedResults.get(counter++).toString(), iter.next().toString());
-        }
-
-        assertEquals(expectedResults.size(), counter);
+        };
+        Schema s = myPig.dumpSchema("F");
+        Util.checkQueryOutputs(iter, expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(s), Util
+                .isSparkExecType(Util.getLocalTestMode()));
     }
 
     @Test
@@ -299,19 +289,14 @@
 
         Iterator<Tuple> iter = myPig.openIterator("E");
 
-        List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
-                new String[] {
-                        "(1L,'apple',3,1L,'apple',1L,{(1L)})",
-                        "(2L,'orange',4,2L,'orange',2L,{(2L)})",
-                        "(3L,'persimmon',5,3L,'persimmon',3L,{(3L)})"
-                });
-
-        int counter = 0;
-        while (iter.hasNext()) {
-            assertEquals(expectedResults.get(counter++).toString(), iter.next().toString());
-        }
-
-        assertEquals(expectedResults.size(), counter);
+        String[] expectedResults = new String[]{
+                "(1L,apple,3,1L,apple,1L,{(1L)})",
+                "(2L,orange,4,2L,orange,2L,{(2L)})",
+                "(3L,persimmon,5,3L,persimmon,3L,{(3L)})"
+        };
+        Schema s = myPig.dumpSchema("E");
+        Util.checkQueryOutputs(iter, expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(s), Util
+                .isSparkExecType(Util.getLocalTestMode()));
     }
 
     @Test
@@ -345,19 +330,14 @@
 
         Iterator<Tuple> iter = myPig.openIterator("joined_session_info");
 
-        List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
-                new String[] {
-                        "('apple',{},{('apple','jar',1L)})",
-                        "('orange',{},{('orange','box',1L)})",
-                        "('strawberry',{(30,'strawberry','quit','bot')},{})"
-                });
+        String[] expectedResults = new String[]{
+                "(apple,{},{(apple,jar,1L)})",
+                "(orange,{},{(orange,box,1L)})",
+                "(strawberry,{(30,strawberry,quit,bot)},{})"};
 
-        int counter = 0;
-        while (iter.hasNext()) {
-            assertEquals(expectedResults.get(counter++).toString(), iter.next().toString());
-        }
-
-        assertEquals(expectedResults.size(), counter);
+        Schema s = myPig.dumpSchema("joined_session_info");
+        Util.checkQueryOutputs(iter, expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(s), Util
+                .isSparkExecType(Util.getLocalTestMode()));
     }
 
     @Test
@@ -861,12 +841,12 @@
         List<Tuple> actualResults = data.get("output1");
         List<Tuple> expectedResults = Util.getTuplesFromConstantTupleStrings(
                 new String[] {"(1)", "(2)"});
-        Util.checkQueryOutputs(actualResults.iterator(), expectedResults);
+        Util.checkQueryOutputsAfterSort(actualResults.iterator(), expectedResults);
 
         actualResults = data.get("output2");
         expectedResults = Util.getTuplesFromConstantTupleStrings(
                 new String[] {"(1, 'world')", "(2, 'world')"});
-        Util.checkQueryOutputs(actualResults.iterator(), expectedResults);
+        Util.checkQueryOutputsAfterSort(actualResults.iterator(), expectedResults);
     }
 
     @Test
@@ -908,6 +888,27 @@
     }
 
     @Test
+    public void testMultiQueryJiraPig4899() throws Exception {
+        myPig.setBatchOn();
+
+        myPig.registerQuery("a = load 'passwd' "
+                + "using PigStorage(':') as (uname:chararray, passwd:chararray, uid:int, gid:int);");
+        myPig.registerQuery("b1 = foreach a generate uname;");
+        myPig.registerQuery("b2 = foreach a generate uid;");
+        myPig.registerQuery("store b1 into 'output1';");
+        myPig.registerQuery("store b2 into 'output2';");
+
+        List<ExecJob> jobs = myPig.executeBatch();
+        for (ExecJob job : jobs) {
+            assertTrue(job.getStatus() == ExecJob.JOB_STATUS.COMPLETED);
+            List<InputStats> stats = job.getStatistics().getInputStats();
+            assertEquals(1,stats.size());
+            InputStats stat = stats.get(0);
+            assertEquals("Number of records in passwd file is 14",14,stat.getNumberRecords());
+        }
+    }
+
+    @Test
     public void testMultiQueryJiraPig4883() throws Exception {
         Storage.Data data = Storage.resetData(myPig);
         data.set("inputLocation",
@@ -934,20 +935,24 @@
 
         List<Tuple> actualResults = data.get("output1");
         String[] expectedResults = new String[]{"(12, 1)"};
-        Util.checkQueryOutputsAfterSortRecursive(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(myPig.dumpSchema("B1")));
+        Util.checkQueryOutputs(actualResults.iterator(), expectedResults, org.apache.pig.newplan
+                .logical.Util.translateSchema(myPig.dumpSchema("B1")), Util.isSparkExecType(Util.getLocalTestMode()));
 
 
         actualResults = data.get("output2");
         expectedResults = new String[]{"(c,1)"};
-        Util.checkQueryOutputsAfterSortRecursive(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(myPig.dumpSchema("B2")));
+        Util.checkQueryOutputs(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util
+                .translateSchema(myPig.dumpSchema("B2")), Util.isSparkExecType(Util.getLocalTestMode()));
 
         actualResults = data.get("output3");
         expectedResults = new String[]{"(-12, 1)"};
-        Util.checkQueryOutputsAfterSortRecursive(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(myPig.dumpSchema("C1")));
+        Util.checkQueryOutputs(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util
+                .translateSchema(myPig.dumpSchema("C1")), Util.isSparkExecType(Util.getLocalTestMode()));
 
         actualResults = data.get("output4");
         expectedResults = new String[]{"(d,1)"};
-        Util.checkQueryOutputsAfterSortRecursive(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util.translateSchema(myPig.dumpSchema("C2")));
+        Util.checkQueryOutputs(actualResults.iterator(), expectedResults, org.apache.pig.newplan.logical.Util
+                .translateSchema(myPig.dumpSchema("C2")), Util.isSparkExecType(Util.getLocalTestMode()));
     }
 
     // --------------------------------------------------------------------------
diff --git a/test/org/apache/pig/test/TestMultiQueryLocal.java b/test/org/apache/pig/test/TestMultiQueryLocal.java
index b9ac035..0cc4a5f 100644
--- a/test/org/apache/pig/test/TestMultiQueryLocal.java
+++ b/test/org/apache/pig/test/TestMultiQueryLocal.java
@@ -487,6 +487,7 @@
     public void testMultiQueryWithIllustrate() throws Exception {
 
         Assume.assumeTrue("illustrate does not work in tez (PIG-3993)", !Util.getLocalTestMode().toString().startsWith("TEZ"));
+        Assume.assumeTrue("illustrate does not work in spark (PIG-4621)", !Util.getLocalTestMode().toString().startsWith("SPARK"));
         System.out.println("===== test multi-query with illustrate =====");
 
         try {
diff --git a/test/org/apache/pig/test/TestNativeMapReduce.java b/test/org/apache/pig/test/TestNativeMapReduce.java
index c4f6573..1a27398 100644
--- a/test/org/apache/pig/test/TestNativeMapReduce.java
+++ b/test/org/apache/pig/test/TestNativeMapReduce.java
@@ -29,6 +29,7 @@
 
 import org.apache.hadoop.mapred.FileAlreadyExistsException;
 import org.apache.pig.PigServer;
+import org.apache.pig.backend.executionengine.ExecException;
 import org.apache.pig.backend.executionengine.ExecJob;
 import org.apache.pig.backend.executionengine.ExecJob.JOB_STATUS;
 import org.apache.pig.backend.hadoop.executionengine.JobCreationException;
@@ -210,6 +211,13 @@
         } catch (JobCreationException e) {
             // Running in Tez mode throw exception
             assertTrue(e.getCause() instanceof FileAlreadyExistsException);
+        } catch (ExecException e) {
+            // Running in spark mode throw exception
+            if (e.getCause() instanceof RuntimeException) {
+                RuntimeException re = (RuntimeException) e.getCause();
+                JobCreationException jce = (JobCreationException) re.getCause();
+                assertTrue(jce.getCause() instanceof FileAlreadyExistsException);
+            }
         }
         finally{
             // We have to manually delete intermediate mapreduce files
diff --git a/test/org/apache/pig/test/TestNullConstant.java b/test/org/apache/pig/test/TestNullConstant.java
index 3ea4509..2a9f8e5 100644
--- a/test/org/apache/pig/test/TestNullConstant.java
+++ b/test/org/apache/pig/test/TestNullConstant.java
@@ -109,16 +109,7 @@
         pigServer.registerQuery("d = foreach c generate flatten((SIZE(a) == 0 ? null: a)), flatten((SIZE(b) == 0 ? null : b));");
         Iterator<Tuple> it = pigServer.openIterator("d");
         Object[][] results = new Object[][]{{10, "will_join", 10, "will_join"}, {11, "will_not_join", null}, {null, 12, "will_not_join"}};
-        int i = 0;
-        while(it.hasNext()) {
-
-            Tuple t = it.next();
-            Object[] result = results[i++];
-            assertEquals(result.length, t.size());
-            for (int j = 0; j < result.length; j++) {
-                assertEquals(result[j], t.get(j));
-            }
-        }
+        Util.checkQueryOutputsAfterSort(it,results);
     }
 
     @Test
diff --git a/test/org/apache/pig/test/TestPigRunner.java b/test/org/apache/pig/test/TestPigRunner.java
index 25380e4..ec08417 100644
--- a/test/org/apache/pig/test/TestPigRunner.java
+++ b/test/org/apache/pig/test/TestPigRunner.java
@@ -41,11 +41,13 @@
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.mapred.Counters;
+import org.apache.pig.ExecType;
 import org.apache.pig.PigConfiguration;
 import org.apache.pig.PigRunner;
 import org.apache.pig.PigRunner.ReturnCode;
 import org.apache.pig.backend.hadoop.datastorage.ConfigurationUtil;
 import org.apache.pig.backend.hadoop.executionengine.mapReduceLayer.MRConfiguration;
+import org.apache.pig.backend.hadoop.executionengine.spark.SparkExecType;
 import org.apache.pig.impl.PigContext;
 import org.apache.pig.impl.io.FileLocalizer;
 import org.apache.pig.impl.plan.OperatorPlan;
@@ -58,6 +60,7 @@
 import org.apache.pig.tools.pigstats.PigStatsUtil;
 import org.apache.pig.tools.pigstats.mapreduce.MRJobStats;
 import org.apache.pig.tools.pigstats.mapreduce.MRPigStatsUtil;
+import org.apache.pig.tools.pigstats.spark.SparkJobStats;
 import org.junit.AfterClass;
 import org.junit.Assume;
 import org.junit.Before;
@@ -207,12 +210,13 @@
             PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
 
             assertTrue(stats.isSuccessful());
-            if (execType.toString().startsWith("tez")) {
-                assertEquals(1, stats.getNumberJobs());
-                assertEquals(stats.getJobGraph().size(), 1);
-            } else {
+            if (execType.equals("mapreduce")) {
                 assertEquals(2, stats.getNumberJobs());
                 assertEquals(stats.getJobGraph().size(), 2);
+            } else {
+                // Tez and Spark
+                assertEquals(1, stats.getNumberJobs());
+                assertEquals(stats.getJobGraph().size(), 1);
             }
 
             Configuration conf = ConfigurationUtil.toConfiguration(stats.getPigProperties());
@@ -274,6 +278,10 @@
                 assertEquals(stats.getJobGraph().size(), 1);
                 // 5 vertices
                 assertEquals(stats.getJobGraph().getSources().get(0).getPlan().size(), 5);
+            } else if (execType.equals("spark")) {
+                // In spark mode,the number of spark job is calculated by the number of POStore.
+                // 1 POStore generates 1 spark job.
+                assertEquals(stats.getJobGraph().size(), 1);
             } else {
                 assertEquals(stats.getJobGraph().size(), 4);
             }
@@ -294,7 +302,12 @@
                 //       Need to investigate
                 // assertEquals("B", ((JobStats) stats.getJobGraph().getPredecessors(
                 //        js).get(0)).getAlias());
+            } else if (execType.equals("spark")) {
+                assertEquals("A,B", ((JobStats) stats.getJobGraph().getSources().get(
+                        0)).getAlias());
+                // TODO: alias is not set for sample-aggregation/partition/sort job.
             } else {
+
                 assertEquals("A", ((JobStats) stats.getJobGraph().getSources().get(
                         0)).getAlias());
                 assertEquals("B", ((JobStats) stats.getJobGraph().getPredecessors(
@@ -323,7 +336,14 @@
             String[] args = { "-x", execType, PIG_FILE };
             PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
             assertTrue(stats.isSuccessful());
-            assertTrue(stats.getJobGraph().size() == 1);
+            if (execType.equals("spark")) {
+                // In spark mode,the number of spark job is calculated by the number of POStore.
+                // 2 POStore generates 2 spark jobs.
+                assertTrue(stats.getJobGraph().size() == 2);
+            } else {
+                assertTrue(stats.getJobGraph().size() == 1);
+            }
+
             // Each output file should include the following:
             // output:
             //   1\t2\t3\n
@@ -372,7 +392,13 @@
             String[] args = { "-x", execType, PIG_FILE };
             PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
             assertTrue(stats.isSuccessful());
-            assertEquals(stats.getJobGraph().size(), 1);
+            if (execType.equals("spark")) {
+                // In spark mode,the number of spark job is calculated by the number of POStore.
+                // 2 POStore generates 2 spark jobs.
+                assertEquals(stats.getJobGraph().size(), 2);
+            } else {
+                assertEquals(stats.getJobGraph().size(), 1);
+            }
 
             // Each output file should include the following:
             // output:
@@ -430,11 +456,15 @@
         w.close();
 
         try {
-            String[] args = { "-x", execType, PIG_FILE };
-            PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
+          String[] args = null;
+          args = new String[]{"-x", execType, PIG_FILE};
+          PigStats stats =  PigRunner.run(args, new TestNotificationListener(execType));
             assertTrue(stats.isSuccessful());
             if (Util.isMapredExecType(cluster.getExecType())) {
                 assertEquals(3, stats.getJobGraph().size());
+            } else if (Util.isSparkExecType(cluster.getExecType())) {
+                // One for each store and 3 for join.
+                assertEquals(4, stats.getJobGraph().size());
             } else {
                 assertEquals(1, stats.getJobGraph().size());
             }
@@ -475,10 +505,14 @@
             });
             assertEquals(5, inputStats.get(0).getNumberRecords());
             assertEquals(3, inputStats.get(1).getNumberRecords());
-            // For mapreduce, since hdfs bytes read includes replicated tables bytes read is wrong
             // Since Tez does has only one load per job its values are correct
+            // the result of inputStats in spark mode is also correct
             if (!Util.isMapredExecType(cluster.getExecType())) {
                 assertEquals(30, inputStats.get(0).getBytes());
+            }
+
+            //TODO PIG-5240:Fix TestPigRunner#simpleMultiQueryTest3 in spark mode for wrong inputStats
+            if (!Util.isMapredExecType(cluster.getExecType()) && !Util.isSparkExecType(cluster.getExecType())) {
                 assertEquals(18, inputStats.get(1).getBytes());
             }
         } finally {
@@ -504,17 +538,17 @@
             PigStats stats = PigRunner.run(args, null);
             Iterator<JobStats> iter = stats.getJobGraph().iterator();
             while (iter.hasNext()) {
-                 JobStats js=iter.next();
-                 if (execType.equals("tez")) {
-                     assertEquals(js.getState().name(), "FAILED");
-                 } else {
-                     if(js.getState().name().equals("FAILED")) {
-                         List<Operator> ops=stats.getJobGraph().getSuccessors(js);
-                         for(Operator op : ops ) {
-                             assertEquals(((JobStats)op).getState().toString(), "UNKNOWN");
-                         }
-                     }
-                 }
+                JobStats js=iter.next();
+                if (execType.equals("mapreduce")) {
+                    if (js.getState().name().equals("FAILED")) {
+                        List<Operator> ops = stats.getJobGraph().getSuccessors(js);
+                        for (Operator op : ops) {
+                            assertEquals(((JobStats) op).getState().toString(), "UNKNOWN");
+                        }
+                    }
+                } else {
+                    assertEquals(js.getState().name(), "FAILED");
+                }
             }
         } finally {
             new File(PIG_FILE).delete();
@@ -723,8 +757,14 @@
             PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
 
             assertTrue(stats.isSuccessful());
-
-            assertEquals(1, stats.getNumberJobs());
+            //In spark mode, one POStore will generate a spark action(spark job).
+            //In this case, the sparkplan has 1 sparkOperator(after multiquery optimization) but has 2 POStores
+            //which generate 2 spark actions(spark jobs).
+            if (execType.equals("spark")) {
+                assertEquals(2, stats.getNumberJobs());
+            } else {
+                assertEquals(1, stats.getNumberJobs());
+            }
             List<OutputStats> outputs = stats.getOutputStats();
             assertEquals(2, outputs.size());
             for (OutputStats outstats : outputs) {
@@ -835,7 +875,14 @@
 
             assertTrue(!stats.isSuccessful());
             assertTrue(stats.getReturnCode() != 0);
-            assertTrue(stats.getOutputStats().size() == 0);
+            if (execType.equals("spark")) {
+                //Currently, even if failed, spark engine will add a failed OutputStats,
+                // see: SparkPigStats.addFailJobStats()
+                assertTrue(stats.getOutputStats().size() == 1);
+                assertTrue(stats.getOutputStats().get(0).isSuccessful() == false);
+            } else {
+                assertTrue(stats.getOutputStats().size() == 0);
+            }
 
         } finally {
             new File(PIG_FILE).delete();
@@ -858,7 +905,14 @@
 
             assertTrue(!stats.isSuccessful());
             assertTrue(stats.getReturnCode() != 0);
-            assertTrue(stats.getOutputStats().size() == 0);
+            //Currently, even if failed, spark engine will add a failed OutputStats,
+            // see: SparkPigStats.addFailJobStats()
+            if (execType.equals("spark")) {
+                assertTrue(stats.getOutputStats().size() == 1);
+                assertTrue(stats.getOutputStats().get(0).isSuccessful() == false);
+            } else {
+                assertTrue(stats.getOutputStats().size() == 0);
+            }
 
         } finally {
             new File(PIG_FILE).delete();
@@ -919,8 +973,14 @@
             PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
 
             assertTrue(stats.isSuccessful());
-
-            assertEquals(1, stats.getNumberJobs());
+            //In spark mode, one POStore will generate a spark action(spark job).
+            //In this case, the sparkplan has 1 sparkOperator(after multiquery optimization) but has 2 POStores
+            //which generate 2 spark actions(spark jobs).
+            if (execType.equals("spark")) {
+                assertEquals(2, stats.getNumberJobs());
+            } else {
+                assertEquals(1, stats.getNumberJobs());
+            }
             List<OutputStats> outputs = stats.getOutputStats();
             assertEquals(2, outputs.size());
             for (OutputStats outstats : outputs) {
@@ -968,7 +1028,13 @@
             List<OutputStats> outputs = stats.getOutputStats();
             assertEquals(1, outputs.size());
             OutputStats outstats = outputs.get(0);
-            assertEquals(9, outstats.getNumberRecords());
+            //In spark mode, if pig.disable.counter = true, the number of records of the
+            //output are not calculated.
+            if (execType.equals("spark")) {
+                assertEquals(-1, outstats.getNumberRecords());
+            } else {
+                assertEquals(9, outstats.getNumberRecords());
+            }
         } finally {
             new File(PIG_FILE).delete();
             Util.deleteFile(cluster, OUTPUT_FILE);
@@ -1008,6 +1074,28 @@
                         MRPigStatsUtil.HDFS_BYTES_WRITTEN).getValue());
                 assertEquals(new File(INPUT_FILE).length(),counter.getGroup(FS_COUNTER_GROUP).getCounterForName(
                         MRPigStatsUtil.HDFS_BYTES_READ).getValue());
+            } else if (execType.equals("spark")) {
+                /** Uncomment code until changes of PIG-4788 are merged to master
+                //There are 2 spark jobs because of 2 POStore although the spark plan is optimized by multiquery optimization.
+                List<JobStats> jobs = stats.getJobGraph().getJobList();
+                JobStats firstJob = jobs.get(0);
+                JobStats secondJob = jobs.get(1);
+                //the hdfs_bytes_read of two spark jobs are same(because the two spark jobs have same poLoad), we only
+                //use one of those to compare with expected hdfs_bytes_read(30)
+                //we count the hdfs_bytes_written of the two spark jobs to calculate the total hdfs_bytes_written
+                long hdfs_bytes_read = 0;
+                long hdfs_bytes_written = 0;
+
+                hdfs_bytes_read += firstJob.getHadoopCounters().getGroup(SparkJobStats.FS_COUNTER_GROUP).getCounterForName(
+                        MRPigStatsUtil.HDFS_BYTES_READ).getValue();
+                hdfs_bytes_written += firstJob.getHadoopCounters().getGroup(SparkJobStats.FS_COUNTER_GROUP).getCounterForName(
+                        MRPigStatsUtil.HDFS_BYTES_WRITTEN).getValue();
+                hdfs_bytes_written += secondJob.getHadoopCounters().getGroup(SparkJobStats.FS_COUNTER_GROUP).getCounterForName(
+                        MRPigStatsUtil.HDFS_BYTES_WRITTEN).getValue();
+
+                assertEquals(30, hdfs_bytes_read);
+                assertEquals(20, hdfs_bytes_written);
+                 **/
             } else {
                 Counters counter= ((MRJobStats)stats.getJobGraph().getSinks().get(0)).getHadoopCounters();
                 assertEquals(5, counter.getGroup(MRPigStatsUtil.TASK_COUNTER_GROUP).getCounterForName(
@@ -1050,8 +1138,12 @@
             PigStats stats = PigRunner.run(args, new TestNotificationListener(execType));
 
             assertTrue(stats.isSuccessful());
-
-            assertEquals(1, stats.getNumberJobs());
+            if (execType.equals("spark")) {
+                //2 POStore generates 2 spark jobs
+                assertEquals(2, stats.getNumberJobs());
+            } else {
+                assertEquals(1, stats.getNumberJobs());
+            }
             List<OutputStats> outputs = stats.getOutputStats();
             assertEquals(2, outputs.size());
             if (execType.equals("tez")) {
@@ -1072,7 +1164,11 @@
             List<InputStats> inputs = stats.getInputStats();
             assertEquals(1, inputs.size());
             InputStats instats = inputs.get(0);
-            assertEquals(5, instats.getNumberRecords());
+            if (execType.equals("spark")) {
+                assertEquals(-1, instats.getNumberRecords());
+            } else {
+                assertEquals(5, instats.getNumberRecords());
+            }
         } finally {
             new File(PIG_FILE).delete();
             Util.deleteFile(cluster, OUTPUT_FILE);
diff --git a/test/org/apache/pig/test/TestPigServer.java b/test/org/apache/pig/test/TestPigServer.java
index 8e28646..45b561d 100644
--- a/test/org/apache/pig/test/TestPigServer.java
+++ b/test/org/apache/pig/test/TestPigServer.java
@@ -62,6 +62,7 @@
 import org.apache.pig.impl.util.Utils;
 import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -547,9 +548,8 @@
     public void testExplainXmlComplex() throws Throwable {
         // TODO: Explain XML output is not supported in non-MR mode. Remove the
         // following condition once it's implemented in Tez.
-        if (cluster.getExecType() != ExecType.MAPREDUCE) {
-            return;
-        }
+        String execType = cluster.getExecType().toString().toLowerCase();
+        Assume.assumeTrue("Skip this test for TEZ", Util.isMapredExecType(cluster.getExecType()) || Util.isSparkExecType(cluster.getExecType()));
         PigServer pig = new PigServer(cluster.getExecType(), properties);
         pig.registerQuery("a = load 'a' as (site: chararray, count: int, itemCounts: bag { itemCountsTuple: tuple (type: chararray, typeCount: int, f: float, m: map[]) } ) ;") ;
         pig.registerQuery("b = foreach a generate site, count, FLATTEN(itemCounts);") ;
@@ -574,6 +574,55 @@
         assertEquals(1, physicalPlan.getLength());
         assertTrue(physicalPlan.item(0).getTextContent().contains("Not Supported"));
 
+
+        if (execType.equals(ExecType.MAPREDUCE.name().toLowerCase())){
+            verifyExplainXmlComplexMR(doc);
+        } else if (execType.equals(MiniGenericCluster.EXECTYPE_SPARK)){
+            verifyExplainXmlComplexSpark(doc);
+        }
+
+
+    }
+
+    private void verifyExplainXmlComplexSpark(Document doc) {
+        NodeList stores = doc.getElementsByTagName("POStore");
+        assertEquals(1, stores.getLength());
+
+        NodeList groups = doc.getElementsByTagName("POJoinGroupSpark");
+        assertEquals(2, groups.getLength());
+
+        Node innerGroup = groups.item(1);
+
+        NodeList groupChildren = innerGroup.getChildNodes();
+
+        int foreachCount = 0;
+        int castCount = 0;
+        int loadCount = 0;
+
+        for (int i = 0; i < groupChildren.getLength(); i++) {
+            Node node = groupChildren.item(i);
+            if (node.getNodeName().equals("POForEach")){
+                ++foreachCount;
+                NodeList foreachNodes = node.getChildNodes();
+                for (int j = 0; j < foreachNodes.getLength(); j++) {
+                    Node innerNode = foreachNodes.item(j);
+                    if (innerNode.getNodeName().equals("alias")){
+                        assertEquals("b",innerNode.getTextContent());
+                    }else if (innerNode.getNodeName().equals("POCast")){
+                        ++castCount;
+                    }else if (innerNode.getNodeName().equals("POLoad")) {
+                        ++loadCount;
+                    }
+                }
+            }
+        }
+
+        assertEquals(1,foreachCount);
+        assertEquals(3,castCount);
+        assertEquals(1,loadCount);
+    }
+
+    private void verifyExplainXmlComplexMR(Document doc) {
         //Verify we have two loads and one is temporary
         NodeList loads = doc.getElementsByTagName("POLoad");
         assertEquals(2, loads.getLength());
diff --git a/test/org/apache/pig/test/TestPigServerLocal.java b/test/org/apache/pig/test/TestPigServerLocal.java
index fbabd03..3817b9d 100644
--- a/test/org/apache/pig/test/TestPigServerLocal.java
+++ b/test/org/apache/pig/test/TestPigServerLocal.java
@@ -255,6 +255,16 @@
             _testSkipParseInRegisterForBatch(false, 8, 4);
             _testSkipParseInRegisterForBatch(true, 5, 1);
             _testParseBatchWithScripting(5, 1);
+        } else if (Util.getLocalTestMode().toString().startsWith("SPARK")) {
+            // 6 = 4 (Once per registerQuery) + 2 (SortConverter , PigRecordReader)
+            // 4 (Once per registerQuery)
+            _testSkipParseInRegisterForBatch(false, 6, 4);
+
+            // 3 = 1 (registerQuery) + 2 (SortConverter, PigRecordReader)
+            // 1 (registerQuery)
+            _testSkipParseInRegisterForBatch(true, 3, 1);
+
+            _testParseBatchWithScripting(3, 1);
         } else {
             // numTimesInitiated = 10. 4 (once per registerQuery) + 6 (launchPlan->RandomSampleLoader,
             // InputSizeReducerEstimator, getSplits->RandomSampleLoader,
diff --git a/test/org/apache/pig/test/TestProjectRange.java b/test/org/apache/pig/test/TestProjectRange.java
index 2e3e7b8..660c411 100644
--- a/test/org/apache/pig/test/TestProjectRange.java
+++ b/test/org/apache/pig/test/TestProjectRange.java
@@ -644,13 +644,14 @@
 
         Iterator<Tuple> it = pigServer.openIterator("f");
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "(10,{(10,20,30,40,50)})",
-                            "(11,{(11,21,31,41,51)})",
-                    });
-        Util.checkQueryOutputs(it, expectedRes);
+        String[] expectedRes =
+                new String[] {
+                        "(10,{(10,20,30,40,50)})",
+                        "(11,{(11,21,31,41,51)})",
+                };
+        Schema s = pigServer.dumpSchema("f");
+        Util.checkQueryOutputs(it, expectedRes,org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
     /**
@@ -732,12 +733,14 @@
 
         Iterator<Tuple> it = pigServer.openIterator("f");
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "(1,{(11,21,31,41,51),(10,20,30,40,50)})",
-                    });
-        Util.checkQueryOutputs(it, expectedRes);
+        String[] expectedRes =
+                new String[] {
+                        "(1,{(11,21,31,41,51),(10,20,30,40,50)})",
+                };
+        Schema s = pigServer.dumpSchema("f");
+        Util.checkQueryOutputs(it, expectedRes,org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
+
     }
 
     private LOSort checkNumExpressionPlansForSort(LogicalPlan lp, int numPlans, boolean[] isAsc) {
@@ -925,8 +928,8 @@
             "  g = group l1 by   .. c,  l2 by .. c;"
             ;
         String expectedSchStr = "grp: (a: int,b: long,c: int)," +
-        		"l1: {t : (a: int,b: long,c: int,d: int,e: int)}," +
-        		"l2: {t : (a: int,b: long,c: int,d: int,e: int)}";
+                "l1: {t : (a: int,b: long,c: int,d: int,e: int)}," +
+                "l2: {t : (a: int,b: long,c: int,d: int,e: int)}";
 
         Schema expectedSch = getCleanedGroupSchema(expectedSchStr);
         compileAndCompareSchema(expectedSch, query, "g");
@@ -938,14 +941,15 @@
 
         Util.registerMultiLineQuery(pigServer, query);
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "((10,20,30),{(10,20,30,40,50)},{(10,20,30,40,50)})",
-                            "((11,21,31),{(11,21,31,41,51)},{(11,21,31,41,51)})",
-                    });
+        String[] expectedRes =
+                new String[] {
+                        "((10,20,30),{(10,20,30,40,50)},{(10,20,30,40,50)})",
+                        "((11,21,31),{(11,21,31,41,51)},{(11,21,31,41,51)})",
+                };
         Iterator<Tuple> it = pigServer.openIterator("g");
-        Util.checkQueryOutputs(it, expectedRes);
+        Schema s = pigServer.dumpSchema("g");
+        Util.checkQueryOutputs(it, expectedRes,org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
     /**
@@ -1005,14 +1009,15 @@
 
         Util.registerMultiLineQuery(pigServer, query);
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "((30,30,40,50),{(10,20,30,40,50)},{(10,20,30,40,50)})",
-                            "((32,31,41,51),{(11,21,31,41,51)},{(11,21,31,41,51)})",
-                    });
+        String[] expectedRes =
+                new String[] {
+                        "((30,30,40,50),{(10,20,30,40,50)},{(10,20,30,40,50)})",
+                        "((32,31,41,51),{(11,21,31,41,51)},{(11,21,31,41,51)})",
+                };
         Iterator<Tuple> it = pigServer.openIterator("g");
-        Util.checkQueryOutputs(it, expectedRes);
+        Schema s = pigServer.dumpSchema("g");
+        Util.checkQueryOutputs(it,expectedRes, org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
     @Test
@@ -1056,14 +1061,15 @@
 
         Util.registerMultiLineQuery(pigServer, query);
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "((30,40,50),{(10,20L,30,40,50)})",
-                            "((31,41,51),{(11,21L,31,41,51)})",
-                    });
+        String[] expectedRes =
+                new String[] {
+                        "((30,40,50),{(10,20L,30,40,50)})",
+                        "((31,41,51),{(11,21L,31,41,51)})",
+                };
         Iterator<Tuple> it = pigServer.openIterator("lim");
-        Util.checkQueryOutputsAfterSort(it, expectedRes);
+        Schema s = pigServer.dumpSchema("lim");
+        Util.checkQueryOutputs(it, expectedRes,org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
 
@@ -1118,14 +1124,15 @@
 
         Util.registerMultiLineQuery(pigServer, query);
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "((30,40,50),{(10,20,30,40,50)})",
-                            "((31,41,51),{(11,21,31,41,51)})",
-                    });
+        String[] expectedRes =
+                new String[] {
+                        "((30,40,50),{(10,20,30,40,50)})",
+                        "((31,41,51),{(11,21,31,41,51)})",
+                };
         Iterator<Tuple> it = pigServer.openIterator("g");
-        Util.checkQueryOutputs(it, expectedRes);
+        Schema s = pigServer.dumpSchema("g");
+        Util.checkQueryOutputs(it, expectedRes, org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
     private void setAliasesToNull(Schema schema) {
@@ -1157,14 +1164,15 @@
 
         Util.registerMultiLineQuery(pigServer, query);
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "(10,20,30,40,50,10,20,30,40,50)",
-                            "(11,21,31,41,51,11,21,31,41,51)",
-                    });
+        String[] expectedRes =
+                new String[] {
+                        "(10,20,30,40,50,10,20,30,40,50)",
+                        "(11,21,31,41,51,11,21,31,41,51)",
+                };
         Iterator<Tuple> it = pigServer.openIterator("j");
-        Util.checkQueryOutputs(it, expectedRes);
+        Schema s = pigServer.dumpSchema("j");
+        Util.checkQueryOutputs(it, expectedRes, org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
     @Test
@@ -1185,14 +1193,15 @@
 
         Util.registerMultiLineQuery(pigServer, query);
 
-        List<Tuple> expectedRes =
-            Util.getTuplesFromConstantTupleStrings(
-                    new String[] {
-                            "(10,20,30,40,50,10,20,30,40,50)",
-                            "(11,21,31,41,51,11,21,31,41,51)",
-                    });
+        String[] expectedRes =
+                new String[] {
+                        "(10,20,30,40,50,10,20,30,40,50)",
+                        "(11,21,31,41,51,11,21,31,41,51)",
+                };
         Iterator<Tuple> it = pigServer.openIterator("j");
-        Util.checkQueryOutputs(it, expectedRes);
+        Schema s = pigServer.dumpSchema("j");
+        Util.checkQueryOutputs(it, expectedRes, org.apache.pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
     }
 
     @Test
@@ -1204,7 +1213,7 @@
             "  l2 = load '" + INP_FILE_5FIELDS +  "';" +
             "  g = cogroup l1 by  ($0 ..  ),  l2 by ($0 .. );";
         Util.checkExceptionMessage(query, "g", "Cogroup/Group by '*' or 'x..' " +
-        		"(range of columns to the end) " +
+                "(range of columns to the end) " +
                         "is only allowed if the input has a schema");
     }
 
diff --git a/test/org/apache/pig/test/TestPruneColumn.java b/test/org/apache/pig/test/TestPruneColumn.java
index f05e0ec..45b911f 100644
--- a/test/org/apache/pig/test/TestPruneColumn.java
+++ b/test/org/apache/pig/test/TestPruneColumn.java
@@ -554,27 +554,9 @@
         pigServer.registerQuery("C = cogroup A by $1, B by $1;");
         pigServer.registerQuery("D = foreach C generate AVG($1.$1);");
         Iterator<Tuple> iter = pigServer.openIterator("D");
-
-        assertTrue(iter.hasNext());
-        Tuple t = iter.next();
-
-        assertEquals(1, t.size());
-        assertNull(t.get(0));
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(1, t.size());
-        assertEquals("2.0", t.get(0).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(1, t.size());
-        assertEquals("5.0", t.get(0).toString());
-
-        assertFalse(iter.hasNext());
-
+        String[] expectedRes = new String[]{"()","(2.0)","(5.0)"};
+        Schema s = pigServer.dumpSchema("D");
+        Util.checkQueryOutputsAfterSortRecursive(iter,expectedRes,org.apache.pig.newplan.logical.Util.translateSchema(s));
         assertTrue(checkLogFileMessage(new String[]{"Columns pruned for B: $0"}));
     }
 
@@ -596,26 +578,19 @@
 
     @Test
     public void testCoGroup3() throws Exception {
-        pigServer.registerQuery("A = load '"+ Util.generateURI(tmpFile1.toString(), pigServer.getPigContext()) + "' AS (a0, a1:int, a2);");
+        pigServer.registerQuery("A = load '" + Util.generateURI(tmpFile1.toString(), pigServer.getPigContext()) + "' AS (a0, a1:int, a2);");
         pigServer.registerQuery("B = group A by $1;");
         pigServer.registerQuery("C = foreach B generate $1, '1';");
         Iterator<Tuple> iter = pigServer.openIterator("C");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
 
-        assertEquals(2, t.size());
-        assertEquals("{(1,2,3)}", t.get(0).toString());
-        assertEquals("1", t.get(1).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(2, t.size());
-        assertEquals("{(2,5,2)}", t.get(0).toString());
-        assertEquals("1", t.get(1).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "({(1,2,3)},1)",
+                "({(2,5,2)},1)"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("C")),
+                Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(emptyLogFileMessage());
     }
@@ -629,27 +604,14 @@
         Iterator<Tuple> iter = pigServer.openIterator("D");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
 
-        assertEquals(2, t.size());
-        assertEquals("{}", t.get(0).toString());
-        assertEquals("{(1)}", t.get(1).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(2, t.size());
-        assertEquals("{(2)}", t.get(0).toString());
-        assertEquals("{(2)}", t.get(1).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(2, t.size());
-        assertEquals("{(5)}", t.get(0).toString());
-        assertEquals("{}", t.get(1).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "({},{(1)})",
+                "({(2)},{(2)})",
+                "({(5)},{})"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("D")),
+                Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(emptyLogFileMessage());
     }
@@ -755,22 +717,12 @@
         Iterator<Tuple> iter = pigServer.openIterator("D");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
 
-        assertEquals(3, t.size());
-        assertEquals("{}", t.get(0).toString());
-        assertEquals("1", t.get(1).toString());
-        assertEquals("1", t.get(2).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(3, t.size());
-        assertEquals("{(1,2,3)}", t.get(0).toString());
-        assertEquals("2", t.get(1).toString());
-        assertEquals("2", t.get(2).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "({},1,1)",
+                "({(1,2,3)},2,2)"
+        };
+        Util.checkQueryOutputsAfterSortRecursive(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("D")));
 
         assertTrue(emptyLogFileMessage());
     }
@@ -784,24 +736,13 @@
         Iterator<Tuple> iter = pigServer.openIterator("D");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
 
-        assertEquals(4, t.size());
-        assertEquals("1", t.get(0).toString());
-        assertEquals("2", t.get(1).toString());
-        assertEquals("3", t.get(2).toString());
-        assertEquals("{(2)}", t.get(3).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(4, t.size());
-        assertEquals("2", t.get(0).toString());
-        assertEquals("5", t.get(1).toString());
-        assertEquals("2", t.get(2).toString());
-        assertEquals("{}", t.get(3).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "(1,2,3,{(2)})",
+                "(2,5,2,{})"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("D")),
+            Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(emptyLogFileMessage());
     }
@@ -900,20 +841,13 @@
         Iterator<Tuple> iter = pigServer.openIterator("D");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
 
-        assertEquals(2, t.size());
-        assertEquals("1", t.get(0).toString());
-        assertEquals("1", t.get(1).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(2, t.size());
-        assertEquals("2", t.get(0).toString());
-        assertEquals("2", t.get(1).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "(1,1)",
+                "(2,2)"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("D")), 
+             Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(checkLogFileMessage(new String[]{"Columns pruned for A: $1, $2",
             "Columns pruned for B: $1"}));
@@ -1022,20 +956,14 @@
         pigServer.registerQuery("B = group A by *;");
         pigServer.registerQuery("C = foreach B generate $0;");
         Iterator<Tuple> iter = pigServer.openIterator("C");
-
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
 
-        assertEquals(1, t.size());
-        assertEquals("(1,2,3)", t.get(0).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-
-        assertEquals(1, t.size());
-        assertEquals("(2,5,2)", t.get(0).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "((1,2,3))",
+                "((2,5,2))"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("C")),
+                Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(emptyLogFileMessage());
     }
@@ -1578,21 +1506,16 @@
         Iterator<Tuple> iter = pigServer.openIterator("G");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
-        assertEquals("({(1)},1)", t.toString());
-        
-        assertTrue(iter.hasNext());
-        t = iter.next();
-        assertEquals("({(2),(2)},2)", t.toString());
-        
-        assertTrue(iter.hasNext());
-        t = iter.next();
-        assertEquals("({(3),(3),(3)},3)", t.toString());
 
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "({(1)},1)",
+                "({(2),(2)},2)",
+                "({(3),(3),(3)},3)"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("G")),
+                Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(checkLogFileMessage(new String[]{"Columns pruned for A: $1"}));
-        
         pigServer.getPigContext().getProperties().remove(PigImplConstants.PIG_OPTIMIZER_RULES_KEY);
     }
 
@@ -1604,26 +1527,15 @@
         pigServer.registerQuery("D = foreach C generate $0, $1;");
 
         Iterator<Tuple> iter = pigServer.openIterator("D");
-
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
-        assertEquals(2, t.size());
-        assertEquals("1", t.get(0).toString());
-        assertEquals("{}", t.get(1).toString());
 
-        assertTrue(iter.hasNext());
-        t = iter.next();
-        assertEquals(2, t.size());
-        assertEquals("2", t.get(0).toString());
-        assertEquals("{(1,2,3)}", t.get(1).toString());
-
-        assertTrue(iter.hasNext());
-        t = iter.next();
-        assertEquals(2, t.size());
-        assertEquals("5", t.get(0).toString());
-        assertEquals("{(2,5,2)}", t.get(1).toString());
-
-        assertFalse(iter.hasNext());
+        String[] expected = new String[] {
+                "(1,{})",
+                "(2,{(1,2,3)})",
+                "(5,{(2,5,2)})"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("D"))
+                , Util.isSparkExecType(Util.getLocalTestMode()));
 
         assertTrue(checkLogFileMessage(new String[]{"Columns pruned for B: $0"}));
     }
@@ -1947,12 +1859,14 @@
         Iterator<Tuple> iter = pigServer.openIterator("C");
 
         assertTrue(iter.hasNext());
-        Tuple t = iter.next();
-        assertEquals("(1,2,3,1)", t.toString());
 
-        assertTrue(iter.hasNext());
-        t = iter.next();
-        assertEquals("(2,5,2,2)", t.toString());
+        String[] expected = new String[] {
+                "(1,2,3,1)",
+                "(2,5,2,2)"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache.pig.newplan.logical.Util.translateSchema(pigServer.dumpSchema("C")),
+                Util.isSparkExecType(Util.getLocalTestMode()));
+
 
         assertTrue(emptyLogFileMessage());
     }
diff --git a/test/org/apache/pig/test/TestRank1.java b/test/org/apache/pig/test/TestRank1.java
index 9e4ef62..a80b824 100644
--- a/test/org/apache/pig/test/TestRank1.java
+++ b/test/org/apache/pig/test/TestRank1.java
@@ -23,7 +23,6 @@
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
diff --git a/test/org/apache/pig/test/TestRank2.java b/test/org/apache/pig/test/TestRank2.java
index fc802a9..19de81c 100644
--- a/test/org/apache/pig/test/TestRank2.java
+++ b/test/org/apache/pig/test/TestRank2.java
@@ -23,7 +23,6 @@
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.pig.PigServer;
 import org.apache.pig.builtin.mock.Storage.Data;
diff --git a/test/org/apache/pig/test/TestRank3.java b/test/org/apache/pig/test/TestRank3.java
index 43af10d..3e7db70 100644
--- a/test/org/apache/pig/test/TestRank3.java
+++ b/test/org/apache/pig/test/TestRank3.java
@@ -23,7 +23,6 @@
 
 import java.io.IOException;
 import java.util.List;
-import java.util.Set;
 
 import org.apache.pig.PigServer;
 import org.apache.pig.backend.executionengine.ExecException;
diff --git a/test/org/apache/pig/test/TestSecondarySort.java b/test/org/apache/pig/test/TestSecondarySort.java
index 4bc3f65..ddad937 100644
--- a/test/org/apache/pig/test/TestSecondarySort.java
+++ b/test/org/apache/pig/test/TestSecondarySort.java
@@ -18,7 +18,6 @@
 package org.apache.pig.test;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
@@ -37,6 +36,7 @@
 import org.apache.pig.impl.logicalLayer.schema.Schema;
 import org.apache.pig.impl.plan.VisitorException;
 import org.junit.AfterClass;
+import org.junit.Assume;
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -237,11 +237,11 @@
         pigServer
                 .registerQuery("D = foreach C { E = limit A 10; F = E.a1; G = DISTINCT F; generate group, COUNT(G);};");
         Iterator<Tuple> iter = pigServer.openIterator("D");
-        assertTrue(iter.hasNext());
-        assertEquals("(2,1)", iter.next().toString());
-        assertTrue(iter.hasNext());
-        assertEquals("(1,2)", iter.next().toString());
-        assertFalse(iter.hasNext());
+        String[] expectedRes = new String[]{"(2,1)","(1,2)"};
+        Schema s = pigServer.dumpSchema("D");
+        Util.checkQueryOutputs(iter,expectedRes,org.apache
+                .pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(cluster.getExecType()));
         Util.deleteFile(cluster, file1ClusterPath);
         Util.deleteFile(cluster, file2ClusterPath);
     }
@@ -265,11 +265,14 @@
         pigServer.registerQuery("B = group A by $0 parallel 2;");
         pigServer.registerQuery("C = foreach B { D = distinct A; generate group, D;};");
         Iterator<Tuple> iter = pigServer.openIterator("C");
-        assertTrue(iter.hasNext());
-        assertEquals("(2,{(2,3,4)})", iter.next().toString());
-        assertTrue(iter.hasNext());
-        assertEquals("(1,{(1,2,3),(1,2,4),(1,3,4)})", iter.next().toString());
-        assertFalse(iter.hasNext());
+        Schema s = pigServer.dumpSchema("C");
+        String expected[] = {
+                "(2,{(2,3,4)})",
+                "(1,{(1,2,3),(1,2,4),(1,3,4)})"
+        };
+        Util.checkQueryOutputs(iter, expected, org.apache
+                .pig.newplan.logical.Util.translateSchema(s),
+                Util.isSparkExecType(Util.getLocalTestMode()));
         Util.deleteFile(cluster, clusterPath);
     }
 
@@ -359,15 +362,10 @@
         pigServer.registerQuery("D = ORDER C BY group;");
         pigServer.registerQuery("E = foreach D { F = limit A 10; G = ORDER F BY a2; generate group, COUNT(G);};");
         Iterator<Tuple> iter = pigServer.openIterator("E");
-        assertTrue(iter.hasNext());
-        assertEquals("((1,2),4)", iter.next().toString());
-        assertTrue(iter.hasNext());
-        assertEquals("((1,3),1)", iter.next().toString());
-        assertTrue(iter.hasNext());
-        assertEquals("((1,4),0)", iter.next().toString());
-        assertTrue(iter.hasNext());
-        assertEquals("((2,3),1)", iter.next().toString());
-        assertFalse(iter.hasNext());
+        Schema s = pigServer.dumpSchema("E");
+        String[] expectedRes = new String[]{"((1,2),4)","((1,3),1)","((1,4),0)","((2,3),1)"};
+        Util.checkQueryOutputs(iter, expectedRes,org.apache.pig.newplan.logical.Util.translateSchema(s),
+        		Util.isSparkExecType(cluster.getExecType()));
         Util.deleteFile(cluster, clusterPath1);
         Util.deleteFile(cluster, clusterPath2);
     }
@@ -515,6 +513,8 @@
     @Test
     // Once custom partitioner is used, we cannot use secondary key optimizer, see PIG-3827
     public void testCustomPartitionerWithSort() throws Exception {
+        Assume.assumeFalse("Skip this test for Spark", Util.isSparkExecType(cluster.getExecType()));
+
         File tmpFile1 = Util.createTempFileDelOnExit("test", "txt");
         PrintStream ps1 = new PrintStream(new FileOutputStream(tmpFile1));
         ps1.println("1\t2\t3");
diff --git a/test/org/apache/pig/test/TestStoreBase.java b/test/org/apache/pig/test/TestStoreBase.java
index eb3b253..029269d 100644
--- a/test/org/apache/pig/test/TestStoreBase.java
+++ b/test/org/apache/pig/test/TestStoreBase.java
@@ -144,24 +144,66 @@
         String outputFileName2 = TESTDIR + "/TestStore-output-" + new Random().nextLong() + ".txt";
 
         Map<String, Boolean> filesToVerify = new HashMap<String, Boolean>();
-        filesToVerify.put(outputFileName1 + "_cleanupOnFailure_succeeded1", Boolean.TRUE);
-        filesToVerify.put(outputFileName2 + "_cleanupOnFailure_succeeded2", Boolean.TRUE);
-        filesToVerify.put(outputFileName1 + "_cleanupOnFailure_failed1", Boolean.FALSE);
-        filesToVerify.put(outputFileName2 + "_cleanupOnFailure_failed2", Boolean.FALSE);
-        filesToVerify.put(DummyOutputCommitter.FILE_SETUPJOB_CALLED + "1", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_SETUPJOB_CALLED + "2", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_SETUPTASK_CALLED + "1", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_SETUPTASK_CALLED + "2", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_COMMITTASK_CALLED + "1", Boolean.FALSE);
-        filesToVerify.put(DummyOutputCommitter.FILE_COMMITTASK_CALLED + "2", Boolean.FALSE);
-        filesToVerify.put(DummyOutputCommitter.FILE_ABORTTASK_CALLED + "1", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_ABORTTASK_CALLED + "2", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_COMMITJOB_CALLED + "1", Boolean.FALSE);
-        filesToVerify.put(DummyOutputCommitter.FILE_COMMITJOB_CALLED + "2", Boolean.FALSE);
-        filesToVerify.put(DummyOutputCommitter.FILE_ABORTJOB_CALLED + "1", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_ABORTJOB_CALLED + "2", Boolean.TRUE);
-        filesToVerify.put(DummyOutputCommitter.FILE_CLEANUPJOB_CALLED + "1", Boolean.FALSE);
-        filesToVerify.put(DummyOutputCommitter.FILE_CLEANUPJOB_CALLED + "2", Boolean.FALSE);
+        if (mode.toString().startsWith("SPARK")) {
+            filesToVerify.put(outputFileName1 + "_cleanupOnFailure_succeeded1", Boolean.TRUE);
+            filesToVerify.put(outputFileName2 + "_cleanupOnFailure_succeeded2", Boolean.TRUE);
+            filesToVerify.put(outputFileName1 + "_cleanupOnFailure_failed1", Boolean.FALSE);
+            filesToVerify.put(outputFileName2 + "_cleanupOnFailure_failed2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPJOB_CALLED + "1", Boolean.TRUE);
+             /* A = load xx;
+             store A into '1.out' using DummyStore('true','1');   -- first job should fail
+             store A into '2.out' using DummyStore('false','1');  -- second job should success
+             After multiQuery optimization the spark plan will be:
+           Split - scope-14
+            |   |
+            |   a: Store(hdfs://1.out:myudfs.DummyStore('true','1')) - scope-4
+            |   |
+            |   a: Store(hdfs://2.out:myudfs.DummyStore('false','1')) - scope-7
+            |
+            |---a: Load(hdfs://zly2.sh.intel.com:8020/user/root/multiStore.txt:org.apache.pig.builtin.PigStorage) - scope-0------
+            In current code base, once the first job fails, the second job will not be executed.
+            the FILE_SETUPJOB_CALLED of second job will not exist.
+            I explain more detail in PIG-4243
+            */
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPJOB_CALLED + "2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPTASK_CALLED + "1", Boolean.TRUE);
+                /*
+            In current code base, once the first job fails, the second job will not be executed.
+            the FILE_SETUPTASK_CALLED of second job will not exist.
+            */
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPTASK_CALLED + "2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITTASK_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITTASK_CALLED + "2", Boolean.FALSE);
+            // OutputCommitter.abortTask will not be invoked in spark mode. Detail see SPARK-7953
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTTASK_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTTASK_CALLED + "2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITJOB_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITJOB_CALLED + "2", Boolean.FALSE);
+            // OutputCommitter.abortJob will not be invoked in spark mode. Detail see SPARK-7953
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTJOB_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTJOB_CALLED + "2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_CLEANUPJOB_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_CLEANUPJOB_CALLED + "2", Boolean.FALSE);
+        } else {
+            filesToVerify.put(outputFileName1 + "_cleanupOnFailure_succeeded1", Boolean.TRUE);
+            filesToVerify.put(outputFileName2 + "_cleanupOnFailure_succeeded2", Boolean.TRUE);
+            filesToVerify.put(outputFileName1 + "_cleanupOnFailure_failed1", Boolean.FALSE);
+            filesToVerify.put(outputFileName2 + "_cleanupOnFailure_failed2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPJOB_CALLED + "1", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPJOB_CALLED + "2", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPTASK_CALLED + "1", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_SETUPTASK_CALLED + "2", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITTASK_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITTASK_CALLED + "2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTTASK_CALLED + "1", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTTASK_CALLED + "2", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITJOB_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_COMMITJOB_CALLED + "2", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTJOB_CALLED + "1", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_ABORTJOB_CALLED + "2", Boolean.TRUE);
+            filesToVerify.put(DummyOutputCommitter.FILE_CLEANUPJOB_CALLED + "1", Boolean.FALSE);
+            filesToVerify.put(DummyOutputCommitter.FILE_CLEANUPJOB_CALLED + "2", Boolean.FALSE);
+        }
 
         String[] inputData = new String[]{"hello\tworld", "bye\tworld"};
 
diff --git a/test/org/apache/pig/test/TezMiniCluster.java b/test/org/apache/pig/test/TezMiniCluster.java
index 0bf7c5a..0d8871f 100644
--- a/test/org/apache/pig/test/TezMiniCluster.java
+++ b/test/org/apache/pig/test/TezMiniCluster.java
@@ -41,19 +41,12 @@
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
 import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 
-public class TezMiniCluster extends MiniGenericCluster {
-    private static final File CONF_DIR = new File("build/classes");
+public class TezMiniCluster extends YarnMiniCluster {
+
     private static final File TEZ_LIB_DIR = new File("build/ivy/lib/Pig");
     private static final File TEZ_CONF_FILE = new File(CONF_DIR, "tez-site.xml");
-    private static final File CORE_CONF_FILE = new File(CONF_DIR, "core-site.xml");
-    private static final File HDFS_CONF_FILE = new File(CONF_DIR, "hdfs-site.xml");
-    private static final File MAPRED_CONF_FILE = new File(CONF_DIR, "mapred-site.xml");
-    private static final File YARN_CONF_FILE = new File(CONF_DIR, "yarn-site.xml");
-    private static final ExecType TEZ = new TezExecType();
 
-    protected MiniMRYarnCluster m_mr = null;
-    private Configuration m_dfs_conf = null;
-    private Configuration m_mr_conf = null;
+    private static final ExecType TEZ = new TezExecType();
 
     @Override
     public ExecType getExecType() {
@@ -61,66 +54,9 @@
     }
 
     @Override
-    public void setupMiniDfsAndMrClusters() {
+    protected void setupMiniDfsAndMrClusters() {
+        super.setupMiniDfsAndMrClusters();
         try {
-            deleteConfFiles();
-            CONF_DIR.mkdirs();
-
-            // Build mini DFS cluster
-            Configuration hdfsConf = new Configuration();
-            m_dfs = new MiniDFSCluster.Builder(hdfsConf)
-                    .numDataNodes(2)
-                    .format(true)
-                    .racks(null)
-                    .build();
-            m_fileSys = m_dfs.getFileSystem();
-            m_dfs_conf = m_dfs.getConfiguration(0);
-            //Create user home directory
-            m_fileSys.mkdirs(m_fileSys.getWorkingDirectory());
-
-            // Write core-site.xml
-            Configuration core_site = new Configuration(false);
-            core_site.set(FileSystem.FS_DEFAULT_NAME_KEY, m_dfs_conf.get(FileSystem.FS_DEFAULT_NAME_KEY));
-            core_site.writeXml(new FileOutputStream(CORE_CONF_FILE));
-
-            Configuration hdfs_site = new Configuration(false);
-            for (Entry<String, String> conf : m_dfs_conf) {
-                if (ArrayUtils.contains(m_dfs_conf.getPropertySources(conf.getKey()), "programatically")) {
-                    hdfs_site.set(conf.getKey(), m_dfs_conf.getRaw(conf.getKey()));
-                }
-            }
-            hdfs_site.writeXml(new FileOutputStream(HDFS_CONF_FILE));
-
-            // Build mini YARN cluster
-            m_mr = new MiniMRYarnCluster("PigMiniCluster", 2);
-            m_mr.init(m_dfs_conf);
-            m_mr.start();
-            m_mr_conf = m_mr.getConfig();
-            File libDir = new File(System.getProperty("ivy.lib.dir", "build/ivy/lib/Pig"));
-            File classesDir = new File(System.getProperty("build.classes", "build/classes"));
-            File testClassesDir = new File(System.getProperty("test.build.classes", "test/build/classes"));
-            String classpath = libDir.getAbsolutePath() + "/*"
-                    + File.pathSeparator + classesDir.getAbsolutePath()
-                    + File.pathSeparator + testClassesDir.getAbsolutePath();
-            m_mr_conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, classpath);
-            m_mr_conf.set(MRJobConfig.MAP_JAVA_OPTS, "-Xmx512m");
-            m_mr_conf.set(MRJobConfig.REDUCE_JAVA_OPTS, "-Xmx512m");
-
-            Configuration mapred_site = new Configuration(false);
-            Configuration yarn_site = new Configuration(false);
-            for (Entry<String, String> conf : m_mr_conf) {
-                if (ArrayUtils.contains(m_mr_conf.getPropertySources(conf.getKey()), "programatically")) {
-                    if (conf.getKey().contains("yarn")) {
-                        yarn_site.set(conf.getKey(), m_mr_conf.getRaw(conf.getKey()));
-                    } else if (!conf.getKey().startsWith("dfs")){
-                        mapred_site.set(conf.getKey(), m_mr_conf.getRaw(conf.getKey()));
-                    }
-                }
-            }
-
-            mapred_site.writeXml(new FileOutputStream(MAPRED_CONF_FILE));
-            yarn_site.writeXml(new FileOutputStream(YARN_CONF_FILE));
-
             // Write tez-site.xml
             Configuration tez_conf = new Configuration(false);
             // TODO PIG-3659 - Remove this once memory management is fixed
@@ -150,12 +86,9 @@
                 }
             }
 
-            m_conf = m_mr_conf;
             // Turn FetchOptimizer off so that we can actually test Tez
             m_conf.set(PigConfiguration.PIG_OPT_FETCH, System.getProperty("test.opt.fetch", "false"));
 
-            System.setProperty("junit.hadoop.conf", CONF_DIR.getPath());
-            System.setProperty("hadoop.log.dir", "build/test/logs");
         } catch (IOException e) {
             throw new RuntimeException(e);
         }
@@ -169,29 +102,15 @@
 
     @Override
     protected void shutdownMiniMrClusters() {
-        deleteConfFiles();
-        if (m_mr != null) {
-            m_mr.stop();
-            m_mr = null;
-        }
+        super.shutdownMiniMrClusters();
     }
 
-    private void deleteConfFiles() {
+    @Override
+    protected void deleteConfFiles() {
+        super.deleteConfFiles();
         if(TEZ_CONF_FILE.exists()) {
             TEZ_CONF_FILE.delete();
         }
-        if(CORE_CONF_FILE.exists()) {
-            CORE_CONF_FILE.delete();
-        }
-        if(HDFS_CONF_FILE.exists()) {
-            HDFS_CONF_FILE.delete();
-        }
-        if(MAPRED_CONF_FILE.exists()) {
-            MAPRED_CONF_FILE.delete();
-        }
-        if(YARN_CONF_FILE.exists()) {
-            YARN_CONF_FILE.delete();
-        }
     }
 
     static public Launcher getLauncher() {
diff --git a/test/org/apache/pig/test/Util.java b/test/org/apache/pig/test/Util.java
index 18b241e..3b71d3a 100644
--- a/test/org/apache/pig/test/Util.java
+++ b/test/org/apache/pig/test/Util.java
@@ -585,6 +585,29 @@
          checkQueryOutputsAfterSort(actualResList, expectedResList);
      }
 
+    /**
+     * Helper function to check if the result of Pig Query is in line with expected results.
+     * It sorts actual and expected results before comparison.
+     * The tuple size in the tuple list can vary. Pass by a two-dimension array, it will be converted to be a tuple list.
+     * e.g.  expectedTwoDimensionObjects is [{{10, "will_join", 10, "will_join"}, {11, "will_not_join", null}, {null, 12, "will_not_join"}}],
+     * the field size of these 3 tuples are [4,3,3]
+     *
+     * @param actualResultsIt
+     * @param expectedTwoDimensionObjects represents a tuple list, in which the tuple can have variable size.
+     */
+    static public void checkQueryOutputsAfterSort(Iterator<Tuple> actualResultsIt,
+                                                  Object[][] expectedTwoDimensionObjects) {
+        List<Tuple> expectedResTupleList = new ArrayList<Tuple>();
+        for (int i = 0; i < expectedTwoDimensionObjects.length; ++i) {
+            Tuple t = TupleFactory.getInstance().newTuple();
+            for (int j = 0; j < expectedTwoDimensionObjects[i].length; ++j) {
+                t.append(expectedTwoDimensionObjects[i][j]);
+            }
+            expectedResTupleList.add(t);
+        }
+        checkQueryOutputsAfterSort(actualResultsIt, expectedResTupleList);
+    }
+
      static public void checkQueryOutputsAfterSort(
             List<Tuple> actualResList, List<Tuple> expectedResList) {
          Collections.sort(actualResList);
@@ -592,7 +615,7 @@
 
          Assert.assertEquals("Comparing actual and expected results. ",
                  expectedResList, actualResList);
-
+         
     }
 
     /**
@@ -757,7 +780,8 @@
         if(Util.WINDOWS){
             filename = filename.replace('\\','/');
         }
-        if (context.getExecType() == ExecType.MAPREDUCE || context.getExecType().name().equals("TEZ")) {
+        if (context.getExecType() == ExecType.MAPREDUCE || context.getExecType().name().equals("TEZ") ||
+                context.getExecType().name().equals("SPARK")) {
             return FileLocalizer.hadoopify(filename, context);
         } else if (context.getExecType().isLocal()) {
             return filename;
@@ -1195,7 +1219,7 @@
     }
 
 
-    static private void convertBagToSortedBag(Tuple t) {
+    static public void convertBagToSortedBag(Tuple t) {
         for (int i=0;i<t.size();i++) {
            Object obj = null;
            try {
@@ -1310,6 +1334,72 @@
         return false;
     }
 
+    public static boolean isSparkExecType(ExecType execType) {
+        if (execType.name().toLowerCase().startsWith("spark")) {
+            return true;
+        }
+
+        return false;
+    }
+
+    public static void sortQueryOutputsIfNeed(List<Tuple> actualResList, boolean toSort){
+        if( toSort == true) {
+            for (Tuple t : actualResList) {
+                Util.convertBagToSortedBag(t);
+            }
+            Collections.sort(actualResList);
+        }
+    }
+
+    public static void checkQueryOutputs(Iterator<Tuple> actualResults, List<Tuple> expectedResults, boolean checkAfterSort) {
+        if (checkAfterSort) {
+            checkQueryOutputsAfterSort(actualResults, expectedResults);
+        } else {
+            checkQueryOutputs(actualResults, expectedResults);
+        }
+    }
+
+    static public void checkQueryOutputs(Iterator<Tuple> actualResultsIt,
+                                         String[] expectedResArray, LogicalSchema schema, boolean
+            checkAfterSort) throws IOException {
+        if (checkAfterSort) {
+            checkQueryOutputsAfterSortRecursive(actualResultsIt,
+                    expectedResArray, schema);
+        } else {
+            checkQueryOutputs(actualResultsIt,
+                    expectedResArray, schema);
+        }
+    }
+
+    static void checkQueryOutputs(Iterator<Tuple> actualResultsIt,
+                                         String[] expectedResArray, LogicalSchema schema) throws IOException {
+        LogicalFieldSchema fs = new LogicalFieldSchema("tuple", schema, DataType.TUPLE);
+        ResourceFieldSchema rfs = new ResourceFieldSchema(fs);
+
+        LoadCaster caster = new Utf8StorageConverter();
+        List<Tuple> actualResList = new ArrayList<Tuple>();
+        while (actualResultsIt.hasNext()) {
+            actualResList.add(actualResultsIt.next());
+        }
+
+        List<Tuple> expectedResList = new ArrayList<Tuple>();
+        for (String str : expectedResArray) {
+            Tuple newTuple = caster.bytesToTuple(str.getBytes(), rfs);
+            expectedResList.add(newTuple);
+        }
+
+        for (Tuple t : actualResList) {
+            convertBagToSortedBag(t);
+        }
+
+        for (Tuple t : expectedResList) {
+            convertBagToSortedBag(t);
+        }
+
+        Assert.assertEquals("Comparing actual and expected results. ",
+                expectedResList, actualResList);
+    }
+
     public static void assertParallelValues(long defaultParallel,
                                              long requestedParallel,
                                              long estimatedParallel,
@@ -1399,11 +1489,15 @@
 
     public static ExecType getLocalTestMode() throws Exception {
         String execType = System.getProperty("test.exec.type");
-        if (execType!=null && execType.equals("tez")) {
-            return ExecTypeProvider.fromString("tez_local");
-        } else {
-            return ExecTypeProvider.fromString("local");
+        if (execType != null) {
+            if (execType.equals("tez")) {
+                return ExecTypeProvider.fromString("tez_local");
+            } else if (execType.equals("spark")) {
+                return ExecTypeProvider.fromString("spark_local");
+            }
         }
+
+        return ExecTypeProvider.fromString("local");
     }
 
     public static void createLogAppender(String appenderName, Writer writer, Class...clazzes) {
diff --git a/test/org/apache/pig/test/YarnMiniCluster.java b/test/org/apache/pig/test/YarnMiniCluster.java
new file mode 100644
index 0000000..6645557
--- /dev/null
+++ b/test/org/apache/pig/test/YarnMiniCluster.java
@@ -0,0 +1,140 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you 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.
+ */
+package org.apache.pig.test;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.v2.MiniMRYarnCluster;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.pig.ExecType;
+
+public abstract class YarnMiniCluster extends MiniGenericCluster {
+    protected static final File CONF_DIR = new File("build/classes");
+    protected static final File CORE_CONF_FILE = new File(CONF_DIR, "core-site.xml");
+    protected static final File HDFS_CONF_FILE = new File(CONF_DIR, "hdfs-site.xml");
+    protected static final File MAPRED_CONF_FILE = new File(CONF_DIR, "mapred-site.xml");
+    protected static final File YARN_CONF_FILE = new File(CONF_DIR, "yarn-site.xml");
+
+
+    protected Configuration m_dfs_conf = null;
+    protected MiniMRYarnCluster m_mr = null;
+    protected Configuration m_mr_conf = null;
+
+
+    @Override
+    protected void setupMiniDfsAndMrClusters() {
+        try {
+            deleteConfFiles();
+            CONF_DIR.mkdirs();
+
+            // Build mini DFS cluster
+            Configuration hdfsConf = new Configuration();
+            m_dfs = new MiniDFSCluster.Builder(hdfsConf)
+                    .numDataNodes(2)
+                    .format(true)
+                    .racks(null)
+                    .build();
+            m_fileSys = m_dfs.getFileSystem();
+            m_dfs_conf = m_dfs.getConfiguration(0);
+
+            //Create user home directory
+            m_fileSys.mkdirs(m_fileSys.getWorkingDirectory());
+            // Write core-site.xml
+            Configuration core_site = new Configuration(false);
+            core_site.set(FileSystem.FS_DEFAULT_NAME_KEY, m_dfs_conf.get(FileSystem.FS_DEFAULT_NAME_KEY));
+            core_site.writeXml(new FileOutputStream(CORE_CONF_FILE));
+
+            Configuration hdfs_site = new Configuration(false);
+            for (Map.Entry<String, String> conf : m_dfs_conf) {
+                if (ArrayUtils.contains(m_dfs_conf.getPropertySources(conf.getKey()), "programatically")) {
+                    hdfs_site.set(conf.getKey(), m_dfs_conf.getRaw(conf.getKey()));
+                }
+            }
+            hdfs_site.writeXml(new FileOutputStream(HDFS_CONF_FILE));
+
+            // Build mini YARN cluster
+            m_mr = new MiniMRYarnCluster("PigMiniCluster", 2);
+            m_mr.init(m_dfs_conf);
+            m_mr.start();
+            m_mr_conf = m_mr.getConfig();
+            File libDir = new File(System.getProperty("ivy.lib.dir", "build/ivy/lib/Pig"));
+            File classesDir = new File(System.getProperty("build.classes", "build/classes"));
+            File testClassesDir = new File(System.getProperty("test.build.classes", "test/build/classes"));
+            String classpath = libDir.getAbsolutePath() + "/*"
+                + File.pathSeparator + classesDir.getAbsolutePath()
+                + File.pathSeparator + testClassesDir.getAbsolutePath();
+            m_mr_conf.set(YarnConfiguration.YARN_APPLICATION_CLASSPATH, classpath);
+            m_mr_conf.set(MRJobConfig.MAP_JAVA_OPTS, "-Xmx512m");
+            m_mr_conf.set(MRJobConfig.REDUCE_JAVA_OPTS, "-Xmx512m");
+
+            Configuration mapred_site = new Configuration(false);
+            Configuration yarn_site = new Configuration(false);
+            for (Map.Entry<String, String> conf : m_mr_conf) {
+                if (ArrayUtils.contains(m_mr_conf.getPropertySources(conf.getKey()), "programatically")) {
+                    if (conf.getKey().contains("yarn")) {
+                        yarn_site.set(conf.getKey(), m_mr_conf.getRaw(conf.getKey()));
+                    } else if (!conf.getKey().startsWith("dfs")) {
+                        mapred_site.set(conf.getKey(), m_mr_conf.getRaw(conf.getKey()));
+                    }
+                }
+            }
+
+            mapred_site.writeXml(new FileOutputStream(MAPRED_CONF_FILE));
+            yarn_site.writeXml(new FileOutputStream(YARN_CONF_FILE));
+
+            m_conf = m_mr_conf;
+            System.setProperty("junit.hadoop.conf", CONF_DIR.getPath());
+            System.setProperty("hadoop.log.dir", "build/test/logs");
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+
+        }
+    }
+
+    protected void deleteConfFiles() {
+        if(CORE_CONF_FILE.exists()) {
+            CORE_CONF_FILE.delete();
+        }
+        if(HDFS_CONF_FILE.exists()) {
+            HDFS_CONF_FILE.delete();
+        }
+        if(MAPRED_CONF_FILE.exists()) {
+            MAPRED_CONF_FILE.delete();
+        }
+        if(YARN_CONF_FILE.exists()) {
+            YARN_CONF_FILE.delete();
+        }
+    }
+
+    @Override
+    protected void shutdownMiniMrClusters() {
+        deleteConfFiles();
+        if (m_mr != null) {
+            m_mr.stop();
+            m_mr = null;
+        }
+    }
+}
\ No newline at end of file
diff --git a/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-dot.gld b/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-dot.gld
new file mode 100644
index 0000000..7120a8b
--- /dev/null
+++ b/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-dot.gld
@@ -0,0 +1,71 @@
+digraph plan {
+compound=true;
+node [shape=rect];
+s487399236_in [label="", style=invis, height=0, width=0];
+s487399236_out [label="", style=invis, height=0, width=0];
+subgraph cluster_487399236 {
+label="Spark(-1,PigStorage)"; style="filled"; fillcolor="#EEEEEE"labelloc=b;
+s0_in [label="", style=invis, height=0, width=0];
+s0_out [label="", style=invis, height=0, width=0];
+subgraph cluster_0 {
+label="spark"; style="filled"; fillcolor="white"labelloc=b;
+487399275 [label="a: Load(file:///tmp/input,\norg.apache.pig.builtin.PigStorage)", style="filled", fillcolor="gray"];
+s487399268_in [label="", style=invis, height=0, width=0];
+s487399268_out [label="", style=invis, height=0, width=0];
+subgraph cluster_487399268 {
+label="a: New For Each(false,false)[bag]"labelloc=b;
+487399274 [label="Project[bytearray][0]"];
+487399273 [label="Cast[int]"];
+487399274 -> 487399273
+s487399268_in -> 487399274 [style=invis];
+487399270 [label="Cast[int]"];
+487399271 [label="Project[bytearray][1]"];
+487399271 -> 487399270
+s487399268_in -> 487399271 [style=invis];
+};
+487399273 -> s487399268_out [style=invis];
+487399270 -> s487399268_out [style=invis];
+487399267 [label="a: Store(file:///tmp/pigoutput,\norg.apache.pig.builtin.PigStorage)", style="filled", fillcolor="gray"];
+487399275 -> s487399268_in [lhead=cluster_487399268]
+s487399268_out -> 487399267
+s0_in -> 487399275 [style=invis];
+};
+487399267 -> s0_out [style=invis];
+s487399236_in -> s0_in [style=invis];
+};
+s0_out -> s487399236_out [style=invis];
+s487399235_in [label="", style=invis, height=0, width=0];
+s487399235_out [label="", style=invis, height=0, width=0];
+subgraph cluster_487399235 {
+label="Spark(-1,PigStorage)"; style="filled"; fillcolor="#EEEEEE"labelloc=b;
+s1_in [label="", style=invis, height=0, width=0];
+s1_out [label="", style=invis, height=0, width=0];
+subgraph cluster_1 {
+label="spark"; style="filled"; fillcolor="white"labelloc=b;
+s487399238_in [label="", style=invis, height=0, width=0];
+s487399238_out [label="", style=invis, height=0, width=0];
+subgraph cluster_487399238 {
+label="b: New For Each(false,false)[bag]"labelloc=b;
+487399244 [label="Project[bytearray][0]"];
+487399243 [label="Cast[int]"];
+487399244 -> 487399243
+s487399238_in -> 487399244 [style=invis];
+487399241 [label="Project[bytearray][1]"];
+487399240 [label="Cast[int]"];
+487399241 -> 487399240
+s487399238_in -> 487399241 [style=invis];
+};
+487399243 -> s487399238_out [style=invis];
+487399240 -> s487399238_out [style=invis];
+487399237 [label="b: Store(file:///tmp/pigoutput1,\norg.apache.pig.builtin.PigStorage)", style="filled", fillcolor="gray"];
+487399266 [label="b: Load(file:///tmp/pigoutput,\norg.apache.pig.builtin.PigStorage)", style="filled", fillcolor="gray"];
+s487399238_out -> 487399237
+487399266 -> s487399238_in [lhead=cluster_487399238]
+s1_in -> 487399266 [style=invis];
+};
+487399237 -> s1_out [style=invis];
+s487399235_in -> s1_in [style=invis];
+};
+s1_out -> s487399235_out [style=invis];
+s487399236_out -> s487399235_in [lhead=cluster_487399235]
+}
diff --git a/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-text.gld b/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-text.gld
new file mode 100644
index 0000000..cc47933
--- /dev/null
+++ b/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-text.gld
@@ -0,0 +1,33 @@
+#--------------------------------------------------
+# Spark Plan
+#--------------------------------------------------
+
+Spark node scope-18
+a: Store(file:///tmp/pigoutput:org.apache.pig.builtin.PigStorage) - scope-8
+|
+|---a: New For Each(false,false)[bag] - scope-7
+    |   |
+    |   Cast[int] - scope-2
+    |   |
+    |   |---Project[bytearray][0] - scope-1
+    |   |
+    |   Cast[int] - scope-5
+    |   |
+    |   |---Project[bytearray][1] - scope-4
+    |
+    |---a: Load(file:///tmp/input:org.apache.pig.builtin.PigStorage) - scope-0--------
+
+Spark node scope-19
+b: Store(file:///tmp/pigoutput1:org.apache.pig.builtin.PigStorage) - scope-17
+|
+|---b: New For Each(false,false)[bag] - scope-16
+    |   |
+    |   Cast[int] - scope-11
+    |   |
+    |   |---Project[bytearray][0] - scope-10
+    |   |
+    |   Cast[int] - scope-14
+    |   |
+    |   |---Project[bytearray][1] - scope-13
+    |
+    |---b: Load(file:///tmp/pigoutput:org.apache.pig.builtin.PigStorage) - scope-9--------
diff --git a/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-xml.gld b/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-xml.gld
new file mode 100644
index 0000000..3e31af5
--- /dev/null
+++ b/test/org/apache/pig/test/data/GoldenFiles/spark/SPARKC-LoadStore-1-xml.gld
@@ -0,0 +1,49 @@
+<sparkPlan>
+  <sparkNode scope="38">
+    <POStore scope="28">
+      <alias>a</alias>
+      <storeFile>file:///tmp/pigoutput</storeFile>
+      <isTmpStore>false</isTmpStore>
+      <POForEach scope="27">
+        <alias>a</alias>
+        <POCast scope="22">
+          <alias>x</alias>
+          <POProject scope="21"/>
+        </POCast>
+        <POCast scope="25">
+          <alias>y</alias>
+          <POProject scope="24"/>
+        </POCast>
+        <POLoad scope="20">
+          <alias>a</alias>
+          <loadFile>file:///tmp/input</loadFile>
+          <isTmpLoad>false</isTmpLoad>
+        </POLoad>
+      </POForEach>
+    </POStore>
+  </sparkNode>
+  <sparkNode scope="39">
+    <POStore scope="37">
+      <alias>b</alias>
+      <storeFile>file:///tmp/pigoutput1</storeFile>
+      <isTmpStore>false</isTmpStore>
+      <POForEach scope="36">
+        <alias>b</alias>
+        <POCast scope="31">
+          <alias>x</alias>
+          <POProject scope="30"/>
+        </POCast>
+        <POCast scope="34">
+          <alias>y</alias>
+          <POProject scope="33"/>
+        </POCast>
+        <POLoad scope="29">
+          <alias>b</alias>
+          <loadFile>file:///tmp/pigoutput</loadFile>
+          <isTmpLoad>false</isTmpLoad>
+        </POLoad>
+      </POForEach>
+    </POStore>
+  </sparkNode>
+</sparkPlan>
+