-
Notifications
You must be signed in to change notification settings - Fork 28.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-10210] [STREAMING] Filter out non-existent blocks before creating BlockRDD #8405
Closed
Closed
Changes from all commits
Commits
Show all changes
4 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
156 changes: 156 additions & 0 deletions
156
streaming/src/test/scala/org/apache/spark/streaming/ReceiverInputDStreamSuite.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,156 @@ | ||
/* | ||
* 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 | ||
|
||
import scala.util.Random | ||
|
||
import org.scalatest.BeforeAndAfterAll | ||
|
||
import org.apache.spark.rdd.BlockRDD | ||
import org.apache.spark.storage.{StorageLevel, StreamBlockId} | ||
import org.apache.spark.streaming.dstream.ReceiverInputDStream | ||
import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD | ||
import org.apache.spark.streaming.receiver.{BlockManagerBasedStoreResult, Receiver, WriteAheadLogBasedStoreResult} | ||
import org.apache.spark.streaming.scheduler.ReceivedBlockInfo | ||
import org.apache.spark.streaming.util.{WriteAheadLogRecordHandle, WriteAheadLogUtils} | ||
import org.apache.spark.{SparkConf, SparkEnv} | ||
|
||
class ReceiverInputDStreamSuite extends TestSuiteBase with BeforeAndAfterAll { | ||
|
||
override def afterAll(): Unit = { | ||
StreamingContext.getActive().map { _.stop() } | ||
} | ||
|
||
testWithoutWAL("createBlockRDD creates empty BlockRDD when no block info") { receiverStream => | ||
val rdd = receiverStream.createBlockRDD(Time(0), Seq.empty) | ||
assert(rdd.isInstanceOf[BlockRDD[_]]) | ||
assert(!rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) | ||
assert(rdd.isEmpty()) | ||
} | ||
|
||
testWithoutWAL("createBlockRDD creates correct BlockRDD with block info") { receiverStream => | ||
val blockInfos = Seq.fill(5) { createBlockInfo(withWALInfo = false) } | ||
val blockIds = blockInfos.map(_.blockId) | ||
|
||
// Verify that there are some blocks that are present, and some that are not | ||
require(blockIds.forall(blockId => SparkEnv.get.blockManager.master.contains(blockId))) | ||
|
||
val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) | ||
assert(rdd.isInstanceOf[BlockRDD[_]]) | ||
assert(!rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) | ||
val blockRDD = rdd.asInstanceOf[BlockRDD[_]] | ||
assert(blockRDD.blockIds.toSeq === blockIds) | ||
} | ||
|
||
testWithoutWAL("createBlockRDD filters non-existent blocks before creating BlockRDD") { | ||
receiverStream => | ||
val presentBlockInfos = Seq.fill(2)(createBlockInfo(withWALInfo = false, createBlock = true)) | ||
val absentBlockInfos = Seq.fill(3)(createBlockInfo(withWALInfo = false, createBlock = false)) | ||
val blockInfos = presentBlockInfos ++ absentBlockInfos | ||
val blockIds = blockInfos.map(_.blockId) | ||
|
||
// Verify that there are some blocks that are present, and some that are not | ||
require(blockIds.exists(blockId => SparkEnv.get.blockManager.master.contains(blockId))) | ||
require(blockIds.exists(blockId => !SparkEnv.get.blockManager.master.contains(blockId))) | ||
|
||
val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) | ||
assert(rdd.isInstanceOf[BlockRDD[_]]) | ||
val blockRDD = rdd.asInstanceOf[BlockRDD[_]] | ||
assert(blockRDD.blockIds.toSeq === presentBlockInfos.map { _.blockId}) | ||
} | ||
|
||
testWithWAL("createBlockRDD creates empty WALBackedBlockRDD when no block info") { | ||
receiverStream => | ||
val rdd = receiverStream.createBlockRDD(Time(0), Seq.empty) | ||
assert(rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) | ||
assert(rdd.isEmpty()) | ||
} | ||
|
||
testWithWAL( | ||
"createBlockRDD creates correct WALBackedBlockRDD with all block info having WAL info") { | ||
receiverStream => | ||
val blockInfos = Seq.fill(5) { createBlockInfo(withWALInfo = true) } | ||
val blockIds = blockInfos.map(_.blockId) | ||
val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) | ||
assert(rdd.isInstanceOf[WriteAheadLogBackedBlockRDD[_]]) | ||
val blockRDD = rdd.asInstanceOf[WriteAheadLogBackedBlockRDD[_]] | ||
assert(blockRDD.blockIds.toSeq === blockIds) | ||
assert(blockRDD.walRecordHandles.toSeq === blockInfos.map { _.walRecordHandleOption.get }) | ||
} | ||
|
||
testWithWAL("createBlockRDD creates BlockRDD when some block info dont have WAL info") { | ||
receiverStream => | ||
val blockInfos1 = Seq.fill(2) { createBlockInfo(withWALInfo = true) } | ||
val blockInfos2 = Seq.fill(3) { createBlockInfo(withWALInfo = false) } | ||
val blockInfos = blockInfos1 ++ blockInfos2 | ||
val blockIds = blockInfos.map(_.blockId) | ||
val rdd = receiverStream.createBlockRDD(Time(0), blockInfos) | ||
assert(rdd.isInstanceOf[BlockRDD[_]]) | ||
val blockRDD = rdd.asInstanceOf[BlockRDD[_]] | ||
assert(blockRDD.blockIds.toSeq === blockIds) | ||
} | ||
|
||
|
||
private def testWithoutWAL(msg: String)(body: ReceiverInputDStream[_] => Unit): Unit = { | ||
test(s"Without WAL enabled: $msg") { | ||
runTest(enableWAL = false, body) | ||
} | ||
} | ||
|
||
private def testWithWAL(msg: String)(body: ReceiverInputDStream[_] => Unit): Unit = { | ||
test(s"With WAL enabled: $msg") { | ||
runTest(enableWAL = true, body) | ||
} | ||
} | ||
|
||
private def runTest(enableWAL: Boolean, body: ReceiverInputDStream[_] => Unit): Unit = { | ||
val conf = new SparkConf() | ||
conf.setMaster("local[4]").setAppName("ReceiverInputDStreamSuite") | ||
conf.set(WriteAheadLogUtils.RECEIVER_WAL_ENABLE_CONF_KEY, enableWAL.toString) | ||
require(WriteAheadLogUtils.enableReceiverLog(conf) === enableWAL) | ||
val ssc = new StreamingContext(conf, Seconds(1)) | ||
val receiverStream = new ReceiverInputDStream[Int](ssc) { | ||
override def getReceiver(): Receiver[Int] = null | ||
} | ||
withStreamingContext(ssc) { ssc => | ||
body(receiverStream) | ||
} | ||
} | ||
|
||
/** | ||
* Create a block info for input to the ReceiverInputDStream.createBlockRDD | ||
* @param withWALInfo Create block with WAL info in it | ||
* @param createBlock Actually create the block in the BlockManager | ||
* @return | ||
*/ | ||
private def createBlockInfo( | ||
withWALInfo: Boolean, | ||
createBlock: Boolean = true): ReceivedBlockInfo = { | ||
val blockId = new StreamBlockId(0, Random.nextLong()) | ||
if (createBlock) { | ||
SparkEnv.get.blockManager.putSingle(blockId, 1, StorageLevel.MEMORY_ONLY, tellMaster = true) | ||
require(SparkEnv.get.blockManager.master.contains(blockId)) | ||
} | ||
val storeResult = if (withWALInfo) { | ||
new WriteAheadLogBasedStoreResult(blockId, None, new WriteAheadLogRecordHandle { }) | ||
} else { | ||
new BlockManagerBasedStoreResult(blockId, None) | ||
} | ||
new ReceivedBlockInfo(0, None, None, storeResult) | ||
} | ||
} |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it worth to add some warning log here? I think the user may forget to enable receiver log.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Added warning.