File tree Expand file tree Collapse file tree 4 files changed +29
-0
lines changed
spark-sql-application/src
main/scala/org/apache/spark/sql
test/scala/org/apache/spark/sql Expand file tree Collapse file tree 4 files changed +29
-0
lines changed Original file line number Diff line number Diff line change @@ -60,6 +60,8 @@ object FlintJob extends Logging with FlintJobExecutor {
60
60
* Without this setup, Spark would not recognize names in the format `my_glue1.default`.
61
61
*/
62
62
conf.set(" spark.sql.defaultCatalog" , dataSource)
63
+ configDYNMaxExecutors(conf, jobType)
64
+
63
65
val streamingRunningCount = new AtomicInteger (0 )
64
66
val jobOperator =
65
67
JobOperator (
Original file line number Diff line number Diff line change @@ -85,6 +85,19 @@ trait FlintJobExecutor {
85
85
" org.opensearch.flint.spark.FlintPPLSparkExtensions,org.opensearch.flint.spark.FlintSparkExtensions" )
86
86
}
87
87
88
+ /*
89
+ * Override dynamicAllocation.maxExecutors with streaming maxExecutors. more detail at
90
+ * https://github.com/opensearch-project/opensearch-spark/issues/324
91
+ */
92
+ def configDYNMaxExecutors (conf : SparkConf , jobType : String ): Unit = {
93
+ if (jobType.equalsIgnoreCase(" streaming" )) {
94
+ conf.set(
95
+ " spark.dynamicAllocation.maxExecutors" ,
96
+ conf
97
+ .get(" spark.flint.streaming.dynamicAllocation.maxExecutors" , " 10" ))
98
+ }
99
+ }
100
+
88
101
def createSparkSession (conf : SparkConf ): SparkSession = {
89
102
val builder = SparkSession .builder().config(conf)
90
103
if (enableHiveSupport) {
Original file line number Diff line number Diff line change @@ -91,6 +91,7 @@ object FlintREPL extends Logging with FlintJobExecutor {
91
91
92
92
if (jobType.equalsIgnoreCase(" streaming" )) {
93
93
logInfo(s """ streaming query ${query}""" )
94
+ configDYNMaxExecutors(conf, jobType)
94
95
val streamingRunningCount = new AtomicInteger (0 )
95
96
val jobOperator =
96
97
JobOperator (
Original file line number Diff line number Diff line change @@ -97,4 +97,17 @@ class FlintJobTest extends SparkFunSuite with JobMatchers {
97
97
|""" .stripMargin
98
98
assert(FlintJob .isSuperset(input, mapping))
99
99
}
100
+
101
+ test(" default streaming query maxExecutors is 10" ) {
102
+ val conf = spark.sparkContext.conf
103
+ FlintJob .configDYNMaxExecutors(conf, " streaming" )
104
+ conf.get(" spark.dynamicAllocation.maxExecutors" ) shouldBe " 10"
105
+ }
106
+
107
+ test(" override streaming query maxExecutors" ) {
108
+ spark.sparkContext.conf.set(" spark.flint.streaming.dynamicAllocation.maxExecutors" , " 30" )
109
+ FlintJob .configDYNMaxExecutors(spark.sparkContext.conf, " streaming" )
110
+ spark.sparkContext.conf.get(" spark.dynamicAllocation.maxExecutors" ) shouldBe " 30"
111
+ }
112
+
100
113
}
You can’t perform that action at this time.
0 commit comments