Skip to content

Commit 20952d1

Browse files
committed
Spark: Initial support for 4.1.0 UT
## Changes | Cause | Type | Category | Description | Affected Files | |-------|------|----------|-------------|----------------| | N/A | Feat | Build | Update build configuration to support Spark 4.1 UT | `.github/workflows/velox_backend_x86.yml`, `gluten-ut/pom.xml`, `gluten-ut/spark41/pom.xml`, `tools/gluten-it/pom.xml` | | [#52165](apache/spark#52165) | Fix | Dependency | Update Parquet dependency version to 1.16.0 to avoid NoSuchMethodError issue | `gluten-ut/spark41/pom.xml` | | [#51477](apache/spark#51477) | Fix | Compatibility | Update imports to reflect streaming runtime package refactoring in Apache Spark | `gluten-ut/spark41/.../GlutenDynamicPartitionPruningSuite.scala`, `gluten-ut/spark41/.../GlutenStreamingQuerySuite.scala` | | [#50674](apache/spark#50674) | Fix | Compatibility | Fix compatibility issue introduced by `TypedConfigBuilder` | `gluten-substrait/.../ExpressionConverter.scala`, `gluten-ut/spark41/.../GlutenCSVSuite.scala`, `gluten-ut/spark41/.../GlutenJsonSuite.scala` | | [#49766](apache/spark#49766) | Fix | Compatibility | Disable V2 bucketing in GlutenDynamicPartitionPruningSuite since spark.sql.sources.v2.bucketing.enabled is now enabled by default | `gluten-ut/spark41/.../GlutenDynamicPartitionPruningSuite.scala` | | [#42414](apache/spark#42414), [#53038](apache/spark#53038) | Fix | Bug Fix | Resolve an issue introduced by SPARK-42414, as identified in SPARK-53038 | `backends-velox/.../VeloxBloomFilterAggregate.scala` | | N/A | Fix | Bug Fix | Enforce row fallback for unsupported cached batches - keep columnar execution only when schema validation succeeds | `backends-velox/.../ColumnarCachedBatchSerializer.scala` | | [SPARK-53132](apache/spark#53132), [SPARK-53142](apache/spark#53142) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 KeyGroupedPartitioningSuite tests. Excluded tests: `SPARK-53322*`, `SPARK-54439*` | `gluten-ut/spark41/.../VeloxTestSettings.scala` | | [SPARK-53535](https://issues.apache.org/jira/browse/SPARK-53535), [SPARK-54220](https://issues.apache.org/jira/browse/SPARK-54220) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenParquetIOSuite tests. Excluded tests: `SPARK-53535*`, `vectorized reader: missing all struct fields*`, `SPARK-54220*` | `gluten-ut/spark41/.../VeloxTestSettings.scala` | | [#52645](apache/spark#52645) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenStreamingQuerySuite tests. Excluded tests: `SPARK-53942: changing the number of stateless shuffle partitions via config`, `SPARK-53942: stateful shuffle partitions are retained from old checkpoint` | `gluten-ut/spark41/.../VeloxTestSettings.scala` | | [#47856](apache/spark#47856) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenDataFrameWindowFunctionsSuite and GlutenJoinSuite tests. Excluded tests: `SPARK-49386: Window spill with more than the inMemoryThreshold and spillSizeThreshold`, `SPARK-49386: test SortMergeJoin (with spill by size threshold)` | `gluten-ut/spark41/.../VeloxTestSettings.scala` | | [#52157](apache/spark#52157) | 4.1.0 | Test Exclusion | Exclude additional Spark 4.1 GlutenQueryExecutionSuite tests. Excluded test: `#53413: Cleanup shuffle dependencies for commands` | `gluten-ut/spark41/.../VeloxTestSettings.scala` | | [#48470](apache/spark#48470) | 4.1.0 | Test Exclusion | Exclude split test in GlutenRegexpExpressionsSuite. Excluded test: `GlutenRegexpExpressionsSuite.SPLIT` | `gluten-ut/spark41/.../VeloxTestSettings.scala` | | [#51623](apache/spark#51623) | 4.1.0 | Test Exclusion | Add `spark.sql.unionOutputPartitioning=false` to Maven test args. Excluded tests: `GlutenBroadcastExchangeSuite.SPARK-52962`, `GlutenDataFrameSetOperationsSuite.SPARK-52921*` | `.github/workflows/velox_backend_x86.yml`, `gluten-ut/spark41/.../VeloxTestSettings.scala`, `tools/gluten-it/common/.../Suite.scala` | | N/A | 4.1.0 | Test Exclusion | Excludes failed SQL tests that need to be fixed for Spark 4.1 compatibility. Excluded tests: `decimalArithmeticOperations.sql`, `identifier-clause.sql`, `keywords.sql`, `literals.sql`, `operators.sql`, `exists-orderby-limit.sql`, `postgreSQL/date.sql`, `nonansi/keywords.sql`, `nonansi/literals.sql`, `datetime-legacy.sql`, `datetime-parsing-invalid.sql`, `misc-functions.sql` | `gluten-ut/spark41/.../VeloxSQLQueryTestSettings.scala` | | #11252 | 4.1.0 | Test Exclusion | Exclude Gluten test for SPARK-47939: Explain should work with parameterized queries | `gluten-ut/spark41/.../VeloxTestSettings.scala` |
1 parent 14797c2 commit 20952d1

