Skip to content

Commit fa4e5fb

Browse files
author
Andrew Or
committed
Pass in input stream name rather than defining it from within
1 parent 1af0b0e commit fa4e5fb

File tree

21 files changed

+63
-96
lines changed

21 files changed

+63
-96
lines changed

core/src/main/scala/org/apache/spark/rdd/RDDOperationScope.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -96,7 +96,7 @@ private[spark] object RDDOperationScope extends Logging {
9696
sc: SparkContext,
9797
allowNesting: Boolean = false)(body: => T): T = {
9898
val stackTrace = Thread.currentThread.getStackTrace().tail // ignore "Thread#getStackTrace"
99-
val ourMethodName = stackTrace(1).getMethodName //
99+
val ourMethodName = stackTrace(1).getMethodName // i.e. withScope
100100
// Climb upwards to find the first method that's called something different
101101
val callerMethodName = stackTrace
102102
.find(_.getMethodName != ourMethodName)

external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -50,8 +50,6 @@ class FlumeInputDStream[T: ClassTag](
5050
enableDecompression: Boolean
5151
) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) {
5252

53-
protected[streaming] override val customScopeName: Option[String] = Some(s"input stream [$id]")
54-
5553
override def getReceiver(): Receiver[SparkFlumeEvent] = {
5654
new FlumeReceiver(host, port, storageLevel, enableDecompression)
5755
}

external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumePollingInputDStream.scala

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -53,10 +53,6 @@ private[streaming] class FlumePollingInputDStream[T: ClassTag](
5353
storageLevel: StorageLevel
5454
) extends ReceiverInputDStream[SparkFlumeEvent](_ssc) {
5555

56-
protected[streaming] override val customScopeName: Option[String] = {
57-
Some(s"flume polling stream [$id]")
58-
}
59-
6056
override def getReceiver(): Receiver[SparkFlumeEvent] = {
6157
new FlumePollingReceiver(addresses, maxBatchSize, parallelism, storageLevel)
6258
}

external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -59,7 +59,7 @@ object FlumeUtils {
5959
port: Int,
6060
storageLevel: StorageLevel,
6161
enableDecompression: Boolean
62-
): ReceiverInputDStream[SparkFlumeEvent] = ssc.withScope {
62+
): ReceiverInputDStream[SparkFlumeEvent] = ssc.withNamedScope("flume stream") {
6363
new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel, enableDecompression)
6464
}
6565

@@ -159,7 +159,7 @@ object FlumeUtils {
159159
storageLevel: StorageLevel,
160160
maxBatchSize: Int,
161161
parallelism: Int
162-
): ReceiverInputDStream[SparkFlumeEvent] = ssc.withScope {
162+
): ReceiverInputDStream[SparkFlumeEvent] = ssc.withNamedScope("flume polling stream") {
163163
new FlumePollingInputDStream[SparkFlumeEvent](ssc, addresses, maxBatchSize,
164164
parallelism, storageLevel)
165165
}

external/kafka/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala

Lines changed: 0 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -65,10 +65,6 @@ class DirectKafkaInputDStream[
6565
val maxRetries = context.sparkContext.getConf.getInt(
6666
"spark.streaming.kafka.maxRetries", 1)
6767

68-
protected[streaming] override val customScopeName: Option[String] = {
69-
Some(s"kafka direct stream [$id]")
70-
}
71-
7268
protected[streaming] override val checkpointData =
7369
new DirectKafkaInputDStreamCheckpointData
7470

external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -55,8 +55,6 @@ class KafkaInputDStream[
5555
storageLevel: StorageLevel
5656
) extends ReceiverInputDStream[(K, V)](ssc_) with Logging {
5757

58-
protected[streaming] override val customScopeName: Option[String] = Some(s"kafka stream [$id]")
59-
6058
def getReceiver(): Receiver[(K, V)] = {
6159
if (!useReliableReceiver) {
6260
new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel)

external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala

Lines changed: 10 additions & 10 deletions
Original file line numberDiff line numberDiff line change
@@ -58,7 +58,7 @@ object KafkaUtils {
5858
groupId: String,
5959
topics: Map[String, Int],
6060
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
61-
): ReceiverInputDStream[(String, String)] = ssc.withScope {
61+
): ReceiverInputDStream[(String, String)] = {
6262
val kafkaParams = Map[String, String](
6363
"zookeeper.connect" -> zkQuorum, "group.id" -> groupId,
6464
"zookeeper.connection.timeout.ms" -> "10000")
@@ -80,7 +80,7 @@ object KafkaUtils {
8080
kafkaParams: Map[String, String],
8181
topics: Map[String, Int],
8282
storageLevel: StorageLevel
83-
): ReceiverInputDStream[(K, V)] = ssc.withScope {
83+
): ReceiverInputDStream[(K, V)] = ssc.withNamedScope("kafka stream") {
8484
val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf)
8585
new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel)
8686
}
@@ -99,7 +99,7 @@ object KafkaUtils {
9999
zkQuorum: String,
100100
groupId: String,
101101
topics: JMap[String, JInt]
102-
): JavaPairReceiverInputDStream[String, String] = jssc.ssc.withScope {
102+
): JavaPairReceiverInputDStream[String, String] = {
103103
createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*))
104104
}
105105

@@ -118,7 +118,7 @@ object KafkaUtils {
118118
groupId: String,
119119
topics: JMap[String, JInt],
120120
storageLevel: StorageLevel
121-
): JavaPairReceiverInputDStream[String, String] = jssc.ssc.withScope {
121+
): JavaPairReceiverInputDStream[String, String] = {
122122
createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*),
123123
storageLevel)
124124
}
@@ -145,7 +145,7 @@ object KafkaUtils {
145145
kafkaParams: JMap[String, String],
146146
topics: JMap[String, JInt],
147147
storageLevel: StorageLevel
148-
): JavaPairReceiverInputDStream[K, V] = jssc.ssc.withScope {
148+
): JavaPairReceiverInputDStream[K, V] = {
149149
implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass)
150150
implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass)
151151

