Skip to content
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

[Backport 0.3] Add maxExecutors configuration for streaming queries #328

Merged
merged 1 commit into from
May 1, 2024
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,8 @@ object FlintJob extends Logging with FlintJobExecutor {
* Without this setup, Spark would not recognize names in the format `my_glue1.default`.
*/
conf.set("spark.sql.defaultCatalog", dataSource)
configDYNMaxExecutors(conf, jobType)

val streamingRunningCount = new AtomicInteger(0)
val jobOperator =
JobOperator(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,19 @@ trait FlintJobExecutor {
"org.opensearch.flint.spark.FlintPPLSparkExtensions,org.opensearch.flint.spark.FlintSparkExtensions")
}

/*
* Override dynamicAllocation.maxExecutors with streaming maxExecutors. more detail at
* https://github.com/opensearch-project/opensearch-spark/issues/324
*/
def configDYNMaxExecutors(conf: SparkConf, jobType: String): Unit = {
if (jobType.equalsIgnoreCase("streaming")) {
conf.set(
"spark.dynamicAllocation.maxExecutors",
conf
.get("spark.flint.streaming.dynamicAllocation.maxExecutors", "10"))
}
}

def createSparkSession(conf: SparkConf): SparkSession = {
val builder = SparkSession.builder().config(conf)
if (enableHiveSupport) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -91,6 +91,7 @@ object FlintREPL extends Logging with FlintJobExecutor {

if (jobType.equalsIgnoreCase("streaming")) {
logInfo(s"""streaming query ${query}""")
configDYNMaxExecutors(conf, jobType)
val streamingRunningCount = new AtomicInteger(0)
val jobOperator =
JobOperator(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,4 +97,17 @@ class FlintJobTest extends SparkFunSuite with JobMatchers {
|""".stripMargin
assert(FlintJob.isSuperset(input, mapping))
}

test("default streaming query maxExecutors is 10") {
val conf = spark.sparkContext.conf
FlintJob.configDYNMaxExecutors(conf, "streaming")
conf.get("spark.dynamicAllocation.maxExecutors") shouldBe "10"
}

test("override streaming query maxExecutors") {
spark.sparkContext.conf.set("spark.flint.streaming.dynamicAllocation.maxExecutors", "30")
FlintJob.configDYNMaxExecutors(spark.sparkContext.conf, "streaming")
spark.sparkContext.conf.get("spark.dynamicAllocation.maxExecutors") shouldBe "30"
}

}
Loading