-
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-23033][SS] Don't use task level retry for continuous processing #20225
Changes from 9 commits
9f7066e
f641be0
761fd26
1bf613f
f175094
5dbc038
3b19fcb
ad2f206
54d3a2c
b40c8f0
b5d621b
cea2ddc
49f1eb6
0122aeb
f97bc9d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,23 @@ | ||
/* | ||
* 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.sql.execution.streaming.continuous | ||
|
||
import org.apache.spark.SparkException | ||
|
||
class ContinuousTaskRetryException | ||
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. Add docs. |
||
extends SparkException("Continuous execution does not support task retry", null) |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,36 +17,18 @@ | |
|
||
package org.apache.spark.sql.streaming.continuous | ||
|
||
import java.io.{File, InterruptedIOException, IOException, UncheckedIOException} | ||
import java.nio.channels.ClosedByInterruptException | ||
import java.util.concurrent.{CountDownLatch, ExecutionException, TimeoutException, TimeUnit} | ||
|
||
import scala.reflect.ClassTag | ||
import scala.util.control.ControlThrowable | ||
|
||
import com.google.common.util.concurrent.UncheckedExecutionException | ||
import org.apache.commons.io.FileUtils | ||
import org.apache.hadoop.conf.Configuration | ||
import java.util.UUID | ||
|
||
import org.apache.spark.{SparkContext, SparkEnv} | ||
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} | ||
import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerTaskStart} | ||
import org.apache.spark.sql._ | ||
import org.apache.spark.sql.catalyst.plans.logical.Range | ||
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes | ||
import org.apache.spark.sql.execution.command.ExplainCommand | ||
import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, WriteToDataSourceV2Exec} | ||
import org.apache.spark.sql.execution.streaming._ | ||
import org.apache.spark.sql.execution.streaming.continuous._ | ||
import org.apache.spark.sql.execution.streaming.sources.MemorySinkV2 | ||
import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProvider} | ||
import org.apache.spark.sql.functions._ | ||
import org.apache.spark.sql.internal.SQLConf | ||
import org.apache.spark.sql.sources.StreamSourceProvider | ||
import org.apache.spark.sql.streaming.{StreamTest, Trigger} | ||
import org.apache.spark.sql.streaming.util.StreamManualClock | ||
import org.apache.spark.sql.test.TestSparkSession | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.util.Utils | ||
|
||
class ContinuousSuiteBase extends StreamTest { | ||
// We need more than the default local[2] to be able to schedule all partitions simultaneously. | ||
|
@@ -219,6 +201,42 @@ class ContinuousSuite extends ContinuousSuiteBase { | |
StopStream) | ||
} | ||
|
||
test("task failure kills the query") { | ||
val df = spark.readStream | ||
.format("rate") | ||
.option("numPartitions", "5") | ||
.option("rowsPerSecond", "5") | ||
.load() | ||
.select('value) | ||
|
||
// Get an arbitrary task from this query to kill. It doesn't matter which one. | ||
var taskId: Long = -1 | ||
val listener = new SparkListener() { | ||
override def onTaskStart(start: SparkListenerTaskStart): Unit = { | ||
taskId = start.taskInfo.taskId | ||
} | ||
} | ||
spark.sparkContext.addSparkListener(listener) | ||
|
||
testStream(df, useV2Sink = true)( | ||
StartStream(Trigger.Continuous(100)), | ||
Execute(waitForRateSourceTriggers(_, 2)), | ||
Execute { query => | ||
// Wait until a task is started, then kill its first attempt. | ||
eventually(timeout(streamingTimeout)) { assert(taskId != -1) } | ||
spark.sparkContext.killTaskAttempt(taskId) | ||
eventually(timeout(streamingTimeout)) { | ||
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. Can this be checked with a "ExpectFailure" test? Better to test using the same harness that is used for microbatch so that we are sure they failure behavior is the same. |
||
assert(query.exception.isDefined) | ||
} | ||
assert( | ||
query.exception.get.getCause != null && | ||
query.exception.get.getCause.getCause != null && | ||
query.exception.get.getCause.getCause.getCause.isInstanceOf[ContinuousTaskRetryException]) | ||
}) | ||
|
||
spark.sparkContext.removeSparkListener(listener) | ||
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. put this in a finally clause. |
||
} | ||
|
||
test("query without test harness") { | ||
val df = spark.readStream | ||
.format("rate") | ||
|
@@ -258,13 +276,9 @@ class ContinuousStressSuite extends ContinuousSuiteBase { | |
AwaitEpoch(0), | ||
Execute(waitForRateSourceTriggers(_, 201)), | ||
IncrementEpoch(), | ||
Execute { query => | ||
val data = query.sink.asInstanceOf[MemorySinkV2].allData | ||
val vals = data.map(_.getLong(0)).toSet | ||
assert(scala.Range(0, 25000).forall { i => | ||
vals.contains(i) | ||
}) | ||
}) | ||
StopStream, | ||
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. What is this for? 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. This is the earlier comment about the overloaded failure mode this PR exposed. |
||
CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_))) | ||
) | ||
} | ||
|
||
test("automatic epoch advancement") { | ||
|
@@ -280,6 +294,7 @@ class ContinuousStressSuite extends ContinuousSuiteBase { | |
AwaitEpoch(0), | ||
Execute(waitForRateSourceTriggers(_, 201)), | ||
IncrementEpoch(), | ||
StopStream, | ||
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. Why are these needed? |
||
CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_)))) | ||
} | ||
|
||
|
@@ -311,6 +326,7 @@ class ContinuousStressSuite extends ContinuousSuiteBase { | |
StopStream, | ||
StartStream(Trigger.Continuous(2012)), | ||
AwaitEpoch(50), | ||
StopStream, | ||
CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_)))) | ||
} | ||
} |
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.
Add comments on what this is.