Skip to content

Commit 03b62b0

Browse files
committed
Merge pull request #7 from apache/master
Update
2 parents 8a00106 + febafef commit 03b62b0

File tree

95 files changed

+1912
-1059
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

95 files changed

+1912
-1059
lines changed

assembly/pom.xml

Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -88,6 +88,20 @@
8888

8989
<build>
9090
<plugins>
91+
<plugin>
92+
<groupId>org.apache.maven.plugins</groupId>
93+
<artifactId>maven-deploy-plugin</artifactId>
94+
<configuration>
95+
<skip>true</skip>
96+
</configuration>
97+
</plugin>
98+
<plugin>
99+
<groupId>org.apache.maven.plugins</groupId>
100+
<artifactId>maven-install-plugin</artifactId>
101+
<configuration>
102+
<skip>true</skip>
103+
</configuration>
104+
</plugin>
91105
<!-- Use the shade plugin to create a big JAR with all the dependencies -->
92106
<plugin>
93107
<groupId>org.apache.maven.plugins</groupId>

bin/compute-classpath.sh

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -43,6 +43,7 @@ if [ -n "$SPARK_PREPEND_CLASSES" ]; then
4343
echo "NOTE: SPARK_PREPEND_CLASSES is set, placing locally compiled Spark"\
4444
"classes ahead of assembly." >&2
4545
CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/classes"
46+
CLASSPATH="$CLASSPATH:$FWDIR/core/target/jars/*"
4647
CLASSPATH="$CLASSPATH:$FWDIR/repl/target/scala-$SCALA_VERSION/classes"
4748
CLASSPATH="$CLASSPATH:$FWDIR/mllib/target/scala-$SCALA_VERSION/classes"
4849
CLASSPATH="$CLASSPATH:$FWDIR/bagel/target/scala-$SCALA_VERSION/classes"

bin/spark-class

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -105,7 +105,7 @@ else
105105
exit 1
106106
fi
107107
fi
108-
JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q')
108+
JAVA_VERSION=$("$RUNNER" -version 2>&1 | sed 's/.* version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q')
109109

110110
# Set JAVA_OPTS to be able to load native libraries and to set heap size
111111
if [ "$JAVA_VERSION" -ge 18 ]; then

core/pom.xml

Lines changed: 27 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -351,6 +351,33 @@
351351
</execution>
352352
</executions>
353353
</plugin>
354+
<!--
355+
Copy guava to the build directory. This is needed to make the SPARK_PREPEND_CLASSES
356+
option work in compute-classpath.sh, since it would put the non-shaded Spark classes in
357+
the runtime classpath.
358+
-->
359+
<plugin>
360+
<groupId>org.apache.maven.plugins</groupId>
361+
<artifactId>maven-dependency-plugin</artifactId>
362+
<executions>
363+
<execution>
364+
<id>copy-dependencies</id>
365+
<phase>package</phase>
366+
<goals>
367+
<goal>copy-dependencies</goal>
368+
</goals>
369+
<configuration>
370+
<outputDirectory>${project.build.directory}</outputDirectory>
371+
<overWriteReleases>false</overWriteReleases>
372+
<overWriteSnapshots>false</overWriteSnapshots>
373+
<overWriteIfNewer>true</overWriteIfNewer>
374+
<useSubDirectoryPerType>true</useSubDirectoryPerType>
375+
<includeArtifactIds>guava</includeArtifactIds>
376+
<silent>true</silent>
377+
</configuration>
378+
</execution>
379+
</executions>
380+
</plugin>
354381
</plugins>
355382

356383
<resources>

core/src/main/scala/org/apache/spark/SparkContext.scala

Lines changed: 0 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -1072,11 +1072,8 @@ class SparkContext(config: SparkConf) extends Logging {
10721072
val callSite = getCallSite
10731073
val cleanedFunc = clean(func)
10741074
logInfo("Starting job: " + callSite.shortForm)
1075-
val start = System.nanoTime
10761075
dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal,
10771076
resultHandler, localProperties.get)
1078-
logInfo(
1079-
"Job finished: " + callSite.shortForm + ", took " + (System.nanoTime - start) / 1e9 + " s")
10801077
rdd.doCheckpoint()
10811078
}
10821079

