Skip to content

Commit d0ce2cd

Browse files
committed
Merge remote-tracking branch 'upstream/master'
2 parents 31399a4 + 6e03de3 commit d0ce2cd

File tree

29 files changed

+334
-122
lines changed

29 files changed

+334
-122
lines changed

core/src/main/scala/org/apache/spark/deploy/worker/StandaloneWorkerShuffleService.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,7 @@ class StandaloneWorkerShuffleService(sparkConf: SparkConf, securityManager: Secu
4040
private val useSasl: Boolean = securityManager.isAuthenticationEnabled()
4141

4242
private val transportConf = SparkTransportConf.fromSparkConf(sparkConf)
43-
private val blockHandler = new ExternalShuffleBlockHandler()
43+
private val blockHandler = new ExternalShuffleBlockHandler(transportConf)
4444
private val transportContext: TransportContext = {
4545
val handler = if (useSasl) new SaslRpcHandler(blockHandler, securityManager) else blockHandler
4646
new TransportContext(transportConf, handler)

core/src/main/scala/org/apache/spark/shuffle/FileShuffleBlockManager.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,7 @@ import scala.collection.JavaConversions._
2727
import org.apache.spark.{Logging, SparkConf, SparkEnv}
2828
import org.apache.spark.executor.ShuffleWriteMetrics
2929
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
30+
import org.apache.spark.network.netty.SparkTransportConf
3031
import org.apache.spark.serializer.Serializer
3132
import org.apache.spark.shuffle.FileShuffleBlockManager.ShuffleFileGroup
3233
import org.apache.spark.storage._
@@ -68,6 +69,8 @@ private[spark]
6869
class FileShuffleBlockManager(conf: SparkConf)
6970
extends ShuffleBlockManager with Logging {
7071

72+
private val transportConf = SparkTransportConf.fromSparkConf(conf)
73+
7174
private lazy val blockManager = SparkEnv.get.blockManager
7275

7376
// Turning off shuffle file consolidation causes all shuffle Blocks to get their own file.
@@ -182,13 +185,14 @@ class FileShuffleBlockManager(conf: SparkConf)
182185
val segmentOpt = iter.next.getFileSegmentFor(blockId.mapId, blockId.reduceId)
183186
if (segmentOpt.isDefined) {
184187
val segment = segmentOpt.get
185-
return new FileSegmentManagedBuffer(segment.file, segment.offset, segment.length)
188+
return new FileSegmentManagedBuffer(
189+
transportConf, segment.file, segment.offset, segment.length)
186190
}
187191
}
188192
throw new IllegalStateException("Failed to find shuffle block: " + blockId)
189193
} else {
190194
val file = blockManager.diskBlockManager.getFile(blockId)
191-
new FileSegmentManagedBuffer(file, 0, file.length)
195+
new FileSegmentManagedBuffer(transportConf, file, 0, file.length)
192196
}
193197
}
194198

core/src/main/scala/org/apache/spark/shuffle/IndexShuffleBlockManager.scala

Lines changed: 6 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,9 @@ import java.nio.ByteBuffer
2222

2323
import com.google.common.io.ByteStreams
2424

25-
import org.apache.spark.SparkEnv
25+
import org.apache.spark.{SparkConf, SparkEnv}
2626
import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer}
27+
import org.apache.spark.network.netty.SparkTransportConf
2728
import org.apache.spark.storage._
2829