File tree

14 files changed

+229
-108
lines changed

14 files changed

+229
-108
lines changed

.github/workflows/velox_backend_x86.yml

Lines changed: 28 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -107,7 +107,7 @@ jobs:
107107
fail-fast: false
108108
matrix:
109109
os: [ "ubuntu:20.04", "ubuntu:22.04" ]
110-
spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0" ]
110+
spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0", "spark-4.1" ]
111111
java: [ "java-8", "java-11", "java-17", "java-21" ]
112112
# Spark supports JDK17 since 3.3.
113113
exclude:
@@ -141,6 +141,10 @@ jobs:
141141
java: java-8
142142
- spark: spark-4.0
143143
java: java-11
144+
- spark: spark-4.1
145+
java: java-8
146+
- spark: spark-4.1
147+
java: java-11
144148

145149
runs-on: ubuntu-22.04
146150
container: ${{ matrix.os }}
@@ -182,11 +186,14 @@ jobs:
182186
cd $GITHUB_WORKSPACE/
183187
export JAVA_HOME=/usr/lib/jvm/${{ matrix.java }}-openjdk-amd64
184188
echo "JAVA_HOME: $JAVA_HOME"
185-
if [ "${{ matrix.spark }}" = "spark-4.0" ]; then
186-
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests
187-
else
188-
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests
189-
fi
189+
case "${{ matrix.spark }}" in
190+
spark-4.0|spark-4.1)
191+
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests
192+
;;
193+
*)
194+
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests
195+
;;
196+
esac
190197
cd $GITHUB_WORKSPACE/tools/gluten-it
191198
$GITHUB_WORKSPACE/$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }}
192199
GLUTEN_IT_JVM_ARGS=-Xmx5G sbin/gluten-it.sh queries-compare \
@@ -200,7 +207,7 @@ jobs:
200207
fail-fast: false
201208
matrix:
202209
os: [ "centos:8" ]
203-
spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0" ]
210+
spark: [ "spark-3.2", "spark-3.3", "spark-3.4", "spark-3.5", "spark-4.0", "spark-4.1" ]
204211
java: [ "java-8", "java-11", "java-17" ]
205212
# Spark supports JDK17 since 3.3.
206213
exclude:
@@ -220,6 +227,10 @@ jobs:
220227
java: java-8
221228
- spark: spark-4.0
222229
java: java-11
230+
- spark: spark-4.1
231+
java: java-8
232+
- spark: spark-4.1
233+
java: java-11
223234

