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-28875][DSTREAMS][SS][TESTS] Add Task retry tests to make sure new consumer used #25582
Closed
Closed
Changes from all commits
Commits
Show all changes
2 commits
Select commit
Hold shift + click to select a range
File filter
Filter by extension
Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
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
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,22 +20,23 @@ package org.apache.spark.sql.kafka010 | |
import java.util.concurrent.{Executors, TimeUnit} | ||
|
||
import scala.collection.JavaConverters._ | ||
import scala.concurrent.{ExecutionContext, Future} | ||
import scala.concurrent.duration.Duration | ||
import scala.util.Random | ||
|
||
import org.apache.kafka.clients.consumer.ConsumerConfig | ||
import org.apache.kafka.clients.consumer.ConsumerConfig._ | ||
import org.apache.kafka.common.TopicPartition | ||
import org.apache.kafka.common.serialization.ByteArrayDeserializer | ||
import org.scalatest.PrivateMethodTester | ||
|
||
import org.apache.spark.{TaskContext, TaskContextImpl} | ||
import org.apache.spark.sql.kafka010.KafkaDataConsumer.CacheKey | ||
import org.apache.spark.sql.test.SharedSparkSession | ||
import org.apache.spark.util.ThreadUtils | ||
|
||
class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester { | ||
|
||
protected var testUtils: KafkaTestUtils = _ | ||
private val topic = "topic" + Random.nextInt() | ||
private val topicPartition = new TopicPartition(topic, 0) | ||
private val groupId = "groupId" | ||
|
||
override def beforeAll(): Unit = { | ||
super.beforeAll() | ||
|
@@ -51,6 +52,15 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester | |
super.afterAll() | ||
} | ||
|
||
private def getKafkaParams() = Map[String, Object]( | ||
GROUP_ID_CONFIG -> "groupId", | ||
BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress, | ||
KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName, | ||
VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName, | ||
AUTO_OFFSET_RESET_CONFIG -> "earliest", | ||
ENABLE_AUTO_COMMIT_CONFIG -> "false" | ||
).asJava | ||
|
||
test("SPARK-19886: Report error cause correctly in reportDataLoss") { | ||
val cause = new Exception("D'oh!") | ||
val reportDataLoss = PrivateMethod[Unit]('reportDataLoss0) | ||
|
@@ -60,23 +70,40 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester | |
assert(e.getCause === cause) | ||
} | ||
|
||
test("new KafkaDataConsumer instance in case of Task retry") { | ||
try { | ||
KafkaDataConsumer.cache.clear() | ||
|
||
val kafkaParams = getKafkaParams() | ||
val key = new CacheKey(groupId, topicPartition) | ||
|
||
val context1 = new TaskContextImpl(0, 0, 0, 0, 0, null, null, null) | ||
TaskContext.setTaskContext(context1) | ||
val consumer1 = KafkaDataConsumer.acquire(topicPartition, kafkaParams, true) | ||
consumer1.release() | ||
|
||
assert(KafkaDataConsumer.cache.size() == 1) | ||
assert(KafkaDataConsumer.cache.get(key).eq(consumer1.internalConsumer)) | ||
|
||
val context2 = new TaskContextImpl(0, 0, 0, 0, 1, null, null, null) | ||
TaskContext.setTaskContext(context2) | ||
val consumer2 = KafkaDataConsumer.acquire(topicPartition, kafkaParams, true) | ||
consumer2.release() | ||
|
||
// The first consumer should be removed from cache and new non-cached should be returned | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I'd say consumer2 should be cached as it's created after invalidation, but here you only address test so that's OK. |
||
assert(KafkaDataConsumer.cache.size() == 0) | ||
assert(consumer1.internalConsumer.ne(consumer2.internalConsumer)) | ||
} finally { | ||
TaskContext.unset() | ||
} | ||
} | ||
|
||
test("SPARK-23623: concurrent use of KafkaDataConsumer") { | ||
val topic = "topic" + Random.nextInt() | ||
val data = (1 to 1000).map(_.toString) | ||
testUtils.createTopic(topic, 1) | ||
testUtils.sendMessages(topic, data.toArray) | ||
val topicPartition = new TopicPartition(topic, 0) | ||
|
||
import ConsumerConfig._ | ||
val kafkaParams = Map[String, Object]( | ||
GROUP_ID_CONFIG -> "groupId", | ||
BOOTSTRAP_SERVERS_CONFIG -> testUtils.brokerAddress, | ||
KEY_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName, | ||
VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[ByteArrayDeserializer].getName, | ||
AUTO_OFFSET_RESET_CONFIG -> "earliest", | ||
ENABLE_AUTO_COMMIT_CONFIG -> "false" | ||
) | ||
|
||
val kafkaParams = getKafkaParams() | ||
val numThreads = 100 | ||
val numConsumerUsages = 500 | ||
|
||
|
@@ -90,8 +117,7 @@ class KafkaDataConsumerSuite extends SharedSparkSession with PrivateMethodTester | |
null | ||
} | ||
TaskContext.setTaskContext(taskContext) | ||
val consumer = KafkaDataConsumer.acquire( | ||
topicPartition, kafkaParams.asJava, useCache) | ||
val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams, useCache) | ||
try { | ||
val range = consumer.getAvailableOffsetRange() | ||
val rcvd = range.earliest until range.latest map { offset => | ||
|
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
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.
That's technically
private[kafka010]
as class scope so seems OK.