@@ -295,7 +295,7 @@ object KafkaUtils {
295295
offsetRanges: Array[OffsetRange],
296296
leaders: JMap[TopicAndPartition, Broker],
297297
messageHandler: JFunction[MessageAndMetadata[K, V], R]
298-
): JavaRDD[R] = jsc.sc.withScope {
298+
): JavaRDD[R] = {
299299
implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
300300
implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
301301
implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)
@@ -348,7 +348,7 @@ object KafkaUtils {
348348
kafkaParams: Map[String, String],
349349
fromOffsets: Map[TopicAndPartition, Long],
350350
messageHandler: MessageAndMetadata[K, V] => R
351-
): InputDStream[R] = ssc.withScope {
351+
): InputDStream[R] = ssc.withNamedScope("kafka direct stream") {
352352
val cleanedHandler = ssc.sc.clean(messageHandler)
353353
new DirectKafkaInputDStream[K, V, KD, VD, R](
354354
ssc, kafkaParams, fromOffsets, cleanedHandler)
@@ -394,7 +394,7 @@ object KafkaUtils {
394394
ssc: StreamingContext,
395395
kafkaParams: Map[String, String],
396396
topics: Set[String]
397-
): InputDStream[(K, V)] = ssc.withScope {
397+
): InputDStream[(K, V)] = ssc.withNamedScope("kafka direct stream") {
398398
val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message)
399399
val kc = new KafkaCluster(kafkaParams)
400400
val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase)
@@ -465,7 +465,7 @@ object KafkaUtils {
465465
kafkaParams: JMap[String, String],
466466
fromOffsets: JMap[TopicAndPartition, JLong],
467467
messageHandler: JFunction[MessageAndMetadata[K, V], R]
468-
): JavaInputDStream[R] = jssc.ssc.withScope {
468+
): JavaInputDStream[R] = {
469469
implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
470470
implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
471471
implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)
@@ -524,7 +524,7 @@ object KafkaUtils {
524524
valueDecoderClass: Class[VD],
525525
kafkaParams: JMap[String, String],
526526
topics: JSet[String]
527-
): JavaPairInputDStream[K, V] = jssc.ssc.withScope {
527+
): JavaPairInputDStream[K, V] = {
528528
implicit val keyCmt: ClassTag[K] = ClassTag(keyClass)
529529
implicit val valueCmt: ClassTag[V] = ClassTag(valueClass)
530530
implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass)

external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -57,8 +57,6 @@ class MQTTInputDStream(
5757
storageLevel: StorageLevel
5858
) extends ReceiverInputDStream[String](ssc_) {
5959

60-
protected[streaming] override val customScopeName: Option[String] = Some(s"MQTT stream [$id]")
61-
6260
def getReceiver(): Receiver[String] = {
6361
new MQTTReceiver(brokerUrl, topic, storageLevel)
6462
}

external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -37,7 +37,7 @@ object MQTTUtils {
3737
brokerUrl: String,
3838
topic: String,
3939
storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2
40-
): ReceiverInputDStream[String] = ssc.withScope {
40+
): ReceiverInputDStream[String] = ssc.withNamedScope("MQTT stream") {
4141
new MQTTInputDStream(ssc, brokerUrl, topic, storageLevel)
4242
}
4343

external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala

Lines changed: 0 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -45,8 +45,6 @@ class TwitterInputDStream(
4545
storageLevel: StorageLevel
4646
) extends ReceiverInputDStream[Status](ssc_) {
4747

48-
protected[streaming] override val customScopeName: Option[String] = Some(s"twitter stream [$id]")
49-
5048
private def createOAuthAuthorization(): Authorization = {
5149
new OAuthAuthorization(new ConfigurationBuilder().build())
5250
}

0 commit comments

Comments
 (0)