core/src/main/scala/org/apache/spark/SparkEnv.scala

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -108,6 +108,14 @@ class SparkEnv (
108108
pythonWorkers.get(key).foreach(_.stopWorker(worker))
109109
}
110110
}
111+
112+
private[spark]
113+
def releasePythonWorker(pythonExec: String, envVars: Map[String, String], worker: Socket) {
114+
synchronized {
115+
val key = (pythonExec, envVars)
116+
pythonWorkers.get(key).foreach(_.releaseWorker(worker))
117+
}
118+
}
111119
}
112120

113121
object SparkEnv extends Logging {

core/src/main/scala/org/apache/spark/TaskContext.scala

Lines changed: 15 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer
2121

2222
import org.apache.spark.annotation.DeveloperApi
2323
import org.apache.spark.executor.TaskMetrics
24-
import org.apache.spark.util.TaskCompletionListener
24+
import org.apache.spark.util.{TaskCompletionListenerException, TaskCompletionListener}
2525

2626

2727
/**
@@ -41,7 +41,7 @@ class TaskContext(
4141
val attemptId: Long,
4242
val runningLocally: Boolean = false,
4343
private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty)
44-
extends Serializable {
44+
extends Serializable with Logging {
4545

4646
@deprecated("use partitionId", "0.8.1")
4747
def splitId = partitionId
@@ -103,8 +103,20 @@ class TaskContext(
103103
/** Marks the task as completed and triggers the listeners. */
104104
private[spark] def markTaskCompleted(): Unit = {
105105
completed = true
106+
val errorMsgs = new ArrayBuffer[String](2)
106107
// Process complete callbacks in the reverse order of registration
107-
onCompleteCallbacks.reverse.foreach { _.onTaskCompletion(this) }
108+
onCompleteCallbacks.reverse.foreach { listener =>
109+
try {
110+
listener.onTaskCompletion(this)
111+
} catch {
112+
case e: Throwable =>
113+
errorMsgs += e.getMessage
114+
logError("Error in TaskCompletionListener", e)
115+
}
116+
}
117+
if (errorMsgs.nonEmpty) {
118+
throw new TaskCompletionListenerException(errorMsgs)
119+
}
108120
}
109121

110122
/** Marks the task for interruption, i.e. cancellation. */

core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala

Lines changed: 6 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,6 +17,7 @@
1717

1818
package org.apache.spark.api.java
1919

20+
import java.io.Closeable
2021
import java.util
2122
import java.util.{Map => JMap}
2223

@@ -40,7 +41,9 @@ import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD}
4041
* A Java-friendly version of [[org.apache.spark.SparkContext]] that returns
4142
* [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones.
4243
*/
43-
class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWorkaround {
44+
class JavaSparkContext(val sc: SparkContext)
45+
extends JavaSparkContextVarargsWorkaround with Closeable {
46+
4447
/**
4548
* Create a JavaSparkContext that loads settings from system properties (for instance, when
4649
* launching with ./bin/spark-submit).
@@ -534,6 +537,8 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork
534537
sc.stop()
535538
}
536539

540+
override def close(): Unit = stop()
541+
537542
/**
538543
* Get Spark's home location from either a value set through the constructor,
539544
* or the spark.home Java property, or the SPARK_HOME environment variable

core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala

Lines changed: 50 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -23,6 +23,7 @@ import java.nio.charset.Charset
2323
import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections}
2424

2525
import scala.collection.JavaConversions._
26+
import scala.collection.mutable
2627
import scala.language.existentials
2728
import scala.reflect.ClassTag
2829
import scala.util.{Try, Success, Failure}
@@ -52,6 +53,7 @@ private[spark] class PythonRDD(
5253
extends RDD[Array[Byte]](parent) {
5354

5455
val bufferSize = conf.getInt("spark.buffer.size", 65536)
56+
val reuse_worker = conf.getBoolean("spark.python.worker.reuse", true)
5557

5658
override def getPartitions = parent.partitions
5759

@@ -63,19 +65,26 @@ private[spark] class PythonRDD(
6365
val localdir = env.blockManager.diskBlockManager.localDirs.map(
6466
f => f.getPath()).mkString(",")
6567
envVars += ("SPARK_LOCAL_DIRS" -> localdir) // it's also used in monitor thread
68+
if (reuse_worker) {
69+
envVars += ("SPARK_REUSE_WORKER" -> "1")
70+
}
6671
val worker: Socket = env.createPythonWorker(pythonExec, envVars.toMap)
6772

6873
// Start a thread to feed the process input from our parent's iterator
6974
val writerThread = new WriterThread(env, worker, split, context)
7075

76+
var complete_cleanly = false
7177
context.addTaskCompletionListener { context =>
7278
writerThread.shutdownOnTaskCompletion()
73-
74-
// Cleanup the worker socket. This will also cause the Python worker to exit.
75-
try {
76-
worker.close()
77-
} catch {
78-
case e: Exception => logWarning("Failed to close worker socket", e)
79+
if (reuse_worker && complete_cleanly) {
80+
env.releasePythonWorker(pythonExec, envVars.toMap, worker)
81+
} else {
82+
try {
83+
worker.close()
84+
} catch {
85+
case e: Exception =>
86+
logWarning("Failed to close worker socket", e)
87+
}
7988
}
8089
}
8190

@@ -115,6 +124,10 @@ private[spark] class PythonRDD(
115124
val total = finishTime - startTime
116125
logInfo("Times: total = %s, boot = %s, init = %s, finish = %s".format(total, boot,
117126
init, finish))
127+
val memoryBytesSpilled = stream.readLong()
128+
val diskBytesSpilled = stream.readLong()
129+
context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled
130+
context.taskMetrics.diskBytesSpilled += diskBytesSpilled
118131
read()
119132
case SpecialLengths.PYTHON_EXCEPTION_THROWN =>
120133
// Signals that an exception has been thrown in python
@@ -133,6 +146,7 @@ private[spark] class PythonRDD(
133146
stream.readFully(update)
134147
accumulator += Collections.singletonList(update)
135148
}
149+
complete_cleanly = true
136150
null
137151
}
138152
} catch {
@@ -195,29 +209,45 @@ private[spark] class PythonRDD(
195209
PythonRDD.writeUTF(include, dataOut)
196210
}
197211
// Broadcast variables
198-
dataOut.writeInt(broadcastVars.length)
212+
val oldBids = PythonRDD.getWorkerBroadcasts(worker)
213+
val newBids = broadcastVars.map(_.id).toSet
214+
// number of different broadcasts
215+
val cnt = oldBids.diff(newBids).size + newBids.diff(oldBids).size
216+
dataOut.writeInt(cnt)
217+
for (bid <- oldBids) {
218+
if (!newBids.contains(bid)) {
219+
// remove the broadcast from worker
220+
dataOut.writeLong(- bid - 1) // bid >= 0
221+
oldBids.remove(bid)
222+
}
223+
}
199224
for (broadcast <- broadcastVars) {
200-
dataOut.writeLong(broadcast.id)
201-
dataOut.writeInt(broadcast.value.length)
202-
dataOut.write(broadcast.value)
225+
if (!oldBids.contains(broadcast.id)) {
226+
// send new broadcast
227+
dataOut.writeLong(broadcast.id)
228+
dataOut.writeInt(broadcast.value.length)
229+
dataOut.write(broadcast.value)
230+
oldBids.add(broadcast.id)
231+
}
203232
}
204233
dataOut.flush()
205234
// Serialized command:
206235
dataOut.writeInt(command.length)
207236
dataOut.write(command)
208237
// Data values
209238
PythonRDD.writeIteratorToStream(parent.iterator(split, context), dataOut)
239+
dataOut.writeInt(SpecialLengths.END_OF_DATA_SECTION)
210240
dataOut.flush()
211241
} catch {
212242
case e: Exception if context.isCompleted || context.isInterrupted =>
213243
logDebug("Exception thrown after task completion (likely due to cleanup)", e)
244+
worker.shutdownOutput()
214245

215246
case e: Exception =>
216247
// We must avoid throwing exceptions here, because the thread uncaught exception handler
217248
// will kill the whole executor (see org.apache.spark.executor.Executor).
218249
_exception = e
219-
} finally {
220-
Try(worker.shutdownOutput()) // kill Python worker process
250+
worker.shutdownOutput()
221251
}
222252
}
223253
}
@@ -278,6 +308,14 @@ private object SpecialLengths {
278308
private[spark] object PythonRDD extends Logging {
279309
val UTF8 = Charset.forName("UTF-8")
280310

311+
// remember the broadcasts sent to each worker
312+
private val workerBroadcasts = new mutable.WeakHashMap[Socket, mutable.Set[Long]]()
313+
private def getWorkerBroadcasts(worker: Socket) = {
314+
synchronized {
315+
workerBroadcasts.getOrElseUpdate(worker, new mutable.HashSet[Long]())
316+
}
317+
}
318+
281319
/**
282320
* Adapter for calling SparkContext#runJob from Python.
283321
*

0 commit comments

Comments
 (0)