224235
runs-on: ubuntu-22.04
225236
container: ${{ matrix.os }}
@@ -263,11 +274,14 @@ jobs:
263274
run: |
264275
echo "JAVA_HOME: $JAVA_HOME"
265276
cd $GITHUB_WORKSPACE/
266-
if [ "${{ matrix.spark }}" = "spark-4.0" ]; then
267-
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests
268-
else
269-
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests
270-
fi
277+
case "${{ matrix.spark }}" in
278+
spark-4.0|spark-4.1)
279+
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pscala-2.13 -Pbackends-velox -DskipTests
280+
;;
281+
*)
282+
$MVN_CMD clean install -P${{ matrix.spark }} -P${{ matrix.java }} -Pbackends-velox -DskipTests
283+
;;
284+
esac
271285
cd $GITHUB_WORKSPACE/tools/gluten-it
272286
$GITHUB_WORKSPACE/build/mvn clean install -P${{ matrix.spark }} -P${{ matrix.java }}
273287
- name: Run TPC-H / TPC-DS
@@ -1521,7 +1535,7 @@ jobs:
15211535
export PATH=$JAVA_HOME/bin:$PATH
15221536
java -version
15231537
$MVN_CMD clean test -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox \
1524-
-Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \
1538+
-Pspark-ut -DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/ -Dspark.sql.unionOutputPartitioning=false" \
15251539
-DtagsToExclude=org.apache.spark.tags.ExtendedSQLTest,org.apache.gluten.tags.UDFTest,org.apache.gluten.tags.EnhancedFeaturesTest,org.apache.gluten.tags.SkipTest
15261540
- name: Upload test report
15271541
if: always()
@@ -1570,7 +1584,7 @@ jobs:
15701584
export PATH=$JAVA_HOME/bin:$PATH
15711585
java -version
15721586
$MVN_CMD clean test -Pspark-4.1 -Pscala-2.13 -Pjava-17 -Pbackends-velox -Pspark-ut \
1573-
-DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/" \
1587+
-DargLine="-Dspark.test.home=/opt/shims/spark41/spark_home/ -Dspark.sql.unionOutputPartitioning=false" \
15741588
-DtagsToInclude=org.apache.spark.tags.ExtendedSQLTest
15751589
- name: Upload test report
15761590
if: always()

backends-velox/src/main/scala/org/apache/gluten/expression/aggregate/VeloxBloomFilterAggregate.scala

Lines changed: 44 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -25,10 +25,14 @@ import org.apache.spark.sql.catalyst.expressions.Expression
2525
import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate
2626
import org.apache.spark.sql.catalyst.trees.TernaryLike
2727
import org.apache.spark.sql.internal.SQLConf
28+
import org.apache.spark.sql.types._
2829
import org.apache.spark.sql.types.DataType
2930
import org.apache.spark.task.TaskResources
31+
import org.apache.spark.unsafe.types.UTF8String
3032
import org.apache.spark.util.sketch.BloomFilter
3133

