You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "ivoson (via GitHub)" <gi...@apache.org> on 2023/11/03 02:45:11 UTC

[PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

ivoson opened a new pull request, #43642:
URL: https://github.com/apache/spark/pull/43642

   ### What changes were proposed in this pull request?
   Fix the deprecated behavior below: 
   `Passing an explicit array value to a Scala varargs method is deprecated (since 2.13.0) and will result in a defensive copy; Use the more efficient non-copying ArraySeq.unsafeWrapArray or an explicit toIndexedSeq call`
   
   It exists in two test suites: `AggregationQuerySuite ` and `ObjectHashAggregateSuite `, for the cases in these 2 test suites, we can take the non-copying method: `ArraySeq.unsafeWrapArray`
   
   
   ### Why are the changes needed?
   Eliminate compile warnings and no longer use deprecated scala APIs.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Pass GA.
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1383150276


##########
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:
##########
@@ -179,7 +179,9 @@ private[spark] object PythonRDD extends Logging {
     type UnrolledPartition = Array[ByteArray]
     val allPartitions: Array[UnrolledPartition] =
       sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions.asScala.toSeq)
-    val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*)
+
+    import org.apache.spark.util.ArrayImplicits._

Review Comment:
   Thanks, done. Moved to top.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1384960063


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   sorry, https://github.com/apache/spark/pull/43642/files#r1383391358 should for this case



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1382887020


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala:
##########
@@ -313,7 +313,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB
     val converter = factory.create(fieldTypes)
 
     val row = new SpecificInternalRow(fieldTypes)
-    val values = Array(new CalendarInterval(0, 7, 0L), null)
+    val values = Seq(new CalendarInterval(0, 7, 0L), null)

Review Comment:
   nit: This might be a bit slower than `toImmutableArraySeq`, but it's also okay.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1791842647

   for example:
   
   https://github.com/apache/spark/blob/eda9911057b893e42f49dbd7448f20f91f2798c4/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala#L182
   
   https://github.com/apache/spark/blob/eda9911057b893e42f49dbd7448f20f91f2798c4/core/src/main/scala/org/apache/spark/rdd/RDD.scala#L1047
   
   and 
   
   https://github.com/apache/spark/blob/eda9911057b893e42f49dbd7448f20f91f2798c4/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala#L54


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1383153910


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala:
##########
@@ -313,7 +313,7 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB
     val converter = factory.create(fieldTypes)
 
     val row = new SpecificInternalRow(fieldTypes)
-    val values = Array(new CalendarInterval(0, 7, 0L), null)
+    val values = Seq(new CalendarInterval(0, 7, 0L), null)

