Skip to content

Commit ec875c5

Browse files
panbingkunLuciferYang
authored andcommitted
[SPARK-45688][SPARK-45693][CORE] Clean up the deprecated API usage related to MapOps & Fix method += in trait Growable is deprecated
### What changes were proposed in this pull request? The pr aims to: - clean up the deprecated API usage related to MapOps. - fix method += in trait Growable is deprecated. ### Why are the changes needed? Eliminate warnings and no longer use `deprecated scala APIs`. ### Does this PR introduce _any_ user-facing change? No. ### How was this patch tested? - Pass GA. - Manually test: ``` build/sbt -Phadoop-3 -Pdocker-integration-tests -Pspark-ganglia-lgpl -Pkinesis-asl -Pkubernetes -Phive-thriftserver -Pconnect -Pyarn -Phive -Phadoop-cloud -Pvolcano -Pkubernetes-integration-tests Test/package streaming-kinesis-asl-assembly/assembly connect/assembly ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43578 from panbingkun/SPARK-45688. Authored-by: panbingkun <[email protected]> Signed-off-by: yangjie01 <[email protected]>
1 parent e9339e6 commit ec875c5

File tree

7 files changed

+25
-25
lines changed

7 files changed

+25
-25
lines changed

core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala

Lines changed: 14 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -715,7 +715,7 @@ private[spark] class SparkSubmit extends Logging {
715715
if (opt.value != null &&
716716
(deployMode & opt.deployMode) != 0 &&
717717
(clusterManager & opt.clusterManager) != 0) {
718-
if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) }
718+
if (opt.clOption != null) { childArgs += opt.clOption += opt.value }
719719
if (opt.confKey != null) {
720720
if (opt.mergeFn.isDefined && sparkConf.contains(opt.confKey)) {
721721
sparkConf.set(opt.confKey, opt.mergeFn.get.apply(sparkConf.get(opt.confKey), opt.value))
@@ -747,15 +747,15 @@ private[spark] class SparkSubmit extends Logging {
747747
if (args.isStandaloneCluster) {
748748
if (args.useRest) {
749749
childMainClass = REST_CLUSTER_SUBMIT_CLASS
750-
childArgs += (args.primaryResource, args.mainClass)
750+
childArgs += args.primaryResource += args.mainClass
751751
} else {
752752
// In legacy standalone cluster mode, use Client as a wrapper around the user class
753753
childMainClass = STANDALONE_CLUSTER_SUBMIT_CLASS
754754
if (args.supervise) { childArgs += "--supervise" }
755-
Option(args.driverMemory).foreach { m => childArgs += ("--memory", m) }
756-
Option(args.driverCores).foreach { c => childArgs += ("--cores", c) }
755+
Option(args.driverMemory).foreach { m => childArgs += "--memory" += m }
756+
Option(args.driverCores).foreach { c => childArgs += "--cores" += c }
757757
childArgs += "launch"
758-
childArgs += (args.master, args.primaryResource, args.mainClass)
758+
childArgs += args.master += args.primaryResource += args.mainClass
759759
}
760760
if (args.childArgs != null) {
761761
childArgs ++= args.childArgs
@@ -777,20 +777,20 @@ private[spark] class SparkSubmit extends Logging {
777777
if (isYarnCluster) {
778778
childMainClass = YARN_CLUSTER_SUBMIT_CLASS
779779
if (args.isPython) {
780-
childArgs += ("--primary-py-file", args.primaryResource)
781-
childArgs += ("--class", "org.apache.spark.deploy.PythonRunner")
780+
childArgs += "--primary-py-file" += args.primaryResource
781+
childArgs += "--class" += "org.apache.spark.deploy.PythonRunner"
782782
} else if (args.isR) {
783783
val mainFile = new Path(args.primaryResource).getName
784-
childArgs += ("--primary-r-file", mainFile)
785-
childArgs += ("--class", "org.apache.spark.deploy.RRunner")
784+
childArgs += "--primary-r-file" += mainFile
785+
childArgs += "--class" += "org.apache.spark.deploy.RRunner"
786786
} else {
787787
if (args.primaryResource != SparkLauncher.NO_RESOURCE) {
788-
childArgs += ("--jar", args.primaryResource)
788+
childArgs += "--jar" += args.primaryResource
789789
}
790-
childArgs += ("--class", args.mainClass)
790+
childArgs += "--class" += args.mainClass
791791
}
792792
if (args.childArgs != null) {
793-
args.childArgs.foreach { arg => childArgs += ("--arg", arg) }
793+
args.childArgs.foreach { arg => childArgs += "--arg" += arg }
794794
}
795795
}
796796

@@ -813,12 +813,12 @@ private[spark] class SparkSubmit extends Logging {
813813
}
814814
if (args.childArgs != null) {
815815
args.childArgs.foreach { arg =>
816-
childArgs += ("--arg", arg)
816+
childArgs += "--arg" += arg
817817
}
818818
}
819819
// Pass the proxyUser to the k8s app so it is possible to add it to the driver args
820820
if (args.proxyUser != null) {
821-
childArgs += ("--proxy-user", args.proxyUser)
821+
childArgs += "--proxy-user" += args.proxyUser
822822
}
823823
}
824824

core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -81,14 +81,15 @@ object CommandUtils extends Logging {
8181

8282
var newEnvironment = if (libraryPathEntries.nonEmpty && libraryPathName.nonEmpty) {
8383
val libraryPaths = libraryPathEntries ++ cmdLibraryPath ++ env.get(libraryPathName)
84-
command.environment + ((libraryPathName, libraryPaths.mkString(File.pathSeparator)))
84+
command.environment ++ Map(libraryPathName -> libraryPaths.mkString(File.pathSeparator))
8585
} else {
8686
command.environment
8787
}
8888

8989
// set auth secret to env variable if needed
9090
if (securityMgr.isAuthenticationEnabled()) {
91-
newEnvironment += (SecurityManager.ENV_AUTH_SECRET -> securityMgr.getSecretKey())
91+
newEnvironment = newEnvironment ++
92+
Map(SecurityManager.ENV_AUTH_SECRET -> securityMgr.getSecretKey())
9293
}
9394
// set SSL env variables if needed
9495
newEnvironment ++= securityMgr.getEnvironmentForSslRpcPasswords

core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -626,7 +626,7 @@ class JsonProtocolSuite extends SparkFunSuite {
626626
val expectedEvent: SparkListenerEnvironmentUpdate = {
627627
val e = JsonProtocol.environmentUpdateFromJson(environmentUpdateJsonString)
628628
e.copy(environmentDetails =
629-
e.environmentDetails + ("Metrics Properties" -> Seq.empty[(String, String)]))
629+
e.environmentDetails ++ Map("Metrics Properties" -> Seq.empty[(String, String)]))
630630
}
631631
val oldEnvironmentUpdateJson = environmentUpdateJsonString
632632
.removeField("Metrics Properties")

resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -53,7 +53,7 @@ private[spark] class YarnClientSchedulerBackend(
5353
sc.ui.foreach { ui => conf.set(DRIVER_APP_UI_ADDRESS, ui.webUrl) }
5454

5555
val argsArrayBuf = new ArrayBuffer[String]()
56-
argsArrayBuf += ("--arg", hostport)
56+
argsArrayBuf += "--arg" += hostport
5757

5858
logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" "))
5959
val args = new ClientArguments(argsArrayBuf.toArray)

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -206,13 +206,12 @@ case class JoinEstimation(join: Join) extends Logging {
206206
case _ =>
207207
computeByNdv(leftKey, rightKey, newMin, newMax)
208208
}
209-
keyStatsAfterJoin += (
209+
keyStatsAfterJoin +=
210210
// Histograms are propagated as unchanged. During future estimation, they should be
211211
// truncated by the updated max/min. In this way, only pointers of the histograms are
212212
// propagated and thus reduce memory consumption.
213-
leftKey -> joinStat.copy(histogram = leftKeyStat.histogram),
214-
rightKey -> joinStat.copy(histogram = rightKeyStat.histogram)
215-
)
213+
(leftKey -> joinStat.copy(histogram = leftKeyStat.histogram)) +=
214+
(rightKey -> joinStat.copy(histogram = rightKeyStat.histogram))
216215
// Return cardinality estimated from the most selective join keys.
217216
if (card < joinCard) joinCard = card
218217
} else {

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DescribeNamespaceExec.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -46,12 +46,12 @@ case class DescribeNamespaceExec(
4646
}
4747

4848
if (isExtended) {
49-
val properties = metadata.asScala -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES
49+
val properties = metadata.asScala.toMap -- CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES
5050
val propertiesStr =
5151
if (properties.isEmpty) {
5252
""
5353
} else {
54-
conf.redactOptions(properties.toMap).toSeq.sortBy(_._1).mkString("(", ", ", ")")
54+
conf.redactOptions(properties).toSeq.sortBy(_._1).mkString("(", ", ", ")")
5555
}
5656
rows += toCatalystRow("Properties", propertiesStr)
5757
}

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -827,7 +827,7 @@ class V2SessionCatalogNamespaceSuite extends V2SessionCatalogBaseSuite {
827827
// remove location and comment that are automatically added by HMS unless they are expected
828828
val toRemove =
829829
CatalogV2Util.NAMESPACE_RESERVED_PROPERTIES.filter(expected.contains)
830-
assert(expected -- toRemove === actual)
830+
assert(expected.toMap -- toRemove === actual)
831831
}
832832

833833
test("listNamespaces: basic behavior") {

0 commit comments

Comments
 (0)