Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion core/src/main/scala/org/apache/spark/SparkContext.scala
Original file line number Diff line number Diff line change
Expand Up @@ -1044,7 +1044,7 @@ class SparkContext(config: SparkConf) extends Logging {
private[spark] def getCallSite(): CallSite = {
Option(getLocalProperty("externalCallSite")) match {
case Some(callSite) => CallSite(callSite, longForm = "")
case None => Utils.getCallSite
case None => Utils.getCallSite()
}
}

Expand Down
14 changes: 11 additions & 3 deletions core/src/main/scala/org/apache/spark/rdd/RDD.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,7 @@

package org.apache.spark.rdd

import java.util.Random
import java.util.{Properties, Random}

import scala.collection.{mutable, Map}
import scala.collection.mutable.ArrayBuffer
Expand All @@ -41,7 +41,7 @@ import org.apache.spark.partial.CountEvaluator
import org.apache.spark.partial.GroupedCountEvaluator
import org.apache.spark.partial.PartialResult
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.{BoundedPriorityQueue, Utils}
import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite}
import org.apache.spark.util.collection.OpenHashMap
import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, SamplingUtils}

Expand Down Expand Up @@ -1220,7 +1220,15 @@ abstract class RDD[T: ClassTag](
private var storageLevel: StorageLevel = StorageLevel.NONE

/** User code that created this RDD (e.g. `textFile`, `parallelize`). */
@transient private[spark] val creationSite = Utils.getCallSite
@transient private[spark] val creationSite = {
val short: String = sc.getLocalProperty(Utils.CALL_SITE_SHORT)
if (short != null) {
CallSite(short, sc.getLocalProperty(Utils.CALL_SITE_LONG))
} else {
Utils.getCallSite()
}
}

private[spark] def getCreationSite: String = Option(creationSite).map(_.shortForm).getOrElse("")

private[spark] def elementClassTag: ClassTag[T] = classTag[T]
Expand Down
15 changes: 12 additions & 3 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -53,6 +53,9 @@ private[spark] case class CallSite(shortForm: String, longForm: String)
private[spark] object Utils extends Logging {
val random = new Random()

private[spark] val CALL_SITE_SHORT: String = "callSite.short"
private[spark] val CALL_SITE_LONG: String = "callSite.long"

/** Serialize an object using Java serialization */
def serialize[T](o: T): Array[Byte] = {
val bos = new ByteArrayOutputStream()
Expand Down Expand Up @@ -856,13 +859,19 @@ private[spark] object Utils extends Logging {
* finding the call site of a method.
*/
private val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r
val SCALA_CLASS_REGEX = """^scala""".r

private def defaultRegexFunc(className: String): Boolean = {
SPARK_CLASS_REGEX.findFirstIn(className).isDefined ||
SCALA_CLASS_REGEX.findFirstIn(className).isDefined
}

/**
* When called inside a class in the spark package, returns the name of the user code class
* (outside the spark package) that called into Spark, as well as which Spark method they called.
* This is used, for example, to tell users where in their code each RDD got created.
*/
def getCallSite: CallSite = {
def getCallSite(regexFunc: String => Boolean = defaultRegexFunc(_)): CallSite = {
val trace = Thread.currentThread.getStackTrace()
.filterNot { ste:StackTraceElement =>
// When running under some profilers, the current stack trace might contain some bogus
Expand All @@ -883,8 +892,8 @@ private[spark] object Utils extends Logging {

for (el <- trace) {
if (insideSpark) {
if (SPARK_CLASS_REGEX.findFirstIn(el.getClassName).isDefined) {
lastSparkMethod = if (el.getMethodName == "<init>") {
if (regexFunc(el.getClassName)) {
lastSparkMethod = if (el.getMethodName == "<init>") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't think this was intended

// Spark method is a constructor; get its class name
el.getClassName.substring(el.getClassName.lastIndexOf('.') + 1)
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,7 @@ import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver}
import org.apache.spark.streaming.scheduler._
import org.apache.spark.streaming.ui.StreamingTab
import org.apache.spark.util.MetadataCleaner
import org.apache.spark.util.Utils

/**
* Main entry point for Spark Streaming functionality. It provides methods used to create
Expand Down Expand Up @@ -441,6 +441,9 @@ class StreamingContext private[streaming] (
throw new SparkException("StreamingContext has already been stopped")
}
validate()
sc.setCallSite(
Utils.getCallSite(org.apache.spark.streaming.util.Utils.streamingRegexFunc).shortForm
)
scheduler.start()
state = Started
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,7 @@ import org.apache.spark.storage.StorageLevel
import org.apache.spark.streaming._
import org.apache.spark.streaming.StreamingContext._
import org.apache.spark.streaming.scheduler.Job
import org.apache.spark.util.MetadataCleaner
import org.apache.spark.util.{CallSite, Utils, MetadataCleaner}

/**
* A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous
Expand Down Expand Up @@ -106,6 +106,21 @@ abstract class DStream[T: ClassTag] (
/** Return the StreamingContext associated with this DStream */
def context = ssc

/* Find the creation callSite */
val creationSite = Utils.getCallSite(org.apache.spark.streaming.util.Utils.streamingRegexFunc)

/* Store the RDD creation callSite in threadlocal */
private def setRDDCreationCallSite(callSite: CallSite = creationSite) = {
ssc.sparkContext.setLocalProperty(Utils.CALL_SITE_SHORT, callSite.shortForm)
ssc.sparkContext.setLocalProperty(Utils.CALL_SITE_LONG, callSite.longForm)
}

/* Return the current callSite */
private def getRDDCreationCallSite(): CallSite = {
CallSite(ssc.sparkContext.getLocalProperty(Utils.CALL_SITE_SHORT),
ssc.sparkContext.getLocalProperty(Utils.CALL_SITE_LONG))
}

/** Persist the RDDs of this DStream with the given storage level */
def persist(level: StorageLevel): DStream[T] = {
if (this.isInitialized) {
Expand Down Expand Up @@ -288,7 +303,9 @@ abstract class DStream[T: ClassTag] (
// (based on sliding time of this DStream), then generate the RDD
case None => {
if (isTimeValid(time)) {
compute(time) match {
val prevCallSite = getRDDCreationCallSite
setRDDCreationCallSite()
val rddOption = compute(time) match {
case Some(newRDD) =>
if (storageLevel != StorageLevel.NONE) {
newRDD.persist(storageLevel)
Expand All @@ -304,10 +321,12 @@ abstract class DStream[T: ClassTag] (
generatedRDDs.put(time, newRDD)
Some(newRDD)
case None =>
None
return None
}
setRDDCreationCallSite(prevCallSite)
return rddOption
} else {
None
return None
}
}
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,34 @@
/*
* 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.spark.streaming.util

import org.apache.spark.util.Utils.SCALA_CLASS_REGEX

/**
* Utility method used by Spark Streaming.
*/
private[streaming] object Utils {
private val SPARK_STREAMING_CLASS_REGEX = """^org\.apache\.spark""".r
private val SPARK_EXAMPLES_CLASS_REGEX = """^org\.apache\.spark\.examples""".r

def streamingRegexFunc(className: String): Boolean = {
(SPARK_STREAMING_CLASS_REGEX.findFirstIn(className).isDefined &&
!SPARK_EXAMPLES_CLASS_REGEX.findFirstIn(className).isDefined) ||
SCALA_CLASS_REGEX.findFirstIn(className).isDefined
}
}