34+
import java.io.Serializable
35+
3236
/**
3337
* Velox's bloom-filter implementation uses different algorithms internally comparing to vanilla
3438
* Spark so produces different intermediate aggregate data. Thus we use different filter function /
@@ -61,6 +65,15 @@ case class VeloxBloomFilterAggregate(
6165
.toLong
6266
)
6367

68+
// Mark as lazy so that `updater` is not evaluated during tree transformation.
69+
private lazy val updater: BloomFilterUpdater = child.dataType match {
70+
case LongType => LongUpdater
71+
case IntegerType => IntUpdater
72+
case ShortType => ShortUpdater
73+
case ByteType => ByteUpdater
74+
case _: StringType => BinaryUpdater
75+
}
76+
6477
override def first: Expression = child
6578

6679
override def second: Expression = estimatedNumItemsExpression
@@ -97,7 +110,7 @@ case class VeloxBloomFilterAggregate(
97110
if (value == null) {
98111
return buffer
99112
}
100-
buffer.putLong(value.asInstanceOf[Long])
113+
updater.update(buffer, value)
101114
buffer
102115
}
103116

@@ -128,3 +141,33 @@ case class VeloxBloomFilterAggregate(
128141
copy(inputAggBufferOffset = newOffset)
129142

130143
}
144+
145+
// see https://github.com/apache/spark/pull/42414
146+
private trait BloomFilterUpdater {
147+
def update(bf: BloomFilter, v: Any): Boolean
148+
}
149+
150+
private object LongUpdater extends BloomFilterUpdater with Serializable {
151+
override def update(bf: BloomFilter, v: Any): Boolean =
152+
bf.putLong(v.asInstanceOf[Long])
153+
}
154+
155+
private object IntUpdater extends BloomFilterUpdater with Serializable {
156+
override def update(bf: BloomFilter, v: Any): Boolean =
157+
bf.putLong(v.asInstanceOf[Int])
158+
}
159+
160+
private object ShortUpdater extends BloomFilterUpdater with Serializable {
161+
override def update(bf: BloomFilter, v: Any): Boolean =
162+
bf.putLong(v.asInstanceOf[Short])
163+
}
164+
165+
private object ByteUpdater extends BloomFilterUpdater with Serializable {
166+
override def update(bf: BloomFilter, v: Any): Boolean =
167+
bf.putLong(v.asInstanceOf[Byte])
168+
}
169+
170+
private object BinaryUpdater extends BloomFilterUpdater with Serializable {
171+
override def update(bf: BloomFilter, v: Any): Boolean =
172+
bf.putBinary(v.asInstanceOf[UTF8String].getBytes)
173+
}

backends-velox/src/main/scala/org/apache/spark/sql/execution/ColumnarCachedBatchSerializer.scala

Lines changed: 78 additions & 68 deletions
Original file line numberDiff line numberDiff line change
@@ -115,24 +115,24 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging {
115115
conf: SQLConf): RDD[CachedBatch] = {
116116
val localSchema = toStructType(schema)
117117
if (!validateSchema(localSchema)) {
118-
// we can not use columnar cache here, as the `RowToColumnar` does not support this schema
119-
return rowBasedCachedBatchSerializer.convertInternalRowToCachedBatch(
118+
// we cannot use columnar cache here, as the `RowToColumnar` does not support this schema
119+
rowBasedCachedBatchSerializer.convertInternalRowToCachedBatch(
120120
input,
121121
schema,
122122
storageLevel,
123123
conf)
124+
} else {
125+
val numRows = conf.columnBatchSize
126+
val rddColumnarBatch = input.mapPartitions {
127+
it =>
128+
RowToVeloxColumnarExec.toColumnarBatchIterator(
129+
it,
130+
localSchema,
131+
numRows,
132+
VeloxConfig.get.veloxPreferredBatchBytes)
133+
}
134+
convertColumnarBatchToCachedBatch(rddColumnarBatch, schema, storageLevel, conf)
124135
}
125-
126-
val numRows = conf.columnBatchSize
127-
val rddColumnarBatch = input.mapPartitions {
128-
it =>
129-
RowToVeloxColumnarExec.toColumnarBatchIterator(
130-
it,
131-
localSchema,
132-
numRows,
133-
VeloxConfig.get.veloxPreferredBatchBytes)
134-
}
135-
convertColumnarBatchToCachedBatch(rddColumnarBatch, schema, storageLevel, conf)
136136
}
137137

138138
override def convertCachedBatchToInternalRow(
@@ -141,18 +141,18 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging {
141141
selectedAttributes: Seq[Attribute],
142142
conf: SQLConf): RDD[InternalRow] = {
143143
if (!validateSchema(cacheAttributes)) {
144-
// if we do not support this schema that means we are using row-based serializer,
144+
// if we do not support this schema, that means we are using row-based serializer,
145145
// see `convertInternalRowToCachedBatch`, so fallback to vanilla Spark serializer
146-
return rowBasedCachedBatchSerializer.convertCachedBatchToInternalRow(
146+
rowBasedCachedBatchSerializer.convertCachedBatchToInternalRow(
147147
input,
148148
cacheAttributes,
149149
selectedAttributes,
150150
conf)
151+
} else {
152+
val rddColumnarBatch =
153+
convertCachedBatchToColumnarBatch(input, cacheAttributes, selectedAttributes, conf)
154+
rddColumnarBatch.mapPartitions(it => VeloxColumnarToRowExec.toRowIterator(it))
151155
}
152-
153-
val rddColumnarBatch =
154-
convertCachedBatchToColumnarBatch(input, cacheAttributes, selectedAttributes, conf)
155-
rddColumnarBatch.mapPartitions(it => VeloxColumnarToRowExec.toRowIterator(it))
156156
}
157157

158158
override def convertColumnarBatchToCachedBatch(
@@ -190,58 +190,68 @@ class ColumnarCachedBatchSerializer extends CachedBatchSerializer with Logging {
190190
cacheAttributes: Seq[Attribute],
191191
selectedAttributes: Seq[Attribute],
192192
conf: SQLConf): RDD[ColumnarBatch] = {
193-
// Find the ordinals and data types of the requested columns.
194-
val requestedColumnIndices = selectedAttributes.map {
195-
a => cacheAttributes.map(_.exprId).indexOf(a.exprId)
196-
}
197-
val shouldSelectAttributes = cacheAttributes != selectedAttributes
198-
val localSchema = toStructType(cacheAttributes)
199-
val timezoneId = SQLConf.get.sessionLocalTimeZone
200-
input.mapPartitions {
201-
it =>
202-
val runtime = Runtimes.contextInstance(
203-
BackendsApiManager.getBackendName,
204-
"ColumnarCachedBatchSerializer#read")
205-
val jniWrapper = ColumnarBatchSerializerJniWrapper
206-
.create(runtime)
207-
val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId)
208-
val arrowAlloc = ArrowBufferAllocators.contextInstance()
209-
val cSchema = ArrowSchema.allocateNew(arrowAlloc)
210-
ArrowAbiUtil.exportSchema(arrowAlloc, schema, cSchema)
211-
val deserializerHandle = jniWrapper
212-
.init(cSchema.memoryAddress())
213-
cSchema.close()
214-
215-
Iterators
216-
.wrap(new Iterator[ColumnarBatch] {
217-
override def hasNext: Boolean = it.hasNext
218-
219-
override def next(): ColumnarBatch = {
220-
val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch]
221-
val batchHandle =
222-
jniWrapper
223-
.deserialize(deserializerHandle, cachedBatch.bytes)
224-
val batch = ColumnarBatches.create(batchHandle)
225-
if (shouldSelectAttributes) {
226-
try {
227-
ColumnarBatches.select(
228-
BackendsApiManager.getBackendName,
229-
batch,
230-
requestedColumnIndices.toArray)
231-
} finally {
232-
batch.close()
193+
if (!validateSchema(cacheAttributes)) {
194+
// if we do not support this schema, that means we are using row-based serializer,
195+
// see `convertInternalRowToCachedBatch`, so fallback to vanilla Spark serializer
196+
rowBasedCachedBatchSerializer.convertCachedBatchToColumnarBatch(
197+
input,
198+
cacheAttributes,
199+
selectedAttributes,
200+
conf)
201+
} else {
202+
// Find the ordinals and data types of the requested columns.
203+
val requestedColumnIndices = selectedAttributes.map {
204+
a => cacheAttributes.map(_.exprId).indexOf(a.exprId)
205+
}
206+
val shouldSelectAttributes = cacheAttributes != selectedAttributes
207+
val localSchema = toStructType(cacheAttributes)
208+
val timezoneId = SQLConf.get.sessionLocalTimeZone
209+
input.mapPartitions {
210+
it =>
211+
val runtime = Runtimes.contextInstance(
212+
BackendsApiManager.getBackendName,
213+
"ColumnarCachedBatchSerializer#read")
214+
val jniWrapper = ColumnarBatchSerializerJniWrapper
215+
.create(runtime)
216+
val schema = SparkArrowUtil.toArrowSchema(localSchema, timezoneId)
217+
val arrowAlloc = ArrowBufferAllocators.contextInstance()
218+
val cSchema = ArrowSchema.allocateNew(arrowAlloc)
219+
ArrowAbiUtil.exportSchema(arrowAlloc, schema, cSchema)
220+
val deserializerHandle = jniWrapper
221+
.init(cSchema.memoryAddress())
222+
cSchema.close()
223+
224+
Iterators
225+
.wrap(new Iterator[ColumnarBatch] {
226+
override def hasNext: Boolean = it.hasNext
227+
228+
override def next(): ColumnarBatch = {
229+
val cachedBatch = it.next().asInstanceOf[CachedColumnarBatch]
230+
val batchHandle =
231+
jniWrapper
232+
.deserialize(deserializerHandle, cachedBatch.bytes)
233+
val batch = ColumnarBatches.create(batchHandle)
234+
if (shouldSelectAttributes) {
235+
try {
236+
ColumnarBatches.select(
237+
BackendsApiManager.getBackendName,
238+
batch,
239+
requestedColumnIndices.toArray)
240+
} finally {
241+
batch.close()
242+
}
243+
} else {
244+
batch
233245
}
234-
} else {
235-
batch
236246
}
247+
})
248+
.protectInvocationFlow()
249+
.recycleIterator {
250+
jniWrapper.close(deserializerHandle)
237251
}
238-
})
239-
.protectInvocationFlow()
240-
.recycleIterator {
241-
jniWrapper.close(deserializerHandle)
242-
}
243-
.recyclePayload(_.close())
244-
.create()
252+
.recyclePayload(_.close())
253+
.create()
254+
}
245255
}
246256
}
247257

gluten-substrait/src/main/scala/org/apache/gluten/expression/ExpressionConverter.scala

Lines changed: 8 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -831,7 +831,14 @@ object ExpressionConverter extends SQLConfHelper with Logging {
831831
case t: TransformKeys =>
832832
// default is `EXCEPTION`
833833
val mapKeyDedupPolicy = SQLConf.get.getConf(SQLConf.MAP_KEY_DEDUP_POLICY)
834-
if (mapKeyDedupPolicy == SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
834+
835+
// Calling `.toString` on both sides ensures compatibility across all Spark versions.
836+
// Starting from Spark 4.1, `SQLConf.get.getConf(SQLConf.MAP_KEY_DEDUP_POLICY)` returns
837+
// an enum instead of a String. Without `.toString`, the comparison
838+
// `mapKeyDedupPolicy == SQLConf.MapKeyDedupPolicy.LAST_WIN.toString` would silently fail
839+
// in tests, producing only a "Comparing unrelated types" warning in IntelliJ IDEA,
840+
// but no compile-time error.
841+
if (mapKeyDedupPolicy.toString == SQLConf.MapKeyDedupPolicy.LAST_WIN.toString) {
835842
// TODO: Remove after fix ready for
836843
// https://github.com/facebookincubator/velox/issues/10219
837844
throw new GlutenNotSupportException(

gluten-ut/pom.xml

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -230,5 +230,11 @@
230230
<module>spark40</module>
231231
</modules>
232232
</profile>
233+
<profile>
234+
<id>spark-4.1</id>
235+
<modules>
236+
<module>spark41</module>
237+
</modules>
238+
</profile>
233239
</profiles>
234240
</project>

0 commit comments

Comments
 (0)