Review Comment:
   Change to `toImmutableArraySeq `



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala:
##########
@@ -89,7 +89,8 @@ object JDBCRDD extends Logging {
    * @return A Catalyst schema corresponding to columns in the given order.
    */
   private def pruneSchema(schema: StructType, columns: Array[String]): StructType = {
-    val fieldMap = Map(schema.fields.map(x => x.name -> x): _*)
+    import org.apache.spark.util.ArrayImplicits._
+    val fieldMap = Map(schema.fields.map(x => x.name -> x).toImmutableArraySeq: _*)

Review Comment:
   Thanks, done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1793348003

   I will set this PR to draft first.
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416694787


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   I'm confused, why don't we use `org.apache.spark.util.ArrayImplicits` here?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416718851


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   It's fine to make `examples` an exception and not use `ArrayImplicits`.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1806995590

   Thanks @LuciferYang @srowen 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1805127752

   friendly ping @srowen Could you take another look? Thanks ~


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416719471


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   Thanks for your confirmation.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1384960843


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1794930570

   The implementation of `toImmutableArraySeq` is as follows:
   
   https://github.com/apache/spark/blob/f6038302dd615f4bf9bed9c4af3d04426f7e5c5e/common/utils/src/main/scala/org/apache/spark/util/ArrayImplicits.scala#L27-L34
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1793713921

   > I will set this PR to draft first.
   
   cc @LuciferYang , this PR is ready for review. Fixed all the warnnings with build command `mvn clean package -DskipTests -Pspark-ganglia-lgpl -Pkinesis-asl -Pdocker-integration-tests -Pyarn -Pkubernetes -Pkubernetes-integration-tests -Phive-thriftserver -Phadoop-cloud`. 
   This should cover all the modules.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1384960520


##########
mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala:
##########
@@ -213,9 +213,10 @@ final class OneVsRestModel private[ml] (
         tmpModel.asInstanceOf[ProbabilisticClassificationModel[_, _]].setProbabilityCol("")
       }
 
+      import org.apache.spark.util.ArrayImplicits._
       tmpModel.transform(df)
         .withColumn(accColName, updateUDF(col(accColName), col(tmpRawPredName)))
-        .select(columns: _*)
+        .select(columns.toImmutableArraySeq: _*)

Review Comment:
   Thanks. Do you mean the case in `examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala`? Changed to use `ArraySeq. unsafeWrapArray` explicitly.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "srowen (via GitHub)" <gi...@apache.org>.
srowen closed pull request #43642: [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated`
URL: https://github.com/apache/spark/pull/43642


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416716407


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   > oh, because `examples` do not depend on common/utils
   
   Just to clarify, the `mllib-local` module indeed does not depend on the `common/utils` module, so `ArrayImplicits ` is not used in the `mllib-local` module.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1793722076

   @ivoson You can temporarily add "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e" to `compilerWarningSettings` in `SparkBuild.scala`, and run the command `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` to ensure that nothing is overlooked.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1385079947


##########
project/SparkBuild.scala:
##########
@@ -254,7 +254,8 @@ object SparkBuild extends PomBuild {
         // SPARK-45627 `enum`, `export` and `given` will become keywords in Scala 3,
         // so they are prohibited from being used as variable names in Scala 2.13 to
         // reduce the cost of migration in subsequent versions.
-        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e"
+        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e",
+        "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e"

Review Comment:
   hmm... I want to confirm again that in Scala 3, this is still just a compilation warning, right?
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1383155194


##########
project/SparkBuild.scala:
##########
@@ -254,7 +254,8 @@ object SparkBuild extends PomBuild {
         // SPARK-45627 `enum`, `export` and `given` will become keywords in Scala 3,
         // so they are prohibited from being used as variable names in Scala 2.13 to
         // reduce the cost of migration in subsequent versions.
-        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e"
+        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e",
+        "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e"

Review Comment:
   If it's not necessary, I'll remove it.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1382878088


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala:
##########
@@ -224,7 +224,7 @@ class StreamSuite extends StreamTest {
             // Parquet write page-level CRC checksums will change the file size and
             // affect the data order when reading these files. Please see PARQUET-1746 for details.
             val outputDf = spark.read.parquet(outputDir.getAbsolutePath).sort($"a").as[Long]
-            checkDataset[Long](outputDf, (0L to 10L).toArray: _*)
+            checkDataset[Long](outputDf, (0L to 10L).toSeq: _*)

Review Comment:
   ```suggestion
               checkDataset[Long](outputDf, 0L to 10L: _*)
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "srowen (via GitHub)" <gi...@apache.org>.
srowen commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1794921304

   Looks OK but the PR description says you'll avoid a copy with unsafeWrapArray or other methods, but the change uses toImmutableArraySeq. That's fine but does it do the same thing?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416714807


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   > oh, because `examples` do not depend on common/utils
   
   `common/utils` is a transitive dependency of the `core` module, so `ArrayImplicits ` is visible to the `examples` module, I previously suggested this only because I believe that `private[spark]` code should not be part of the examples. In this context, what is your suggestion? @cloud-fan 
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1791865923

   > Could you check again? IIRC, there should be more than 40+ files involved in this issue...
   
   Thanks @LuciferYang ... checking


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1383391358


##########
mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala:
##########
@@ -213,9 +213,10 @@ final class OneVsRestModel private[ml] (
         tmpModel.asInstanceOf[ProbabilisticClassificationModel[_, _]].setProbabilityCol("")
       }
 
+      import org.apache.spark.util.ArrayImplicits._
       tmpModel.transform(df)
         .withColumn(accColName, updateUDF(col(accColName), col(tmpRawPredName)))
-        .select(columns: _*)
+        .select(columns.toImmutableArraySeq: _*)

Review Comment:
   For the cases in the example module, it is recommended to directly use `toIndexedSeq` or `ArraySeq. unsafeWrapArra` because `ArrayImplicits` is `private[spark]`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1382871796


##########
project/SparkBuild.scala:
##########
@@ -254,7 +254,8 @@ object SparkBuild extends PomBuild {
         // SPARK-45627 `enum`, `export` and `given` will become keywords in Scala 3,
         // so they are prohibited from being used as variable names in Scala 2.13 to
         // reduce the cost of migration in subsequent versions.
-        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e"
+        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e",
+        "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e"

Review Comment:
   In the end, we may not need to add this new compile option.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1383154286


##########
sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala:
##########
@@ -540,21 +541,21 @@ class ParametersSuite extends QueryTest with SharedSparkSession {
     }
     def createMap(keys: Array[_], values: Array[_]): Column = {
       val zipped = keys.map(k => Column(Literal(k))).zip(values.map(v => Column(Literal(v))))
-      map(zipped.map { case (k, v) => Seq(k, v) }.flatten: _*)
+      map(zipped.map { case (k, v) => Seq(k, v) }.flatten.toImmutableArraySeq: _*)
     }
     def callMap(keys: Array[_], values: Array[_]): Column = {
       val zipped = keys.map(k => Column(Literal(k))).zip(values.map(v => Column(Literal(v))))
-      call_function("map", zipped.map { case (k, v) => Seq(k, v) }.flatten: _*)
+      call_function("map", zipped.map { case (k, v) => Seq(k, v) }.flatten.toImmutableArraySeq: _*)

Review Comment:
   Thanks, done.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1384955151


##########
project/SparkBuild.scala:
##########
@@ -254,7 +254,8 @@ object SparkBuild extends PomBuild {
         // SPARK-45627 `enum`, `export` and `given` will become keywords in Scala 3,
         // so they are prohibited from being used as variable names in Scala 2.13 to
         // reduce the cost of migration in subsequent versions.
-        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e"
+        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e",
+        "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e"

Review Comment:
   removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416698056


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   `ArrayImplicits` is currently in the `private[spark]` scope, should we expose them in the examples code? Sorry, I'm not very clear about this rule.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416697880


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   oh, because `examples` do not depend on common/utils



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1416718851


##########
examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala:
##########
@@ -51,7 +51,7 @@ object Analytics {
         case _ => throw new IllegalArgumentException(s"Invalid argument: $arg")
       }
     }
-    val options = mutable.Map(optionsList: _*)
+    val options = mutable.Map(immutable.ArraySeq.unsafeWrapArray(optionsList): _*)

Review Comment:
   It's fine to make `examples` an exception.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1385077519


##########
project/SparkBuild.scala:
##########
@@ -254,7 +254,8 @@ object SparkBuild extends PomBuild {
         // SPARK-45627 `enum`, `export` and `given` will become keywords in Scala 3,
         // so they are prohibited from being used as variable names in Scala 2.13 to
         // reduce the cost of migration in subsequent versions.
-        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e"
+        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e",
+        "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e"

Review Comment:
   > Shall we keep it to avoid other folks adding the case again?
   
   This will cause some memory consumption and performance difference in collection copy, so it is indeed a problem. However, I prefer to wait for a while and see if the related cases increase rapidly again. If so, we can clean them up again and make it a stricter compile check.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][CORE][SQL][ML][MLLIB][KUBERNETES][EXAMPLES][CONNECT][STRUCTURED STREAMING] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "srowen (via GitHub)" <gi...@apache.org>.
srowen commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1805811299

   Merged to master


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1382888133


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRDD.scala:
##########
@@ -89,7 +89,8 @@ object JDBCRDD extends Logging {
    * @return A Catalyst schema corresponding to columns in the given order.
    */
   private def pruneSchema(schema: StructType, columns: Array[String]): StructType = {
-    val fieldMap = Map(schema.fields.map(x => x.name -> x): _*)
+    import org.apache.spark.util.ArrayImplicits._
+    val fieldMap = Map(schema.fields.map(x => x.name -> x).toImmutableArraySeq: _*)

Review Comment:
   how about `val fieldMap = schema.fields.map(x => x.name -> x).toMap`?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1791840682

   Could you check again? IIRC, there should be more than 40+ files involved in this issue...


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1383153579


##########
project/SparkBuild.scala:
##########
@@ -254,7 +254,8 @@ object SparkBuild extends PomBuild {
         // SPARK-45627 `enum`, `export` and `given` will become keywords in Scala 3,
         // so they are prohibited from being used as variable names in Scala 2.13 to
         // reduce the cost of migration in subsequent versions.
-        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e"
+        "-Wconf:cat=deprecation&msg=it will become a keyword in Scala 3:e",
+        "-Wconf:cat=deprecation&msg=Passing an explicit array value to a Scala varargs method is deprecated:e"

Review Comment:
   Shall we keep it to avoid other folks adding the case again?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1382890921


##########
sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala:
##########
@@ -540,21 +541,21 @@ class ParametersSuite extends QueryTest with SharedSparkSession {
     }
     def createMap(keys: Array[_], values: Array[_]): Column = {
       val zipped = keys.map(k => Column(Literal(k))).zip(values.map(v => Column(Literal(v))))
-      map(zipped.map { case (k, v) => Seq(k, v) }.flatten: _*)
+      map(zipped.map { case (k, v) => Seq(k, v) }.flatten.toImmutableArraySeq: _*)

Review Comment:
   ```suggestion
         map(zipped.flatMap { case (k, v) => Seq(k, v) }.toImmutableArraySeq: _*)
   ```



##########
sql/core/src/test/scala/org/apache/spark/sql/ParametersSuite.scala:
##########
@@ -540,21 +541,21 @@ class ParametersSuite extends QueryTest with SharedSparkSession {
     }
     def createMap(keys: Array[_], values: Array[_]): Column = {
       val zipped = keys.map(k => Column(Literal(k))).zip(values.map(v => Column(Literal(v))))
-      map(zipped.map { case (k, v) => Seq(k, v) }.flatten: _*)
+      map(zipped.map { case (k, v) => Seq(k, v) }.flatten.toImmutableArraySeq: _*)
     }
     def callMap(keys: Array[_], values: Array[_]): Column = {
       val zipped = keys.map(k => Column(Literal(k))).zip(values.map(v => Column(Literal(v))))
-      call_function("map", zipped.map { case (k, v) => Seq(k, v) }.flatten: _*)
+      call_function("map", zipped.map { case (k, v) => Seq(k, v) }.flatten.toImmutableArraySeq: _*)

Review Comment:
   ditto



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "ivoson (via GitHub)" <gi...@apache.org>.
ivoson commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1791831105

   cc @LuciferYang please take a look at this PR. Thanks.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on code in PR #43642:
URL: https://github.com/apache/spark/pull/43642#discussion_r1382870557


##########
core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala:
##########
@@ -179,7 +179,9 @@ private[spark] object PythonRDD extends Logging {
     type UnrolledPartition = Array[ByteArray]
     val allPartitions: Array[UnrolledPartition] =
       sc.runJob(rdd, (x: Iterator[ByteArray]) => x.toArray, partitions.asScala.toSeq)
-    val flattenedPartition: UnrolledPartition = Array.concat(allPartitions: _*)
+
+    import org.apache.spark.util.ArrayImplicits._

Review Comment:
   I think we can change them to top-level imports, then SPARK-45686 can reuse these imports.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-45687][SQL][MLLIB][KUBERNETES][EXAMPLES][CONNECT] Fix `Passing an explicit array value to a Scala varargs method is deprecated` [spark]

Posted by "LuciferYang (via GitHub)" <gi...@apache.org>.
LuciferYang commented on PR #43642:
URL: https://github.com/apache/spark/pull/43642#issuecomment-1794907059

   cc @srowen FYI


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org