2930
/**
@@ -38,10 +39,12 @@ import org.apache.spark.storage._
3839
// Note: Changes to the format in this file should be kept in sync with
3940
// org.apache.spark.network.shuffle.StandaloneShuffleBlockManager#getSortBasedShuffleBlockData().
4041
private[spark]
41-
class IndexShuffleBlockManager extends ShuffleBlockManager {
42+
class IndexShuffleBlockManager(conf: SparkConf) extends ShuffleBlockManager {
4243

4344
private lazy val blockManager = SparkEnv.get.blockManager
4445

46+
private val transportConf = SparkTransportConf.fromSparkConf(conf)
47+
4548
/**
4649
* Mapping to a single shuffleBlockId with reduce ID 0.
4750
* */
@@ -109,6 +112,7 @@ class IndexShuffleBlockManager extends ShuffleBlockManager {
109112
val offset = in.readLong()
110113
val nextOffset = in.readLong()
111114
new FileSegmentManagedBuffer(
115+
transportConf,
112116
getDataFile(blockId.shuffleId, blockId.mapId),
113117
offset,
114118
nextOffset - offset)

core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,7 +25,7 @@ import org.apache.spark.shuffle.hash.HashShuffleReader
2525

2626
private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager {
2727

28-
private val indexShuffleBlockManager = new IndexShuffleBlockManager()
28+
private val indexShuffleBlockManager = new IndexShuffleBlockManager(conf)
2929
private val shuffleMapNumber = new ConcurrentHashMap[Int, Int]()
3030

3131
/**

core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -39,7 +39,7 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll {
3939

4040
override def beforeAll() {
4141
val transportConf = SparkTransportConf.fromSparkConf(conf)
42-
rpcHandler = new ExternalShuffleBlockHandler()
42+
rpcHandler = new ExternalShuffleBlockHandler(transportConf)
4343
val transportContext = new TransportContext(transportConf, rpcHandler)
4444
server = transportContext.createServer()
4545

docs/running-on-yarn.md

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -4,7 +4,7 @@ title: Running Spark on YARN
44
---
55

66
Support for running on [YARN (Hadoop
7-
NextGen)](http://hadoop.apache.org/docs/r2.0.2-alpha/hadoop-yarn/hadoop-yarn-site/YARN.html)
7+
NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html)
88
was added to Spark in version 0.6.0, and improved in subsequent releases.
99

1010
# Preparations
Lines changed: 29 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,29 @@
1+
#
2+
# Licensed to the Apache Software Foundation (ASF) under one or more
3+
# contributor license agreements. See the NOTICE file distributed with
4+
# this work for additional information regarding copyright ownership.
5+
# The ASF licenses this file to You under the Apache License, Version 2.0
6+
# (the "License"); you may not use this file except in compliance with
7+
# the License. You may obtain a copy of the License at
8+
#
9+
# http://www.apache.org/licenses/LICENSE-2.0
10+
#
11+
# Unless required by applicable law or agreed to in writing, software
12+
# distributed under the License is distributed on an "AS IS" BASIS,
13+
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
# See the License for the specific language governing permissions and
15+
# limitations under the License.
16+
#
17+
18+
# Set everything to be logged to the file streaming/target/unit-tests.log
19+
log4j.rootCategory=INFO, file
20+
# log4j.appender.file=org.apache.log4j.FileAppender
21+
log4j.appender.file=org.apache.log4j.FileAppender
22+
log4j.appender.file.append=false
23+
log4j.appender.file.file=target/unit-tests.log
24+
log4j.appender.file.layout=org.apache.log4j.PatternLayout
25+
log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n
26+
27+
# Ignore messages below warning level from Jetty, because it's a bit verbose
28+
log4j.logger.org.eclipse.jetty=WARN
29+

external/flume-sink/src/test/scala/org/apache/spark/streaming/flume/sink/SparkSinkSuite.scala

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -159,6 +159,7 @@ class SparkSinkSuite extends FunSuite {
159159
channelContext.put("transactionCapacity", 1000.toString)
160160
channelContext.put("keep-alive", 0.toString)
161161
channelContext.putAll(overrides)
162+
channel.setName(scala.util.Random.nextString(10))
162163
channel.configure(channelContext)
163164

164165
val sink = new SparkSink()

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

Lines changed: 0 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -26,8 +26,6 @@ import java.util.concurrent.Executors
2626
import kafka.consumer._
2727
import kafka.serializer.Decoder
2828
import kafka.utils.VerifiableProperties
29-
import kafka.utils.ZKStringSerializer
30-
import org.I0Itec.zkclient._
3129

3230
import org.apache.spark.Logging
3331
import org.apache.spark.storage.StorageLevel
@@ -97,12 +95,6 @@ class KafkaReceiver[
9795
consumerConnector = Consumer.create(consumerConfig)
9896
logInfo("Connected to " + zkConnect)
9997

100-
// When auto.offset.reset is defined, it is our responsibility to try and whack the
101-
// consumer group zk node.
102-
if (kafkaParams.contains("auto.offset.reset")) {
103-
tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id"))
104-
}
105-
10698
val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties])
10799
.newInstance(consumerConfig.props)
108100
.asInstanceOf[Decoder[K]]
@@ -139,26 +131,4 @@ class KafkaReceiver[
139131
}
140132
}
141133
}
142-
143-
// It is our responsibility to delete the consumer group when specifying auto.offset.reset. This
144-
// is because Kafka 0.7.2 only honors this param when the group is not in zookeeper.
145-
//
146-
// The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied
147-
// from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to
148-
// 'smallest'/'largest':
149-
// scalastyle:off
150-
// https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
151-
// scalastyle:on
152-
private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) {
153-
val dir = "/consumers/" + groupId
154-
logInfo("Cleaning up temporary Zookeeper data under " + dir + ".")
155-
val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer)
156-
try {
157-
zk.deleteRecursive(dir)
158-
} catch {
159-
case e: Throwable => logWarning("Error cleaning up temporary Zookeeper data", e)
160-
} finally {
161-
zk.close()
162-
}
163-
}
164134
}

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

Lines changed: 5 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -17,19 +17,18 @@
1717

1818
package org.apache.spark.streaming.kafka
1919

20-
import scala.reflect.ClassTag
21-
import scala.collection.JavaConversions._
22-
2320
import java.lang.{Integer => JInt}
2421
import java.util.{Map => JMap}
2522

23+
import scala.reflect.ClassTag
24+
import scala.collection.JavaConversions._
25+
2626
import kafka.serializer.{Decoder, StringDecoder}
2727

2828
import org.apache.spark.storage.StorageLevel
2929
import org.apache.spark.streaming.StreamingContext
30-
import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext, JavaPairDStream}
31-
import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream}
32-
30+
import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext}
31+
import org.apache.spark.streaming.dstream.ReceiverInputDStream
3332

3433
object KafkaUtils {
3534
/**

0 commit comments

Comments
 (0)