From 8a4890ddb5541acacdde2b42fa0ff8781290c907 Mon Sep 17 00:00:00 2001 From: yangjie01 Date: Wed, 29 Nov 2023 13:09:48 -0600 Subject: [PATCH] [SPARK-46124][CORE][SQL][SS][CONNECT][DSTREAM][MLLIB][ML][PYTHON][R][AVRO][K8S][YARN][UI] Replace explicit `ArrayOps#toSeq` with `s.c.immutable.ArraySeq.unsafeWrapArray` ### What changes were proposed in this pull request? There is a behavioral difference between Scala 2.13 and 2.12 for explicit `ArrayOps.toSeq` calls, similar to the implicit conversion from `Array` to `Seq`. In Scala 2.12, `ArrayOps.toSeq` will return `thisCollection` , and use implicit conversion rules to wrap the `Array` as `mutable.WrappedArray`, this process does not involve any collection copy: ```scala Welcome to Scala 2.12.18 (OpenJDK 64-Bit Server VM, Java 17.0.9). Type in expressions for evaluation. Or try :help. scala> Array(1,2,3).toSeq res0: Seq[Int] = WrappedArray(1, 2, 3) ``` However, in Scala 2.13, it returns an `immutable.ArraySeq` that with collection copy. Since we have always used the non-collection copy behavior for this explicit conversion in the era of Scala 2.12, it is safe to assume that no collection copy is needed for Scala 2.13. Therefore, this pr replaces explicit `ArrayOps.toSeq` in the Spark code with `s.c.immutable.ArraySeq.unsafeWrapArray` to avoid a collection copy, and this pr only involves changes to the production code, and does not involve changes to the test code. ### Why are the changes needed? Replace `ArrayOps#toSeq` with `s.c.immutable.ArraySeq.unsafeWrapArray` to save a collection copy, which has potential benefits for performance." ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass GitHub Action ### Was this patch authored or co-authored using generative AI tooling? No Closes #44041 from LuciferYang/ArrayToSeq-2-ArrayToImmutableArraySeq. Authored-by: yangjie01 Signed-off-by: Sean Owen --- .../spark/sql/avro/AvroSerializer.scala | 3 ++- .../scala/org/apache/spark/sql/Column.scala | 3 ++- .../spark/sql/DataFrameNaFunctions.scala | 18 +++++++------ .../scala/org/apache/spark/sql/Dataset.scala | 14 +++++----- .../common/DataTypeProtoConverter.scala | 3 ++- .../spark/sql/connect/dsl/package.scala | 9 ++++--- .../connect/planner/SparkConnectPlanner.scala | 6 ++--- .../service/SparkConnectAnalyzeHandler.scala | 3 ++- .../sql/kafka010/KafkaOffsetReaderAdmin.scala | 2 +- .../kafka010/KafkaOffsetReaderConsumer.scala | 2 +- .../org/apache/spark/MapOutputTracker.scala | 5 ++-- .../scala/org/apache/spark/SparkConf.scala | 4 +-- .../apache/spark/api/java/JavaRDDLike.scala | 16 ++++++------ .../spark/api/java/JavaSparkContext.scala | 6 +++-- .../apache/spark/api/python/PythonRDD.scala | 3 ++- .../apache/spark/api/python/SerDeUtil.scala | 3 ++- .../apache/spark/api/r/RBackendHandler.scala | 3 ++- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../org/apache/spark/deploy/SparkSubmit.scala | 6 ++--- .../deploy/history/EventLogFileReaders.scala | 3 ++- .../deploy/history/FsHistoryProvider.scala | 3 ++- .../spark/deploy/master/ui/MasterWebUI.scala | 3 ++- .../deploy/rest/StandaloneRestServer.scala | 3 ++- .../apache/spark/deploy/worker/Worker.scala | 3 ++- .../apache/spark/rdd/AsyncRDDActions.scala | 3 ++- .../spark/rdd/ParallelCollectionRDD.scala | 3 ++- .../spark/resource/ResourceInformation.scala | 5 ++-- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../cluster/StandaloneSchedulerBackend.scala | 7 ++--- .../apache/spark/storage/BlockManager.scala | 2 +- .../spark/storage/DiskBlockManager.scala | 2 +- .../org/apache/spark/util/JsonProtocol.scala | 26 ++++++++++--------- .../org/apache/spark/ml/linalg/Matrices.scala | 5 ++-- .../scala/org/apache/spark/ml/Pipeline.scala | 3 ++- .../spark/ml/attribute/AttributeGroup.scala | 5 ++-- .../spark/ml/attribute/attributes.scala | 12 +++++---- .../org/apache/spark/ml/clustering/LDA.scala | 2 +- .../ml/evaluation/ClusteringMetrics.scala | 3 ++- .../apache/spark/ml/feature/Binarizer.scala | 9 ++++--- .../apache/spark/ml/feature/Bucketizer.scala | 5 ++-- .../spark/ml/feature/ChiSqSelector.scala | 3 ++- .../ml/feature/QuantileDiscretizer.scala | 3 ++- .../spark/ml/feature/StringIndexer.scala | 4 +-- .../apache/spark/ml/feature/Tokenizer.scala | 2 +- .../feature/UnivariateFeatureSelector.scala | 3 ++- .../feature/VarianceThresholdSelector.scala | 4 +-- .../spark/ml/feature/VectorAssembler.scala | 2 +- .../spark/ml/linalg/JsonMatrixConverter.scala | 10 ++++--- .../spark/ml/linalg/JsonVectorConverter.scala | 8 +++--- .../org/apache/spark/ml/param/params.scala | 10 ++++--- .../ml/r/AFTSurvivalRegressionWrapper.scala | 3 ++- .../spark/ml/r/BisectingKMeansWrapper.scala | 5 ++-- .../ml/r/DecisionTreeClassifierWrapper.scala | 3 ++- .../ml/r/DecisionTreeRegressorWrapper.scala | 3 ++- .../spark/ml/r/FMClassifierWrapper.scala | 5 ++-- .../spark/ml/r/FMRegressorWrapper.scala | 3 ++- .../spark/ml/r/GBTClassifierWrapper.scala | 3 ++- .../spark/ml/r/GBTRegressorWrapper.scala | 3 ++- .../GeneralizedLinearRegressionWrapper.scala | 5 ++-- .../ml/r/IsotonicRegressionWrapper.scala | 3 ++- .../org/apache/spark/ml/r/KMeansWrapper.scala | 5 ++-- .../spark/ml/r/LinearRegressionWrapper.scala | 3 ++- .../apache/spark/ml/r/LinearSVCWrapper.scala | 5 ++-- .../ml/r/LogisticRegressionWrapper.scala | 5 ++-- .../apache/spark/ml/r/NaiveBayesWrapper.scala | 5 ++-- .../ml/r/RandomForestClassifierWrapper.scala | 3 ++- .../ml/r/RandomForestRegressorWrapper.scala | 3 ++- .../spark/ml/tuning/CrossValidator.scala | 5 ++-- .../ml/tuning/TrainValidationSplit.scala | 5 ++-- .../spark/ml/tuning/ValidatorParams.scala | 3 ++- .../spark/ml/util/Instrumentation.scala | 7 ++--- .../mllib/api/python/PythonMLLibAPI.scala | 3 ++- .../spark/mllib/clustering/LDAModel.scala | 7 ++--- .../spark/mllib/fpm/AssociationRules.scala | 5 ++-- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 6 +++-- .../apache/spark/mllib/linalg/Matrices.scala | 5 ++-- .../apache/spark/mllib/linalg/Vectors.scala | 6 ++--- .../mllib/regression/IsotonicRegression.scala | 3 ++- .../mllib/tree/model/DecisionTreeModel.scala | 3 ++- .../HadoopConfDriverFeatureStep.scala | 3 ++- .../k8s/features/LocalDirsFeatureStep.scala | 2 +- .../k8s/submit/KubernetesClientUtils.scala | 3 ++- ...esLocalDiskShuffleExecutorComponents.scala | 3 ++- .../org/apache/spark/deploy/yarn/Client.scala | 11 ++++---- .../main/scala/org/apache/spark/sql/Row.scala | 3 ++- .../org/apache/spark/sql/types/Metadata.scala | 3 ++- .../apache/spark/sql/util/ArrowUtils.scala | 2 +- .../spark/sql/catalyst/StructFilters.scala | 3 ++- .../sql/catalyst/analysis/Analyzer.scala | 4 ++- .../catalyst/analysis/FunctionRegistry.scala | 5 ++-- .../sql/catalyst/analysis/unresolved.scala | 4 ++- .../sql/catalyst/catalog/SessionCatalog.scala | 5 ++-- .../expressions/ApplyFunctionExpression.scala | 2 +- .../expressions/V2ExpressionUtils.scala | 4 +-- .../expressions/aggregate/V2Aggregator.scala | 3 ++- .../expressions/conditionalExpressions.scala | 3 ++- .../expressions/objects/objects.scala | 4 +-- .../sql/catalyst/json/JsonInferSchema.scala | 5 ++-- .../optimizer/NormalizeFloatingNumbers.scala | 3 ++- .../plans/logical/FunctionBuilderBase.scala | 3 ++- .../spark/sql/DataFrameNaFunctions.scala | 18 +++++++------ .../apache/spark/sql/DataFrameWriter.scala | 4 ++- .../scala/org/apache/spark/sql/Dataset.scala | 12 +++++---- .../spark/sql/RelationalGroupedDataset.scala | 3 ++- .../spark/sql/execution/GenerateExec.scala | 5 ++-- .../spark/sql/execution/HiveResult.scala | 2 +- .../execution/columnar/InMemoryRelation.scala | 2 +- .../command/createDataSourceTables.scala | 3 ++- .../PlanPythonDataSourceScan.scala | 3 ++- .../datasources/parquet/ParquetFilters.scala | 3 ++- .../datasources/v2/DataSourceV2Strategy.scala | 2 +- .../v2/DistributionAndOrderingUtils.scala | 3 ++- .../execution/datasources/v2/FileTable.scala | 4 ++- .../datasources/v2/PushDownUtils.scala | 4 +-- .../datasources/v2/V2SessionCatalog.scala | 3 ++- .../execution/datasources/v2/V2Writes.scala | 3 ++- .../datasources/v2/state/StateTable.scala | 3 ++- .../continuous/ContinuousExecution.scala | 3 ++- .../streaming/state/RocksDBFileManager.scala | 4 +-- .../execution/window/AggregateProcessor.scala | 4 ++- .../window/WindowFunctionFrame.scala | 5 ++-- .../spark/sql/internal/CatalogImpl.scala | 3 ++- .../sql/streaming/DataStreamWriter.scala | 3 ++- .../org/apache/spark/sql/hive/HiveUtils.scala | 6 +++-- .../apache/spark/sql/hive/TableReader.scala | 2 +- .../streaming/api/java/JavaDStreamLike.scala | 3 ++- 126 files changed, 349 insertions(+), 231 deletions(-) diff --git a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala index 5d052c3672bda..1d9eada94658e 100644 --- a/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala +++ b/connector/avro/src/main/scala/org/apache/spark/sql/avro/AvroSerializer.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.DataSourceUtils import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ /** * A serializer to serialize data in catalyst format to data in avro format. @@ -309,7 +310,7 @@ private[sql] class AvroSerializer( avroPath: Seq[String]): InternalRow => Any = { val nonNullTypes = nonNullUnionBranches(unionType) val expectedFieldNames = nonNullTypes.indices.map(i => s"member$i") - val catalystFieldNames = catalystStruct.fieldNames.toSeq + val catalystFieldNames = catalystStruct.fieldNames.toImmutableArraySeq if (positionalFieldMatch) { if (expectedFieldNames.length != catalystFieldNames.length) { throw new IncompatibleSchemaException(s"Generic Avro union at ${toFieldStr(avroPath)} " + diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala index 326b9ee2c7a85..140c4ce7cd94b 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Column.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions.lit import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ /** * A column that will be computed based on the data in a `DataFrame`. @@ -1004,7 +1005,7 @@ class Column private[sql] (@DeveloperApi val expr: proto.Expression) extends Log * @group expr_ops * @since 3.4.0 */ - def as(aliases: Array[String]): Column = as(aliases.toSeq) + def as(aliases: Array[String]): Column = as(aliases.toImmutableArraySeq) /** * Gives the column an alias. diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 37c2ae1b26288..04970f67e99f5 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -24,6 +24,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.connect.proto.{NAReplace, Relation} import org.apache.spark.connect.proto.Expression.{Literal => GLiteral} import org.apache.spark.connect.proto.NAReplace.Replacement +import org.apache.spark.util.ArrayImplicits._ /** * Functionality for working with missing data in `DataFrame`s. @@ -57,7 +58,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def drop(cols: Array[String]): DataFrame = drop(cols.toSeq) + def drop(cols: Array[String]): DataFrame = drop(cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that drops rows containing any null or NaN values @@ -76,7 +77,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def drop(how: String, cols: Array[String]): DataFrame = drop(how, cols.toSeq) + def drop(how: String, cols: Array[String]): DataFrame = drop(how, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that drops rows containing null or NaN values in @@ -107,7 +108,8 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def drop(minNonNulls: Int, cols: Array[String]): DataFrame = drop(minNonNulls, cols.toSeq) + def drop(minNonNulls: Int, cols: Array[String]): DataFrame = + drop(minNonNulls, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that drops rows containing less than `minNonNulls` @@ -152,7 +154,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def fill(value: Long, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: Long, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that replaces null or NaN values in specified @@ -179,7 +181,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def fill(value: Double, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: Double, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that replaces null or NaN values in specified @@ -206,7 +208,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def fill(value: String, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: String, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that replaces null values in specified string @@ -233,7 +235,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * * @since 3.4.0 */ - def fill(value: Boolean, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: Boolean, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that replaces null values in specified boolean @@ -374,7 +376,7 @@ final class DataFrameNaFunctions private[sql] (sparkSession: SparkSession, root: * @since 3.4.0 */ def replace[T](cols: Array[String], replacement: java.util.Map[T, T]): DataFrame = { - replace(cols.toSeq, replacement.asScala.toMap) + replace(cols.toImmutableArraySeq, replacement.asScala.toMap) } /** diff --git a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala index d760c9d97693b..801696c1b79e1 100644 --- a/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/connector/connect/client/jvm/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.functions.{struct, to_json} import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types.{Metadata, StructType} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkClassUtils /** @@ -650,7 +651,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def join(right: Dataset[_], usingColumns: Array[String]): DataFrame = { - join(right, usingColumns.toSeq) + join(right, usingColumns.toImmutableArraySeq) } /** @@ -729,7 +730,7 @@ class Dataset[T] private[sql] ( * @since 3.4.0 */ def join(right: Dataset[_], usingColumns: Array[String], joinType: String): DataFrame = { - join(right, usingColumns.toSeq, joinType) + join(right, usingColumns.toImmutableArraySeq, joinType) } /** @@ -1306,12 +1307,12 @@ class Dataset[T] private[sql] ( valueColumnName: String): DataFrame = sparkSession.newDataFrame { builder => val unpivot = builder.getUnpivotBuilder .setInput(plan.getRoot) - .addAllIds(ids.toSeq.map(_.expr).asJava) + .addAllIds(ids.toImmutableArraySeq.map(_.expr).asJava) .setValueColumnName(variableColumnName) .setValueColumnName(valueColumnName) valuesOption.foreach { values => unpivot.getValuesBuilder - .addAllValues(values.toSeq.map(_.expr).asJava) + .addAllValues(values.toImmutableArraySeq.map(_.expr).asJava) } } @@ -2496,7 +2497,8 @@ class Dataset[T] private[sql] ( * @group typedrel * @since 3.4.0 */ - def dropDuplicates(colNames: Array[String]): Dataset[T] = dropDuplicates(colNames.toSeq) + def dropDuplicates(colNames: Array[String]): Dataset[T] = + dropDuplicates(colNames.toImmutableArraySeq) /** * Returns a new [[Dataset]] with duplicate rows removed, considering only the subset of @@ -2518,7 +2520,7 @@ class Dataset[T] private[sql] ( } def dropDuplicatesWithinWatermark(colNames: Array[String]): Dataset[T] = { - dropDuplicatesWithinWatermark(colNames.toSeq) + dropDuplicatesWithinWatermark(colNames.toImmutableArraySeq) } @scala.annotation.varargs diff --git a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala index 34b5aae21dcb5..90776c3c5c788 100644 --- a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala +++ b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/common/DataTypeProtoConverter.scala @@ -21,6 +21,7 @@ import scala.jdk.CollectionConverters._ import org.apache.spark.connect.proto import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.SparkClassUtils /** @@ -225,7 +226,7 @@ object DataTypeProtoConverter { .build() case StructType(fields: Array[StructField]) => - val protoFields = fields.toSeq.map { + val protoFields = fields.toImmutableArraySeq.map { case StructField( name: String, dataType: DataType, diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala index 18c71ae4ace45..ccdae26bd35a6 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/dsl/package.scala @@ -29,6 +29,7 @@ import org.apache.spark.sql.connect.common.DataTypeProtoConverter import org.apache.spark.sql.connect.common.LiteralValueProtoConverter.toLiteralProto import org.apache.spark.sql.connect.planner.{SaveModeConverter, TableSaveMethodConverter} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** @@ -473,8 +474,8 @@ package object dsl { proto.StatApproxQuantile .newBuilder() .setInput(logicalPlan) - .addAllCols(cols.toSeq.asJava) - .addAllProbabilities(probabilities.toSeq.map(Double.box).asJava) + .addAllCols(cols.toImmutableArraySeq.asJava) + .addAllProbabilities(probabilities.toImmutableArraySeq.map(Double.box).asJava) .setRelativeError(relativeError) .build()) .build() @@ -500,7 +501,7 @@ package object dsl { proto.StatFreqItems .newBuilder() .setInput(logicalPlan) - .addAllCols(cols.toSeq.asJava) + .addAllCols(cols.toImmutableArraySeq.asJava) .setSupport(support) .build()) .build() @@ -1082,7 +1083,7 @@ package object dsl { weights.sum > 0, s"Sum of weights must be positive, but got ${weights.mkString("[", ",", "]")}") - val sum = weights.toSeq.sum + val sum = weights.toImmutableArraySeq.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights .sliding(2) diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala index abfc063139056..4f8b7c008887b 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/planner/SparkConnectPlanner.scala @@ -379,7 +379,7 @@ class SparkConnectPlanner( val values = rel.getValuesList.asScala.toArray if (values.length == 1) { val value = LiteralValueProtoConverter.toCatalystValue(values.head) - val columns = if (cols.nonEmpty) Some(cols.toSeq) else None + val columns = if (cols.nonEmpty) Some(cols.toImmutableArraySeq) else None dataset.na.fillValue(value, columns).logicalPlan } else { val valueMap = mutable.Map.empty[String, Any] @@ -2434,7 +2434,7 @@ class SparkConnectPlanner( .sort(pivotCol) // ensure that the output columns are in a consistent logical order .collect() .map(_.get(0)) - .toSeq + .toImmutableArraySeq .map(expressions.Literal.apply) } @@ -3073,7 +3073,7 @@ class SparkConnectPlanner( val progressReports = if (command.getLastProgress) { Option(query.lastProgress).toSeq } else { - query.recentProgress.toSeq + query.recentProgress.toImmutableArraySeq } respBuilder.setRecentProgress( StreamingQueryCommandResult.RecentProgressResult diff --git a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala index f6fb42d9fcaa0..7a701aea1b78b 100644 --- a/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala +++ b/connector/connect/server/src/main/scala/org/apache/spark/sql/connect/service/SparkConnectAnalyzeHandler.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.Dataset import org.apache.spark.sql.connect.common.{DataTypeProtoConverter, InvalidPlanInput, StorageLevelProtoConverter} import org.apache.spark.sql.connect.planner.SparkConnectPlanner import org.apache.spark.sql.execution.{CodegenMode, CostMode, ExtendedMode, FormattedMode, SimpleMode} +import org.apache.spark.util.ArrayImplicits._ private[connect] class SparkConnectAnalyzeHandler( responseObserver: StreamObserver[proto.AnalyzePlanResponse]) @@ -128,7 +129,7 @@ private[connect] class SparkConnectAnalyzeHandler( builder.setInputFiles( proto.AnalyzePlanResponse.InputFiles .newBuilder() - .addAllFiles(inputFiles.toSeq.asJava) + .addAllFiles(inputFiles.toImmutableArraySeq.asJava) .build()) case proto.AnalyzePlanRequest.AnalyzeCase.SPARK_VERSION => diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala index e13c79625cc55..9206dfe9b3f2f 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderAdmin.scala @@ -419,7 +419,7 @@ private[kafka010] class KafkaOffsetReaderAdmin( val end = splitOffsetRanges.last.copy(untilOffset = untilOffsetsMap(tp)) Seq(first) ++ splitOffsetRanges.drop(1).dropRight(1) :+ end } - }.toArray.toSeq + }.toArray.toImmutableArraySeq } else { offsetRangesBase } diff --git a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala index c4ac4c7d57db2..a859cd3d55a9b 100644 --- a/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala +++ b/connector/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReaderConsumer.scala @@ -466,7 +466,7 @@ private[kafka010] class KafkaOffsetReaderConsumer( val end = splitOffsetRanges.last.copy(untilOffset = untilOffsetsMap(tp)) Seq(first) ++ splitOffsetRanges.drop(1).dropRight(1) :+ end } - }.toArray.toSeq + }.toArray.toImmutableArraySeq } else { offsetRangesBase } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index a787cdefe8085..851ecfc662e49 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -42,6 +42,7 @@ import org.apache.spark.scheduler.{MapStatus, MergeStatus, ShuffleOutputStatus} import org.apache.spark.shuffle.MetadataFetchFailedException import org.apache.spark.storage.{BlockId, BlockManagerId, ShuffleBlockId, ShuffleMergedBlockId} import org.apache.spark.util._ +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -1054,7 +1055,7 @@ private[spark] class MapOutputTrackerMaster( val blockManagerIds = getLocationsWithLargestOutputs(dep.shuffleId, partitionId, dep.partitioner.numPartitions, REDUCER_PREF_LOCS_FRACTION) if (blockManagerIds.nonEmpty) { - blockManagerIds.get.map(_.host).distinct.toSeq + blockManagerIds.get.map(_.host).distinct.toImmutableArraySeq } else { Nil } @@ -1142,7 +1143,7 @@ private[spark] class MapOutputTrackerMaster( if (startMapIndex < endMapIndex && (startMapIndex >= 0 && endMapIndex <= statuses.length)) { val statusesPicked = statuses.slice(startMapIndex, endMapIndex).filter(_ != null) - statusesPicked.map(_.location.host).distinct.toSeq + statusesPicked.map(_.location.host).distinct.toImmutableArraySeq } else { Nil } diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index af33ef2415783..f1ef36bbf19c2 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -129,7 +129,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ def setJars(jars: Array[String]): SparkConf = { - setJars(jars.toSeq) + setJars(jars.toImmutableArraySeq) } /** @@ -158,7 +158,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria * (Java-friendly version.) */ def setExecutorEnv(variables: Array[(String, String)]): SparkConf = { - setExecutorEnv(variables.toSeq) + setExecutorEnv(variables.toImmutableArraySeq) } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 9bad4d9e163df..f90ed8b9e9d8b 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -59,7 +59,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def rdd: RDD[T] /** Set of partitions in this RDD. */ - def partitions: JList[Partition] = rdd.partitions.toSeq.asJava + def partitions: JList[Partition] = rdd.partitions.toImmutableArraySeq.asJava /** Return the number of partitions in this RDD. */ @Since("1.6.0") @@ -224,7 +224,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD created by coalescing all elements within each partition into an array. */ def glom(): JavaRDD[JList[T]] = - new JavaRDD(rdd.glom().map(_.toSeq.asJava)) + new JavaRDD(rdd.glom().map(_.toImmutableArraySeq.asJava)) /** * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of @@ -360,7 +360,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * all the data is loaded into the driver's memory. */ def collect(): JList[T] = - rdd.collect().toSeq.asJava + rdd.collect().toImmutableArraySeq.asJava /** * Return an iterator that contains all of the elements in this RDD. @@ -377,7 +377,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { // This is useful for implementing `take` from other language frontends // like Python where the data is serialized. val res = context.runJob(rdd, (it: Iterator[T]) => it.toArray, partitionIds.toImmutableArraySeq) - res.map(_.toSeq.asJava) + res.map(_.toImmutableArraySeq.asJava) } /** @@ -539,13 +539,13 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * all the data is loaded into the driver's memory. */ def take(num: Int): JList[T] = - rdd.take(num).toSeq.asJava + rdd.take(num).toImmutableArraySeq.asJava def takeSample(withReplacement: Boolean, num: Int): JList[T] = takeSample(withReplacement, num, Utils.random.nextLong) def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = - rdd.takeSample(withReplacement, num, seed).toSeq.asJava + rdd.takeSample(withReplacement, num, seed).toImmutableArraySeq.asJava /** * Return the first element in this RDD. @@ -628,7 +628,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * @return an array of top elements */ def top(num: Int, comp: Comparator[T]): JList[T] = { - rdd.top(num)(Ordering.comparatorToOrdering(comp)).toSeq.asJava + rdd.top(num)(Ordering.comparatorToOrdering(comp)).toImmutableArraySeq.asJava } /** @@ -656,7 +656,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * @return an array of top elements */ def takeOrdered(num: Int, comp: Comparator[T]): JList[T] = { - rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)).toSeq.asJava + rdd.takeOrdered(num)(Ordering.comparatorToOrdering(comp)).toImmutableArraySeq.asJava } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index c016910ed760e..1c6c44e773dbd 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -36,6 +36,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD} import org.apache.spark.resource.ResourceInformation +import org.apache.spark.util.ArrayImplicits._ /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns @@ -89,7 +90,7 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { * system or HDFS, HTTP, HTTPS, or FTP URLs. */ def this(master: String, appName: String, sparkHome: String, jars: Array[String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq)) + this(new SparkContext(master, appName, sparkHome, jars.toImmutableArraySeq)) /** * @param master Cluster URL to connect to (e.g. spark://host:port, local[4]). @@ -101,7 +102,8 @@ class JavaSparkContext(val sc: SparkContext) extends Closeable { */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment.asScala)) + this( + new SparkContext(master, appName, sparkHome, jars.toImmutableArraySeq, environment.asScala)) private[spark] val env = sc.env diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e98259562c92f..1d0c905164ad4 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -106,7 +106,8 @@ private[spark] case class SimplePythonFunction( pythonVer: String, broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: PythonAccumulatorV2) = { - this(command.toSeq, envVars, pythonIncludes, pythonExec, pythonVer, broadcastVars, accumulator) + this(command.toImmutableArraySeq, + envVars, pythonIncludes, pythonExec, pythonVer, broadcastVars, accumulator) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala index 4b6d0768005bc..6a46b611019fa 100644 --- a/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/SerDeUtil.scala @@ -30,6 +30,7 @@ import org.apache.spark.SparkException import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.util.ArrayImplicits._ /** Utilities for serialization / deserialization between Python and Java, using Pickle. */ private[spark] object SerDeUtil extends Logging { @@ -121,7 +122,7 @@ private[spark] object SerDeUtil extends Logging { val obj = unpickle.loads(row) if (batched) { obj match { - case array: Array[Any] => array.toSeq + case array: Array[Any] => array.toImmutableArraySeq case _ => obj.asInstanceOf[JArrayList[_]].asScala } } else { diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 6925c3f65c09c..3f7a3ea70a7e7 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -29,6 +29,7 @@ import org.apache.spark.api.r.SerDe._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.R._ import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.ArrayImplicits._ /** * Handler for RBackend @@ -266,7 +267,7 @@ private[r] class RBackendHandler(server: RBackend) for (i <- 0 until numArgs) { if (parameterTypes(i) == classOf[Seq[Any]] && args(i).getClass.isArray) { // Convert a Java array to scala Seq - args(i) = args(i).asInstanceOf[Array[_]].toSeq + args(i) = args(i).asInstanceOf[Array[_]].toImmutableArraySeq } } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index dab880b37dc22..50906f76b6e10 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -237,7 +237,7 @@ private[spark] class SparkHadoopUtil extends Logging { def globPath(fs: FileSystem, pattern: Path): Seq[Path] = { Option(fs.globStatus(pattern)).map { statuses => - statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toSeq + statuses.map(_.getPath.makeQualified(fs.getUri, fs.getWorkingDirectory)).toImmutableArraySeq }.getOrElse(Seq.empty[Path]) } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 29bb8f84ee6cb..5e7dc799ab071 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -539,7 +539,7 @@ private[spark] class SparkSubmit extends Logging { } if (localPyFiles != null) { - sparkConf.set(SUBMIT_PYTHON_FILES, localPyFiles.split(",").toSeq) + sparkConf.set(SUBMIT_PYTHON_FILES, localPyFiles.split(",").toImmutableArraySeq) } // In YARN mode for an R app, add the SparkR package archive and the R package @@ -861,7 +861,7 @@ private[spark] class SparkSubmit extends Logging { // locally. resolvedPyFiles } - sparkConf.set(SUBMIT_PYTHON_FILES, formattedPyFiles.split(",").toSeq) + sparkConf.set(SUBMIT_PYTHON_FILES, formattedPyFiles.split(",").toImmutableArraySeq) if (args.verbose && isSqlShell(childMainClass)) { childArgs ++= Seq("--verbose") @@ -1162,7 +1162,7 @@ private[spark] object SparkSubmitUtils { } def parseSparkConfProperty(pair: String): (String, String) = { - pair.split("=", 2).toSeq match { + pair.split("=", 2).toImmutableArraySeq match { case Seq(k, v) => (k, v) case _ => throw new SparkException(s"Spark config without '=': $pair") } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala index 8c3dda4727784..8827fcde7b733 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileReaders.scala @@ -29,6 +29,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.history.EventLogFileWriter.codecName import org.apache.spark.internal.Logging import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** The base class of reader which will read the information of event log file(s). */ @@ -220,7 +221,7 @@ private[history] class RollingEventLogFilesFileReader( import RollingEventLogFilesWriter._ private lazy val files: Seq[FileStatus] = { - val ret = fs.listStatus(rootPath).toSeq + val ret = fs.listStatus(rootPath).toImmutableArraySeq require(ret.exists(isEventLogFile), "Log directory must contain at least one event log file!") require(ret.exists(isAppStatusFile), "Log directory must contain an appstatus file!") ret diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index e76d72194c39f..565499bb610b7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -457,7 +457,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // right after this check and before the check for stale entities will be identified as stale // and will be deleted from the UI until the next 'checkForLogs' run. val notStale = mutable.HashSet[String]() - val updated = Option(fs.listStatus(new Path(logDir))).map(_.toSeq).getOrElse(Nil) + val updated = Option(fs.listStatus(new Path(logDir))) + .map(_.toImmutableArraySeq).getOrElse(Nil) .filter { entry => isAccessible(entry.getPath) } .filter { entry => if (isProcessing(entry.getPath)) { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index af94bd6d9e0f2..8f65ca204b3c0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.config.UI.MASTER_UI_DECOMMISSION_ALLOW_MODE import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.ArrayImplicits._ /** * Web UI server for the standalone master. @@ -57,7 +58,7 @@ class MasterWebUI( attachHandler(createServletHandler("/workers/kill", new HttpServlet { override def doPost(req: HttpServletRequest, resp: HttpServletResponse): Unit = { val hostnames: Seq[String] = Option(req.getParameterValues("host")) - .getOrElse(Array[String]()).toSeq + .getOrElse(Array[String]()).toImmutableArraySeq if (!isDecommissioningRequestAllowed(req)) { resp.sendError(HttpServletResponse.SC_METHOD_NOT_ALLOWED) } else { diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index d382ec12847dd..fa1ac80d25d5c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.config import org.apache.spark.launcher.{JavaModuleOptions, SparkLauncher} import org.apache.spark.resource.ResourceUtils import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** @@ -207,7 +208,7 @@ private[rest] class StandaloneSubmitRequestServlet( .getOrElse(Seq.empty) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) - val javaModuleOptions = JavaModuleOptions.defaultModuleOptionArray().toSeq + val javaModuleOptions = JavaModuleOptions.defaultModuleOptionArray().toImmutableArraySeq val javaOpts = javaModuleOptions ++ sparkJavaOpts ++ defaultJavaOpts ++ extraJavaOpts val command = new Command( "org.apache.spark.deploy.worker.DriverWrapper", diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index ddbba55e00b44..eae12648b95ab 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -45,6 +45,7 @@ import org.apache.spark.resource.ResourceInformation import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ import org.apache.spark.util.{RpcUtils, SignalUtils, SparkUncaughtExceptionHandler, ThreadUtils, Utils} +import org.apache.spark.util.ArrayImplicits._ private[deploy] class Worker( override val rpcEnv: RpcEnv, @@ -596,7 +597,7 @@ private[deploy] class Worker( logWarning(s"${e.getMessage}. Ignoring this directory.") None } - }.toSeq + }.toImmutableArraySeq if (dirs.isEmpty) { throw new IOException("No subfolder can be created in " + s"${localRootDirs.mkString(",")}.") diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 612a3da048566..7a9c0263631f3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -26,6 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, JobSubmitter} import org.apache.spark.internal.Logging import org.apache.spark.internal.config.{RDD_LIMIT_INITIAL_NUM_PARTITIONS, RDD_LIMIT_SCALE_UP_FACTOR} +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.ThreadUtils /** @@ -59,7 +60,7 @@ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Loggi def collectAsync(): FutureAction[Seq[T]] = self.withScope { val results = new Array[Array[T]](self.partitions.length) self.context.submitJob[T, Array[T], Seq[T]](self, _.toArray, Range(0, self.partitions.length), - (index, data) => results(index) = data, results.flatten.toSeq) + (index, data) => results(index) = data, results.flatten.toImmutableArraySeq) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala index f0239cdd9136d..cbbfa4274cff1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ParallelCollectionRDD.scala @@ -26,6 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils private[spark] class ParallelCollectionPartition[T: ClassTag]( @@ -149,7 +150,7 @@ private object ParallelCollectionRDD { case _ => val array = seq.toArray // To prevent O(n^2) operations for List etc positions(array.length, numSlices).map { case (start, end) => - array.slice(start, end).toSeq + array.slice(start, end).toImmutableArraySeq }.toSeq } } diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala index c9e5ba1ad8e04..51b6f19513ba5 100644 --- a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala +++ b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala @@ -48,13 +48,14 @@ class ResourceInformation( obj match { case that: ResourceInformation => that.getClass == this.getClass && - that.name == name && that.addresses.toSeq == addresses.toSeq + that.name == name && + that.addresses.toImmutableArraySeq == addresses.toImmutableArraySeq case _ => false } } - override def hashCode(): Int = Seq(name, addresses.toSeq).hashCode() + override def hashCode(): Int = Seq(name, addresses.toImmutableArraySeq).hashCode() // TODO(SPARK-39658): reconsider whether we want to expose a third-party library's // symbols as part of a public API: diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index f23902eb68a91..e8519b4bb4d46 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -151,7 +151,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // scheduler can modify the SparkConf object before this view is created. private lazy val sparkProperties = scheduler.sc.conf.getAll .filter { case (k, _) => k.startsWith("spark.") } - .toSeq + .toImmutableArraySeq private val logUrlHandler: ExecutorLogUrlHandler = new ExecutorLogUrlHandler( conf.get(UI.CUSTOM_EXECUTOR_LOG_URL)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index 482e18640411a..8da1a568cb4cb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -36,6 +36,7 @@ import org.apache.spark.rpc.{RpcAddress, RpcEndpointAddress} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.ArrayImplicits._ /** * A [[SchedulerBackend]] implementation for Spark's standalone cluster manager. @@ -94,16 +95,16 @@ private[spark] class StandaloneSchedulerBackend( val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS) .map(Utils.splitCommandString).getOrElse(Seq.empty) val classPathEntries = sc.conf.get(config.EXECUTOR_CLASS_PATH) - .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) + .map(_.split(java.io.File.pathSeparator).toImmutableArraySeq).getOrElse(Nil) val libraryPathEntries = sc.conf.get(config.EXECUTOR_LIBRARY_PATH) - .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) + .map(_.split(java.io.File.pathSeparator).toImmutableArraySeq).getOrElse(Nil) // When testing, expose the parent class path to the child. This is processed by // compute-classpath.{cmd,sh} and makes all needed jars available to child processes // when the assembly is built with the "*-provided" profiles enabled. val testingClassPath = if (sys.props.contains(IS_TESTING.key)) { - sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq + sys.props("java.class.path").split(java.io.File.pathSeparator).toImmutableArraySeq } else { Nil } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 8c22f8473e698..e996444d78467 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -848,7 +848,7 @@ private[spark] class BlockManager( (blockInfoManager.entries.map(_._1) ++ diskBlockManager.getAllBlocks()) .filter(filter) .toArray - .toSeq + .toImmutableArraySeq } /** diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 512ee3cc806fb..7446a55fc7c37 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -168,7 +168,7 @@ private[spark] class DiskBlockManager( } }.filter(_ != null).flatMap { dir => val files = dir.listFiles() - if (files != null) files.toSeq else Seq.empty + if (files != null) files.toImmutableArraySeq else Seq.empty }.toImmutableArraySeq } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 02f4912326365..987e5b4328f9b 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -957,11 +957,12 @@ private[spark] object JsonProtocol extends JsonUtils { val jobId = json.get("Job ID").extractInt val submissionTime = jsonOption(json.get("Submission Time")).map(_.extractLong).getOrElse(-1L) - val stageIds = json.get("Stage IDs").extractElements.map(_.extractInt).toArray.toSeq + val stageIds = + json.get("Stage IDs").extractElements.map(_.extractInt).toArray.toImmutableArraySeq val properties = propertiesFromJson(json.get("Properties")) // The "Stage Infos" field was added in Spark 1.2.0 val stageInfos = jsonOption(json.get("Stage Infos")) - .map(_.extractElements.map(stageInfoFromJson).toArray.toSeq).getOrElse { + .map(_.extractElements.map(stageInfoFromJson).toArray.toImmutableArraySeq).getOrElse { stageIds.map { id => new StageInfo(id, 0, "unknown", 0, Seq.empty, Seq.empty, "unknown", resourceProfileId = ResourceProfile.DEFAULT_RESOURCE_PROFILE_ID) @@ -1093,9 +1094,10 @@ private[spark] object JsonProtocol extends JsonUtils { val stageId = json.get("Stage ID").extractInt val stageAttemptId = json.get("Stage Attempt ID").extractInt val updates = - json.get("Accumulator Updates").extractElements.map(accumulableInfoFromJson).toArray.toSeq + json.get("Accumulator Updates").extractElements.map(accumulableInfoFromJson) + .toArray.toImmutableArraySeq (taskId, stageId, stageAttemptId, updates) - }.toArray.toSeq + }.toArray.toImmutableArraySeq val executorUpdates = jsonOption(json.get("Executor Metrics Updated")).map { value => value.extractElements.map { json => val stageId = json.get("Stage ID").extractInt @@ -1131,7 +1133,7 @@ private[spark] object JsonProtocol extends JsonUtils { val numTasks = json.get("Number of Tasks").extractInt val rddInfos = json.get("RDD Info").extractElements.map(rddInfoFromJson).toArray val parentIds = jsonOption(json.get("Parent IDs")) - .map { l => l.extractElements.map(_.extractInt).toArray.toSeq } + .map { l => l.extractElements.map(_.extractInt).toArray.toImmutableArraySeq } .getOrElse(Seq.empty) val details = jsonOption(json.get("Details")).map(_.asText).getOrElse("") val submissionTime = jsonOption(json.get("Submission Time")).map(_.extractLong) @@ -1179,7 +1181,7 @@ private[spark] object JsonProtocol extends JsonUtils { val failed = json.get("Failed").extractBoolean val killed = jsonOption(json.get("Killed")).exists(_.extractBoolean) val accumulables = jsonOption(json.get("Accumulables")).map(_.extractElements) match { - case Some(values) => values.map(accumulableInfoFromJson).toArray.toSeq + case Some(values) => values.map(accumulableInfoFromJson).toArray.toImmutableArraySeq case None => Seq.empty[AccumulableInfo] } @@ -1224,7 +1226,7 @@ private[spark] object JsonProtocol extends JsonUtils { val id = BlockId(blockJson.get("Block ID").extractString) val status = blockStatusFromJson(blockJson.get("Status")) (id, status) - }.toArray.toSeq.asJava + }.toArray.toImmutableArraySeq.asJava } else { throw new IllegalArgumentException(s"unexpected json value $value for " + "accumulator " + name.get) @@ -1327,7 +1329,7 @@ private[spark] object JsonProtocol extends JsonUtils { val id = BlockId(blockJson.get("Block ID").extractString) val status = blockStatusFromJson(blockJson.get("Status")) (id, status) - }.toArray.toSeq) + }.toArray.toImmutableArraySeq) } metrics @@ -1373,10 +1375,10 @@ private[spark] object JsonProtocol extends JsonUtils { jsonOption(json.get("Full Stack Trace")).map(_.asText).orNull // Fallback on getting accumulator updates from TaskMetrics, which was logged in Spark 1.x val accumUpdates = jsonOption(json.get("Accumulator Updates")) - .map(_.extractElements.map(accumulableInfoFromJson).toArray.toSeq) + .map(_.extractElements.map(accumulableInfoFromJson).toArray.toImmutableArraySeq) .getOrElse(taskMetricsFromJson(json.get("Metrics")).accumulators().map(acc => { acc.toInfo(Some(acc.value), None) - }).toArray.toSeq) + }).toArray.toImmutableArraySeq) ExceptionFailure(className, description, stackTrace, fullStackTrace, None, accumUpdates) case `taskResultLost` => TaskResultLost case `taskKilled` => @@ -1385,7 +1387,7 @@ private[spark] object JsonProtocol extends JsonUtils { .map(_.asText).getOrElse("unknown reason") // The "Accumulator Updates" field was added in Spark 2.4.0: val accumUpdates = jsonOption(json.get("Accumulator Updates")) - .map(_.extractElements.map(accumulableInfoFromJson).toArray.toSeq) + .map(_.extractElements.map(accumulableInfoFromJson).toArray.toImmutableArraySeq) .getOrElse(Seq[AccumulableInfo]()) TaskKilled(killReason, accumUpdates) case `taskCommitDenied` => @@ -1443,7 +1445,7 @@ private[spark] object JsonProtocol extends JsonUtils { .map(RDDOperationScope.fromJson) val callsite = jsonOption(json.get("Callsite")).map(_.asText).getOrElse("") val parentIds = jsonOption(json.get("Parent IDs")) - .map { l => l.extractElements.map(_.extractInt).toArray.toSeq } + .map { l => l.extractElements.map(_.extractInt).toArray.toImmutableArraySeq } .getOrElse(Seq.empty) val storageLevel = storageLevelFromJson(json.get("Storage Level")) // The "Barrier" field was added in Spark 3.0.0: diff --git a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala index 3985d8a673db2..a5ac0f24f385f 100644 --- a/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala +++ b/mllib-local/src/main/scala/org/apache/spark/ml/linalg/Matrices.scala @@ -19,6 +19,7 @@ package org.apache.spark.ml.linalg import java.util.{Arrays, Random} +import scala.collection.immutable import scala.collection.mutable.{ArrayBuffer, ArrayBuilder => MArrayBuilder, HashSet => MHashSet} import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} @@ -1241,7 +1242,7 @@ object Matrices { cnt += 1 } startCol += nCols - data.toSeq + immutable.ArraySeq.unsafeWrapArray(data) case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => @@ -1310,7 +1311,7 @@ object Matrices { cnt += 1 } startRow += nRows - data.toSeq + immutable.ArraySeq.unsafeWrapArray(data) case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index b3420270bb73f..42106372a203d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -34,6 +34,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ /** * A stage in a pipeline, either an [[Estimator]] or a [[Transformer]]. @@ -246,7 +247,7 @@ object Pipeline extends MLReadable[Pipeline] { sc: SparkContext, path: String): Unit = instrumented { instr => val stageUids = stages.map(_.uid) - val jsonParams = List("stageUids" -> parse(compact(render(stageUids.toSeq)))) + val jsonParams = List("stageUids" -> parse(compact(render(stageUids.toImmutableArraySeq)))) DefaultParamsWriter.saveMetadata(instance, path, sc, paramMap = Some(jsonParams)) // Save stages diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index 992e91e5445fc..f2fe125db67f9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.ml.linalg.VectorUDT import org.apache.spark.sql.types.{Metadata, MetadataBuilder, StructField} +import org.apache.spark.util.ArrayImplicits._ /** * Attributes that describe a vector ML column. @@ -165,7 +166,7 @@ class AttributeGroup private ( case o: AttributeGroup => (name == o.name) && (numAttributes == o.numAttributes) && - (attributes.map(_.toSeq) == o.attributes.map(_.toSeq)) + (attributes.map(_.toImmutableArraySeq) == o.attributes.map(_.toImmutableArraySeq)) case _ => false } @@ -175,7 +176,7 @@ class AttributeGroup private ( var sum = 17 sum = 37 * sum + name.hashCode sum = 37 * sum + numAttributes.hashCode - sum = 37 * sum + attributes.map(_.toSeq).hashCode + sum = 37 * sum + attributes.map(_.toImmutableArraySeq).hashCode sum } diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala index f11cd865843d2..91bdee8120d18 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/attributes.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.attribute import scala.annotation.varargs import org.apache.spark.sql.types.{DoubleType, Metadata, MetadataBuilder, NumericType, StructField} +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.collection.Utils /** @@ -434,7 +435,7 @@ class NominalAttribute private[ml] ( (index == o.index) && (isOrdinal == o.isOrdinal) && (numValues == o.numValues) && - (values.map(_.toSeq) == o.values.map(_.toSeq)) + (values.map(_.toImmutableArraySeq) == o.values.map(_.toImmutableArraySeq)) case _ => false } @@ -446,7 +447,7 @@ class NominalAttribute private[ml] ( sum = 37 * sum + index.hashCode sum = 37 * sum + isOrdinal.hashCode sum = 37 * sum + numValues.hashCode - sum = 37 * sum + values.map(_.toSeq).hashCode + sum = 37 * sum + values.map(_.toImmutableArraySeq).hashCode sum } } @@ -485,7 +486,8 @@ class BinaryAttribute private[ml] ( extends Attribute { values.foreach { v => - require(v.length == 2, s"Number of values must be 2 for a binary attribute but got ${v.toSeq}.") + require(v.length == 2, + s"Number of values must be 2 for a binary attribute but got ${v.toImmutableArraySeq}.") } override def attrType: AttributeType = AttributeType.Binary @@ -534,7 +536,7 @@ class BinaryAttribute private[ml] ( case o: BinaryAttribute => (name == o.name) && (index == o.index) && - (values.map(_.toSeq) == o.values.map(_.toSeq)) + (values.map(_.toImmutableArraySeq) == o.values.map(_.toImmutableArraySeq)) case _ => false } @@ -544,7 +546,7 @@ class BinaryAttribute private[ml] ( var sum = 17 sum = 37 * sum + name.hashCode sum = 37 * sum + index.hashCode - sum = 37 * sum + values.map(_.toSeq).hashCode + sum = 37 * sum + values.map(_.toImmutableArraySeq).hashCode sum } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 23402599e543d..2dac53fe51690 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -592,7 +592,7 @@ abstract class LDAModel private[ml] ( def describeTopics(maxTermsPerTopic: Int): DataFrame = { val topics = getModel.describeTopics(maxTermsPerTopic).zipWithIndex.map { case ((termIndices, termWeights), topic) => - (topic, termIndices.toSeq, termWeights.toSeq) + (topic, termIndices.toImmutableArraySeq, termWeights.toImmutableArraySeq) } sparkSession.createDataFrame(topics.toImmutableArraySeq) .toDF("topic", "termIndices", "termWeights") diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala index ee87f49806ad9..a433f5a6e5607 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringMetrics.scala @@ -25,6 +25,7 @@ import org.apache.spark.ml.util.DatasetUtils._ import org.apache.spark.sql.{Column, DataFrame, Dataset} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.DoubleType +import org.apache.spark.util.ArrayImplicits._ /** @@ -49,7 +50,7 @@ class ClusteringMetrics private[spark](dataset: Dataset[_]) { */ @Since("3.1.0") def silhouette(): Double = { - val columns = dataset.columns.toSeq + val columns = dataset.columns.toImmutableArraySeq if (distanceMeasure.equalsIgnoreCase("squaredEuclidean")) { SquaredEuclideanSilhouette.computeSilhouetteScore( dataset, columns(0), columns(1), columns(2)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala index 2e09e74449572..3727eb17dcd0a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Binarizer.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ /** * Binarize a column of continuous features given a threshold. @@ -104,9 +105,11 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) val (inputColNames, outputColNames, tds) = if (isSet(inputCols)) { if (isSet(thresholds)) { - ($(inputCols).toSeq, $(outputCols).toSeq, $(thresholds).toSeq) + ($(inputCols).toImmutableArraySeq, + $(outputCols).toImmutableArraySeq, $(thresholds).toImmutableArraySeq) } else { - ($(inputCols).toSeq, $(outputCols).toSeq, Seq.fill($(inputCols).length)($(threshold))) + ($(inputCols).toImmutableArraySeq, + $(outputCols).toImmutableArraySeq, Seq.fill($(inputCols).length)($(threshold))) } } else { (Seq($(inputCol)), Seq($(outputCol)), Seq($(threshold))) @@ -185,7 +188,7 @@ final class Binarizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) } val (inputColNames, outputColNames) = if (isSet(inputCols)) { - ($(inputCols).toSeq, $(outputCols).toSeq) + ($(inputCols).toImmutableArraySeq, $(outputCols).toImmutableArraySeq) } else { (Seq($(inputCol)), Seq($(outputCol))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index 9aeddae78ed76..967d46ff7cd53 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructField, StructType} +import org.apache.spark.util.ArrayImplicits._ /** * `Bucketizer` maps a column of continuous features to a column of feature buckets. @@ -143,7 +144,7 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String val transformedSchema = transformSchema(dataset.schema) val (inputColumns, outputColumns) = if (isSet(inputCols)) { - ($(inputCols).toSeq, $(outputCols).toSeq) + ($(inputCols).toImmutableArraySeq, $(outputCols).toImmutableArraySeq) } else { (Seq($(inputCol)), Seq($(outputCol))) } @@ -158,7 +159,7 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String } val seqOfSplits = if (isSet(inputCols)) { - $(splitsArray).toSeq + $(splitsArray).toImmutableArraySeq } else { Seq($(splits)) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala index fc6c6157f850b..10149a65a9549 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ChiSqSelector.scala @@ -25,6 +25,7 @@ import org.apache.spark.ml.stat.ChiSquareTest import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ /** @@ -173,7 +174,7 @@ object ChiSqSelectorModel extends MLReadable[ChiSqSelectorModel] { override protected def saveImpl(path: String): Unit = { DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.selectedFeatures.toSeq) + val data = Data(instance.selectedFeatures.toImmutableArraySeq) val dataPath = new Path(path, "data").toString sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 4d60172e80076..177c90a767983 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -26,6 +26,7 @@ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.sql.Dataset import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ /** * Params for [[QuantileDiscretizer]]. @@ -179,7 +180,7 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui } val (inputColNames, outputColNames) = if (isSet(inputCols)) { - ($(inputCols).toSeq, $(outputCols).toSeq) + ($(inputCols).toImmutableArraySeq, $(outputCols).toImmutableArraySeq) } else { (Seq($(inputCol)), Seq($(outputCol))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 4250b50673e81..f52f56174ed23 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -210,7 +210,7 @@ class StringIndexer @Since("1.4.0") ( val (inputCols, _) = getInOutCols() val sortFunc = StringIndexer.getSortFunc(ascending = ascending) - val orgStrings = countByValue(dataset, inputCols).toSeq + val orgStrings = countByValue(dataset, inputCols).toImmutableArraySeq ThreadUtils.parmap(orgStrings, "sortingStringLabels", 8) { counts => counts.toSeq.sortWith(sortFunc).map(_._1).toArray }.toArray @@ -221,7 +221,7 @@ class StringIndexer @Since("1.4.0") ( val selectedCols = getSelectedCols(dataset, inputCols.toImmutableArraySeq).map(collect_set) val allLabels = dataset.select(selectedCols: _*) - .collect().toSeq.flatMap(_.toSeq) + .collect().toImmutableArraySeq.flatMap(_.toSeq) .asInstanceOf[scala.collection.Seq[scala.collection.Seq[String]]].toSeq ThreadUtils.parmap(allLabels, "sortingStringLabels", 8) { labels => val sorted = labels.filter(_ != null).sorted diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 8337e305e2b31..e7b3ff76a8d8c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -146,7 +146,7 @@ class RegexTokenizer @Since("1.4.0") (@Since("1.4.0") override val uid: String) // scalastyle:off caselocale val str = if ($(toLowercase)) originStr.toLowerCase() else originStr // scalastyle:on caselocale - val tokens = if ($(gaps)) re.split(str).toSeq else re.findAllIn(str).toSeq + val tokens = if ($(gaps)) re.split(str).toImmutableArraySeq else re.findAllIn(str).toSeq val minLength = $(minTokenLength) tokens.filter(_.length >= minLength) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala index 61424472dc03b..35e5b27183ad8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/UnivariateFeatureSelector.scala @@ -32,6 +32,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.ArrayImplicits._ /** @@ -349,7 +350,7 @@ object UnivariateFeatureSelectorModel extends MLReadable[UnivariateFeatureSelect override protected def saveImpl(path: String): Unit = { DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.selectedFeatures.toSeq) + val data = Data(instance.selectedFeatures.toImmutableArraySeq) val dataPath = new Path(path, "data").toString sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala index 3f4466c006188..82b49bd800670 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VarianceThresholdSelector.scala @@ -29,7 +29,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType - +import org.apache.spark.util.ArrayImplicits._ /** * Params for [[VarianceThresholdSelector]] and [[VarianceThresholdSelectorModel]]. @@ -188,7 +188,7 @@ object VarianceThresholdSelectorModel extends MLReadable[VarianceThresholdSelect override protected def saveImpl(path: String): Unit = { DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.selectedFeatures.toSeq) + val data = Data(instance.selectedFeatures.toImmutableArraySeq) val dataPath = new Path(path, "data").toString sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index fe54347b818a0..daad072f42bcc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -113,7 +113,7 @@ class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) case _: VectorUDT => val attributeGroup = AttributeGroup.fromStructField(field) if (attributeGroup.attributes.isDefined) { - attributeGroup.attributes.get.zipWithIndex.toSeq.map { case (attr, i) => + attributeGroup.attributes.get.zipWithIndex.toImmutableArraySeq.map { case (attr, i) => if (attr.name.isDefined) { // TODO: Define a rigorous naming scheme. attr.withName(c + "_" + attr.name.get) diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonMatrixConverter.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonMatrixConverter.scala index 8f03a29eb991a..c58306470e2f2 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonMatrixConverter.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonMatrixConverter.scala @@ -20,6 +20,8 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, parse => parseJson, render} +import org.apache.spark.util.ArrayImplicits._ + private[ml] object JsonMatrixConverter { /** Unique class name for identifying JSON object encoded by this class. */ @@ -61,9 +63,9 @@ private[ml] object JsonMatrixConverter { ("type" -> 0) ~ ("numRows" -> numRows) ~ ("numCols" -> numCols) ~ - ("colPtrs" -> colPtrs.toSeq) ~ - ("rowIndices" -> rowIndices.toSeq) ~ - ("values" -> values.toSeq) ~ + ("colPtrs" -> colPtrs.toImmutableArraySeq) ~ + ("rowIndices" -> rowIndices.toImmutableArraySeq) ~ + ("values" -> values.toImmutableArraySeq) ~ ("isTransposed" -> isTransposed) compact(render(jValue)) case DenseMatrix(numRows, numCols, values, isTransposed) => @@ -71,7 +73,7 @@ private[ml] object JsonMatrixConverter { ("type" -> 1) ~ ("numRows" -> numRows) ~ ("numCols" -> numCols) ~ - ("values" -> values.toSeq) ~ + ("values" -> values.toImmutableArraySeq) ~ ("isTransposed" -> isTransposed) compact(render(jValue)) case _ => diff --git a/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala b/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala index 1b949d75eeaa0..129a99c4e400c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/linalg/JsonVectorConverter.scala @@ -21,6 +21,8 @@ import org.json4s.DefaultFormats import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods.{compact, parse => parseJson, render} +import org.apache.spark.util.ArrayImplicits._ + private[ml] object JsonVectorConverter { /** @@ -51,11 +53,11 @@ private[ml] object JsonVectorConverter { case SparseVector(size, indices, values) => val jValue = ("type" -> 0) ~ ("size" -> size) ~ - ("indices" -> indices.toSeq) ~ - ("values" -> values.toSeq) + ("indices" -> indices.toImmutableArraySeq) ~ + ("values" -> values.toImmutableArraySeq) compact(render(jValue)) case DenseVector(values) => - val jValue = ("type" -> 1) ~ ("values" -> values.toSeq) + val jValue = ("type" -> 1) ~ ("values" -> values.toImmutableArraySeq) compact(render(jValue)) case _ => throw new IllegalArgumentException(s"Unknown vector type ${v.getClass}.") diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index ef1adea592492..d97f92355bdc6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -32,6 +32,7 @@ import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.{JsonMatrixConverter, JsonVectorConverter, Matrix, Vector} import org.apache.spark.ml.util.Identifiable +import org.apache.spark.util.ArrayImplicits._ /** * A param with self-contained documentation and optionally default value. Primitive-typed param @@ -524,7 +525,7 @@ class StringArrayParam(parent: Params, name: String, doc: String, isValid: Array override def jsonEncode(value: Array[String]): String = { import org.json4s.JsonDSL._ - compact(render(value.toSeq)) + compact(render(value.toImmutableArraySeq)) } override def jsonDecode(json: String): Array[String] = { @@ -548,7 +549,7 @@ class DoubleArrayParam(parent: Params, name: String, doc: String, isValid: Array override def jsonEncode(value: Array[Double]): String = { import org.json4s.JsonDSL._ - compact(render(value.toSeq.map(DoubleParam.jValueEncode))) + compact(render(value.toImmutableArraySeq.map(DoubleParam.jValueEncode))) } override def jsonDecode(json: String): Array[Double] = { @@ -580,7 +581,8 @@ class DoubleArrayArrayParam( override def jsonEncode(value: Array[Array[Double]]): String = { import org.json4s.JsonDSL._ - compact(render(value.toSeq.map(_.toSeq.map(DoubleParam.jValueEncode)))) + compact( + render(value.toImmutableArraySeq.map(_.toImmutableArraySeq.map(DoubleParam.jValueEncode)))) } override def jsonDecode(json: String): Array[Array[Double]] = { @@ -613,7 +615,7 @@ class IntArrayParam(parent: Params, name: String, doc: String, isValid: Array[In override def jsonEncode(value: Array[Int]): String = { import org.json4s.JsonDSL._ - compact(render(value.toSeq)) + compact(render(value.toImmutableArraySeq)) } override def jsonDecode(json: String): Array[Int] = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala index 594d9f315f508..1d9316bfc3847 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.feature.RFormula import org.apache.spark.ml.regression.{AFTSurvivalRegression, AFTSurvivalRegressionModel} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class AFTSurvivalRegressionWrapper private ( val pipeline: PipelineModel, @@ -126,7 +127,7 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala index 71712c1c5eec5..d4486f1b80a10 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/BisectingKMeansWrapper.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.clustering.{BisectingKMeans, BisectingKMeansModel} import org.apache.spark.ml.feature.RFormula import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class BisectingKMeansWrapper private ( val pipeline: PipelineModel, @@ -115,8 +116,8 @@ private[r] object BisectingKMeansWrapper extends MLReadable[BisectingKMeansWrapp val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) ~ - ("size" -> instance.size.toSeq) + ("features" -> instance.features.toImmutableArraySeq) ~ + ("size" -> instance.size.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala index a90cae5869b2a..992a0c18819fc 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeClassifierWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class DecisionTreeClassifierWrapper private ( val pipeline: PipelineModel, @@ -127,7 +128,7 @@ private[r] object DecisionTreeClassifierWrapper extends MLReadable[DecisionTreeC val rMetadata = ("class" -> instance.getClass.getName) ~ ("formula" -> instance.formula) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala index de712d67e6df5..db421b5a1875e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/DecisionTreeRegressorWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{DecisionTreeRegressionModel, DecisionTreeRegressor} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class DecisionTreeRegressorWrapper private ( val pipeline: PipelineModel, @@ -110,7 +111,7 @@ private[r] object DecisionTreeRegressorWrapper extends MLReadable[DecisionTreeRe val rMetadata = ("class" -> instance.getClass.getName) ~ ("formula" -> instance.formula) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala index a6c6ad6881a02..635af0563da0d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FMClassifierWrapper.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.feature.{IndexToString, RFormula} import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class FMClassifierWrapper private ( val pipeline: PipelineModel, @@ -147,8 +148,8 @@ private[r] object FMClassifierWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) ~ - ("labels" -> instance.labels.toSeq) + ("features" -> instance.features.toImmutableArraySeq) ~ + ("labels" -> instance.labels.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala index ae2cf68781b77..b036a1d102d97 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/FMRegressorWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.regression.{FMRegressionModel, FMRegressor} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class FMRegressorWrapper private ( val pipeline: PipelineModel, @@ -129,7 +130,7 @@ private[r] object FMRegressorWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala index ecaeec5a7791a..777191ef5e5c6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GBTClassifierWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class GBTClassifierWrapper private ( val pipeline: PipelineModel, @@ -134,7 +135,7 @@ private[r] object GBTClassifierWrapper extends MLReadable[GBTClassifierWrapper] val rMetadata = ("class" -> instance.getClass.getName) ~ ("formula" -> instance.formula) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala index b568d7859221f..6e5ca47fabae6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GBTRegressorWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{GBTRegressionModel, GBTRegressor} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class GBTRegressorWrapper private ( val pipeline: PipelineModel, @@ -118,7 +119,7 @@ private[r] object GBTRegressorWrapper extends MLReadable[GBTRegressorWrapper] { val rMetadata = ("class" -> instance.getClass.getName) ~ ("formula" -> instance.formula) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala index cc5f61eab0dd5..60cf0631f91de 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/GeneralizedLinearRegressionWrapper.scala @@ -30,6 +30,7 @@ import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.regression._ import org.apache.spark.ml.util._ import org.apache.spark.sql._ +import org.apache.spark.util.ArrayImplicits._ private[r] class GeneralizedLinearRegressionWrapper private ( val pipeline: PipelineModel, @@ -159,8 +160,8 @@ private[r] object GeneralizedLinearRegressionWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("rFeatures" -> instance.rFeatures.toSeq) ~ - ("rCoefficients" -> instance.rCoefficients.toSeq) ~ + ("rFeatures" -> instance.rFeatures.toImmutableArraySeq) ~ + ("rCoefficients" -> instance.rCoefficients.toImmutableArraySeq) ~ ("rDispersion" -> instance.rDispersion) ~ ("rNullDeviance" -> instance.rNullDeviance) ~ ("rDeviance" -> instance.rDeviance) ~ diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala index 73fd3c46a5d4c..d4a3adea460fa 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/IsotonicRegressionWrapper.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.feature.RFormula import org.apache.spark.ml.regression.{IsotonicRegression, IsotonicRegressionModel} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class IsotonicRegressionWrapper private ( val pipeline: PipelineModel, @@ -96,7 +97,7 @@ private[r] object IsotonicRegressionWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala index dc514ebbb5b71..78c9a15aac597 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/KMeansWrapper.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.clustering.{KMeans, KMeansModel} import org.apache.spark.ml.feature.RFormula import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class KMeansWrapper private ( val pipeline: PipelineModel, @@ -118,8 +119,8 @@ private[r] object KMeansWrapper extends MLReadable[KMeansWrapper] { val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) ~ - ("size" -> instance.size.toSeq) + ("features" -> instance.features.toImmutableArraySeq) ~ + ("size" -> instance.size.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala index 19cd4dff0acee..96b00fab7e344 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LinearRegressionWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.regression.{LinearRegression, LinearRegressionModel} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class LinearRegressionWrapper private ( val pipeline: PipelineModel, @@ -124,7 +125,7 @@ private[r] object LinearRegressionWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala index 7a22a71c3a819..3645af3e53115 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LinearSVCWrapper.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.feature.{IndexToString, RFormula} import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class LinearSVCWrapper private ( val pipeline: PipelineModel, @@ -133,8 +134,8 @@ private[r] object LinearSVCWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) ~ - ("labels" -> instance.labels.toSeq) + ("features" -> instance.features.toImmutableArraySeq) ~ + ("labels" -> instance.labels.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala index 18acf7d21656f..cac3d0609b209 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/LogisticRegressionWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.{Matrices, Vector, Vectors} import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class LogisticRegressionWrapper private ( val pipeline: PipelineModel, @@ -188,8 +189,8 @@ private[r] object LogisticRegressionWrapper val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("features" -> instance.features.toSeq) ~ - ("labels" -> instance.labels.toSeq) + ("features" -> instance.features.toImmutableArraySeq) ~ + ("labels" -> instance.labels.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala index fbf9f462ff5f6..d5e8e0ef4890a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/NaiveBayesWrapper.scala @@ -28,6 +28,7 @@ import org.apache.spark.ml.feature.{IndexToString, RFormula} import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class NaiveBayesWrapper private ( val pipeline: PipelineModel, @@ -98,8 +99,8 @@ private[r] object NaiveBayesWrapper extends MLReadable[NaiveBayesWrapper] { val pipelinePath = new Path(path, "pipeline").toString val rMetadata = ("class" -> instance.getClass.getName) ~ - ("labels" -> instance.labels.toSeq) ~ - ("features" -> instance.features.toSeq) + ("labels" -> instance.labels.toImmutableArraySeq) ~ + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala index 64c2d04ac7462..7c4175a6c5914 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestClassifierWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.r.RWrapperUtils._ import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class RandomForestClassifierWrapper private ( val pipeline: PipelineModel, @@ -137,7 +138,7 @@ private[r] object RandomForestClassifierWrapper extends MLReadable[RandomForestC val rMetadata = ("class" -> instance.getClass.getName) ~ ("formula" -> instance.formula) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala index 60ecc24ae2745..911571cac77de 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/RandomForestRegressorWrapper.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.regression.{RandomForestRegressionModel, RandomForestRegressor} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.util.ArrayImplicits._ private[r] class RandomForestRegressorWrapper private ( val pipeline: PipelineModel, @@ -120,7 +121,7 @@ private[r] object RandomForestRegressorWrapper extends MLReadable[RandomForestRe val rMetadata = ("class" -> instance.getClass.getName) ~ ("formula" -> instance.formula) ~ - ("features" -> instance.features.toSeq) + ("features" -> instance.features.toImmutableArraySeq) val rMetadataJson: String = compact(render(rMetadata)) sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index f186b2442cdc2..33b7963788fa5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -37,6 +37,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.util.MLUtils import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.ThreadUtils /** @@ -191,7 +192,7 @@ class CrossValidator @Since("1.2.0") (@Since("1.4.0") override val uid: String) foldMetrics }.transpose.map(_.sum / $(numFolds)) // Calculate average metric over all splits - instr.logInfo(s"Average cross-validation metrics: ${metrics.toSeq}") + instr.logInfo(s"Average cross-validation metrics: ${metrics.toImmutableArraySeq}") val (bestMetric, bestIndex) = if (eval.isLargerBetter) metrics.zipWithIndex.maxBy(_._1) else metrics.zipWithIndex.minBy(_._1) @@ -398,7 +399,7 @@ object CrossValidatorModel extends MLReadable[CrossValidatorModel] { val persistSubModels = persistSubModelsParam.toBoolean import org.json4s.JsonDSL._ - val extraMetadata = ("avgMetrics" -> instance.avgMetrics.toSeq) ~ + val extraMetadata = ("avgMetrics" -> instance.avgMetrics.toImmutableArraySeq) ~ ("persistSubModels" -> persistSubModels) ValidatorParams.saveImpl(path, instance, sc, Some(extraMetadata)) val bestModelPath = new Path(path, "bestModel").toString diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala index 32a17c11c56a8..58487b6ccbb8e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/TrainValidationSplit.scala @@ -37,6 +37,7 @@ import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.ThreadUtils /** @@ -167,7 +168,7 @@ class TrainValidationSplit @Since("1.5.0") (@Since("1.5.0") override val uid: St trainingDataset.unpersist() validationDataset.unpersist() - instr.logInfo(s"Train validation split metrics: ${metrics.toSeq}") + instr.logInfo(s"Train validation split metrics: ${metrics.toImmutableArraySeq}") val (bestMetric, bestIndex) = if (eval.isLargerBetter) metrics.zipWithIndex.maxBy(_._1) else metrics.zipWithIndex.minBy(_._1) @@ -362,7 +363,7 @@ object TrainValidationSplitModel extends MLReadable[TrainValidationSplitModel] { val persistSubModels = persistSubModelsParam.toBoolean import org.json4s.JsonDSL._ - val extraMetadata = ("validationMetrics" -> instance.validationMetrics.toSeq) ~ + val extraMetadata = ("validationMetrics" -> instance.validationMetrics.toImmutableArraySeq) ~ ("persistSubModels" -> persistSubModels) ValidatorParams.saveImpl(path, instance, sc, Some(extraMetadata)) val bestModelPath = new Path(path, "bestModel").toString diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala index f6308213525d6..77ab7d45eda43 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala @@ -29,6 +29,7 @@ import org.apache.spark.ml.param.shared.HasSeed import org.apache.spark.ml.util._ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ /** * Common params for [[TrainValidationSplitParams]] and [[CrossValidatorParams]]. @@ -147,7 +148,7 @@ private[ml] object ValidatorParams { "isJson" -> compact(render(JBool(true)))) } } - }.toSeq + }.toImmutableArraySeq )) val params = instance.extractParamMap().toSeq diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala index 2215c2b071584..a243ab8d27c9c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala @@ -32,6 +32,7 @@ import org.apache.spark.ml.{MLEvents, PipelineStage} import org.apache.spark.ml.param.{Param, Params} import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** @@ -143,15 +144,15 @@ private[spark] class Instrumentation private () extends Logging with MLEvents { } def logNamedValue(name: String, value: Array[String]): Unit = { - logInfo(compact(render(name -> compact(render(value.toSeq))))) + logInfo(compact(render(name -> compact(render(value.toImmutableArraySeq))))) } def logNamedValue(name: String, value: Array[Long]): Unit = { - logInfo(compact(render(name -> compact(render(value.toSeq))))) + logInfo(compact(render(name -> compact(render(value.toImmutableArraySeq))))) } def logNamedValue(name: String, value: Array[Double]): Unit = { - logInfo(compact(render(name -> compact(render(value.toSeq))))) + logInfo(compact(render(name -> compact(render(value.toImmutableArraySeq))))) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index c895c64fc2310..7a925c204d1be 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -55,6 +55,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.types.LongType import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** @@ -1316,7 +1317,7 @@ private[spark] abstract class SerDeBase { obj match { // Pickler in Python side cannot deserialize Scala Array normally. See SPARK-12834. case array: Array[_] => new Pickler(/* useMemo = */ true, - /* valueCompare = */ false).dumps(array.toSeq.asJava) + /* valueCompare = */ false).dumps(array.toImmutableArraySeq.asJava) case _ => new Pickler(/* useMemo = */ true, /* valueCompare = */ false).dumps(obj) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index aa8b6a00a427f..7891159cf44db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -33,6 +33,7 @@ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.util.{BoundedPriorityQueue, Utils} +import org.apache.spark.util.ArrayImplicits._ /** * Latent Dirichlet Allocation (LDA) model. @@ -456,7 +457,7 @@ object LocalLDAModel extends Loader[LocalLDAModel] { val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> k) ~ ("vocabSize" -> topicsMatrix.numRows) ~ - ("docConcentration" -> docConcentration.toArray.toSeq) ~ + ("docConcentration" -> docConcentration.toArray.toImmutableArraySeq) ~ ("topicConcentration" -> topicConcentration) ~ ("gammaShape" -> gammaShape))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) @@ -864,9 +865,9 @@ object DistributedLDAModel extends Loader[DistributedLDAModel] { val metadata = compact(render (("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> k) ~ ("vocabSize" -> vocabSize) ~ - ("docConcentration" -> docConcentration.toArray.toSeq) ~ + ("docConcentration" -> docConcentration.toArray.toImmutableArraySeq) ~ ("topicConcentration" -> topicConcentration) ~ - ("iterationTimes" -> iterationTimes.toSeq) ~ + ("iterationTimes" -> iterationTimes.toImmutableArraySeq) ~ ("gammaShape" -> gammaShape))) sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 79f482347289a..4c3ab2ddc42f6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -26,6 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.fpm.AssociationRules.Rule import org.apache.spark.mllib.fpm.FPGrowth.FreqItemset import org.apache.spark.rdd.RDD +import org.apache.spark.util.ArrayImplicits._ /** * Generates association rules from a `RDD[FreqItemset[Item]]`. This method only generates @@ -80,14 +81,14 @@ class AssociationRules private[fpm] ( items.flatMap { item => items.partition(_ == item) match { case (consequent, antecedent) if !antecedent.isEmpty => - Some((antecedent.toSeq, (consequent.toSeq, itemset.freq))) + Some((antecedent.toImmutableArraySeq, (consequent.toImmutableArraySeq, itemset.freq))) case _ => None } } } // Join to get (X, ((Y, freq(X union Y)), freq(X))), generate rules, and filter by confidence - candidates.join(freqItemsets.map(x => (x.items.toSeq, x.freq))) + candidates.join(freqItemsets.map(x => (x.items.toImmutableArraySeq, x.freq))) .map { case (antecedent, ((consequent, freqUnion), freqAntecedent)) => new Rule(antecedent.toArray, consequent.toArray, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index c74ad6b5c1aed..856baa65b0d75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.collection.Utils /** @@ -120,7 +121,7 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { StructField("freq", LongType)) val schema = StructType(fields) val rowDataRDD = model.freqItemsets.map { x => - Row(x.items.toSeq, x.freq) + Row(x.items.toImmutableArraySeq, x.freq) } spark.createDataFrame(rowDataRDD, schema).write.parquet(Loader.dataPath(path)) } @@ -247,7 +248,8 @@ class FPGrowth private[spark] ( data.flatMap { t => val uniq = t.toSet if (t.length != uniq.size) { - throw new SparkException(s"Items in a transaction must be unique but got ${t.toSeq}.") + throw new SparkException( + s"Items in a transaction must be unique but got ${t.toImmutableArraySeq}.") } t }.map(v => (v, 1L)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index c957e37b9a561..fc0c8d42579a9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods +import org.apache.spark.util.ArrayImplicits._ /** * Trait for a local matrix. @@ -1140,7 +1141,7 @@ object Matrices { cnt += 1 } startCol += nCols - data.toSeq + data.toImmutableArraySeq case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => @@ -1209,7 +1210,7 @@ object Matrices { cnt += 1 } startRow += nRows - data.toSeq + data.toImmutableArraySeq case dnMat: DenseMatrix => val data = new ArrayBuffer[(Int, Int, Double)]() dnMat.foreachActive { (i, j, v) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index fbdb5843eb99d..8626cffb2bc6d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -759,7 +759,7 @@ class DenseVector @Since("1.0.0") ( @Since("1.6.0") override def toJson: String = { - val jValue = ("type" -> 1) ~ ("values" -> values.toSeq) + val jValue = ("type" -> 1) ~ ("values" -> values.toImmutableArraySeq) compact(render(jValue)) } @@ -972,8 +972,8 @@ class SparseVector @Since("1.0.0") ( override def toJson: String = { val jValue = ("type" -> 0) ~ ("size" -> size) ~ - ("indices" -> indices.toSeq) ~ - ("values" -> values.toSeq) + ("indices" -> indices.toImmutableArraySeq) ~ + ("values" -> values.toImmutableArraySeq) compact(render(jValue)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index ba115c278eb44..a21baf1b1fdce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -34,6 +34,7 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession +import org.apache.spark.util.ArrayImplicits._ /** * Regression model for isotonic regression. @@ -190,7 +191,7 @@ object IsotonicRegressionModel extends Loader[IsotonicRegressionModel] { sc.parallelize(Seq(metadata), 1).saveAsTextFile(metadataPath(path)) spark.createDataFrame( - boundaries.toSeq.zip(predictions).map { case (b, p) => Data(b, p) } + boundaries.toImmutableArraySeq.zip(predictions).map { case (b, p) => Data(b, p) } ).write.parquet(dataPath(path)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index aaf751c0a95cd..eec7cb5b90ea3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -33,6 +33,7 @@ import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SparkSession} +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** @@ -260,7 +261,7 @@ object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { (treeId, constructTree(data)) }.sortBy(_._1) val numTrees = trees.length - val treeIndices = trees.map(_._1).toSeq + val treeIndices = trees.map(_._1).toImmutableArraySeq assert(treeIndices == (0 until numTrees), s"Tree indices must start from 0 and increment by 1, but we found $treeIndices.") trees.map(_._2) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala index 409be81598d42..e266d0f904e46 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala @@ -27,6 +27,7 @@ import io.fabric8.kubernetes.api.model._ import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.util.ArrayImplicits._ /** * Mounts the Hadoop configuration - either a pre-defined config map, or a local configuration @@ -47,7 +48,7 @@ private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf) private lazy val confFiles: Seq[File] = { val dir = new File(confDir.get) if (dir.isDirectory) { - dir.listFiles.filter(_.isFile).toSeq + dir.listFiles.filter(_.isFile).toImmutableArraySeq } else { Nil } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala index 679ee2991eb9e..b52ad732dfba4 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala @@ -52,7 +52,7 @@ private[spark] class LocalDirsFeatureStep( .getOrElse(defaultLocalDir) .split(",") randomize(resolvedLocalDirs) - localDirs = resolvedLocalDirs.toSeq + localDirs = resolvedLocalDirs.toImmutableArraySeq localDirVolumes = resolvedLocalDirs .zipWithIndex .map { case (_, index) => diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala index a699801dca199..39f41474d0057 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientUtils.scala @@ -32,6 +32,7 @@ import org.apache.spark.deploy.k8s.{Config, Constants, KubernetesUtils} import org.apache.spark.deploy.k8s.Config.{KUBERNETES_DNS_SUBDOMAIN_NAME_MAX_LENGTH, KUBERNETES_NAMESPACE} import org.apache.spark.deploy.k8s.Constants.ENV_SPARK_CONF_DIR import org.apache.spark.internal.Logging +import org.apache.spark.util.ArrayImplicits._ private[spark] object KubernetesClientUtils extends Logging { @@ -171,7 +172,7 @@ private[spark] object KubernetesClientUtils extends Logging { val confFiles: Seq[File] = { val dir = new File(confDir) if (dir.isDirectory) { - dir.listFiles.filter(x => fileFilter(x)).toSeq + dir.listFiles.filter(x => fileFilter(x)).toImmutableArraySeq } else { Nil } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala index a858db374dfb4..376218df57702 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/shuffle/KubernetesLocalDiskShuffleExecutorComponents.scala @@ -33,6 +33,7 @@ import org.apache.spark.shuffle.ShuffleChecksumUtils.{compareChecksums, getCheck import org.apache.spark.shuffle.api.{ShuffleExecutorComponents, ShuffleMapOutputWriter, SingleSpillShuffleMapOutputWriter} import org.apache.spark.shuffle.sort.io.LocalDiskShuffleExecutorComponents import org.apache.spark.storage.{BlockId, BlockManager, ShuffleDataBlockId, StorageLevel, UnrecognizedBlockId} +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils class KubernetesLocalDiskShuffleExecutorComponents(sparkConf: SparkConf) @@ -88,7 +89,7 @@ object KubernetesLocalDiskShuffleExecutorComponents extends Logging { .flatMap(_.listFiles).filter(_.isDirectory) // blockmgr-xxx .flatMap(_.listFiles).filter(_.isDirectory) // 00 .flatMap(_.listFiles) - if (files != null) files.toSeq else Seq.empty + if (files != null) files.toImmutableArraySeq else Seq.empty } .partition(_.getName.contains(".checksum")) val (indexFiles, dataFiles) = files.partition(_.getName.endsWith(".index")) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 8153ed5bf81e1..22037a1111d58 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -62,6 +62,7 @@ import org.apache.spark.launcher.{JavaModuleOptions, LauncherBackend, SparkAppHa import org.apache.spark.resource.ResourceProfile import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.{CallerContext, Utils, YarnContainerInfoHelper} +import org.apache.spark.util.ArrayImplicits._ private[spark] class Client( val args: ClientArguments, @@ -1379,7 +1380,7 @@ private[spark] class Client( private def findPySparkArchives(): Seq[String] = { sys.env.get("PYSPARK_ARCHIVES_PATH") - .map(_.split(",").toSeq) + .map(_.split(",").toImmutableArraySeq) .getOrElse { val pyLibPath = Seq(sys.env("SPARK_HOME"), "python", "lib").mkString(File.separator) val pyArchivesFile = new File(pyLibPath, "pyspark.zip") @@ -1458,21 +1459,21 @@ private[spark] object Client extends Logging { private def getYarnAppClasspath(conf: Configuration): Seq[String] = Option(conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) match { - case Some(s) => s.toSeq + case Some(s) => s.toImmutableArraySeq case None => getDefaultYarnApplicationClasspath } private def getMRAppClasspath(conf: Configuration): Seq[String] = Option(conf.getStrings("mapreduce.application.classpath")) match { - case Some(s) => s.toSeq + case Some(s) => s.toImmutableArraySeq case None => getDefaultMRApplicationClasspath } private[yarn] def getDefaultYarnApplicationClasspath: Seq[String] = - YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH.toSeq + YarnConfiguration.DEFAULT_YARN_APPLICATION_CLASSPATH.toImmutableArraySeq private[yarn] def getDefaultMRApplicationClasspath: Seq[String] = - StringUtils.getStrings(MRJobConfig.DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH).toSeq + StringUtils.getStrings(MRJobConfig.DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH).toImmutableArraySeq /** * Populate the classpath entry in the given environment map. diff --git a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala index ae6a7a1e31eca..a1735aa250ebd 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/Row.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.errors.DataTypeErrors import org.apache.spark.sql.internal.SqlApiConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.util.ArrayImplicits._ /** * @since 1.3.0 @@ -485,7 +486,7 @@ trait Row extends Serializable { values.update(i, get(i)) i += 1 } - values.toSeq + values.toImmutableArraySeq } /** Displays all elements of this sequence in a string (without a separator). */ diff --git a/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala index dc5e49fc35b5c..a04b1ec66503f 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -24,6 +24,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.Stable import org.apache.spark.sql.errors.DataTypeErrors +import org.apache.spark.util.ArrayImplicits._ /** @@ -213,7 +214,7 @@ object Metadata { map.view.mapValues(hash).toMap.## case arr: Array[_] => // Seq.empty[T] has the same hashCode regardless of T. - arr.toSeq.map(hash).## + arr.toImmutableArraySeq.map(hash).## case x: Long => x.## case x: Double => diff --git a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index eb2ad180e3670..99c46c01785a2 100644 --- a/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala +++ b/sql/api/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -110,7 +110,7 @@ private[sql] object ArrowUtils { new Field(name, fieldType, fields.map { field => toArrowField(field.name, field.dataType, field.nullable, timeZoneId, largeVarTypes) - }.toSeq.asJava) + }.toImmutableArraySeq.asJava) case MapType(keyType, valueType, valueContainsNull) => val mapType = new FieldType(nullable, new ArrowType.Map(false), null) // Note: Map Type struct can not be null, Struct Type key field can not be null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala index ff67b6fccfae9..6e7d8a058ae1a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/StructFilters.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.StructFilters._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.sources import org.apache.spark.sql.types.{BooleanType, StructType} +import org.apache.spark.util.ArrayImplicits._ /** * The class provides API for applying pushed down filters to partially or @@ -131,7 +132,7 @@ object StructFilters { case sources.IsNotNull(attribute) => toRef(attribute).map(IsNotNull) case sources.In(attribute, values) => - val literals = values.toSeq.flatMap(toLiteral) + val literals = values.toImmutableArraySeq.flatMap(toLiteral) if (literals.length == values.length) { toRef(attribute).map(In(_, literals)) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4f3d9158e828b..7e687d93091da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1274,7 +1274,9 @@ class Analyzer(override val catalogManager: CatalogManager) extends RuleExecutor resolveTempView(u.multipartIdentifier, u.isStreaming, finalTimeTravelSpec.isDefined).orElse { expandIdentifier(u.multipartIdentifier) match { case CatalogAndIdentifier(catalog, ident) => - val key = ((catalog.name +: ident.namespace :+ ident.name).toSeq, finalTimeTravelSpec) + val key = + ((catalog.name +: ident.namespace :+ ident.name).toImmutableArraySeq, + finalTimeTravelSpec) AnalysisContext.get.relationCache.get(key).map(_.transform { case multi: MultiInstanceRelation => val newRelation = multi.newInstance() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index e72ce91b063cf..c8b0e8be4c074 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -124,7 +124,8 @@ object FunctionRegistryBase { runtimeClass.getConstructors } // See if we can find a constructor that accepts Seq[Expression] - val varargCtor = constructors.find(_.getParameterTypes.toSeq == Seq(classOf[Seq[_]])) + val varargCtor = + constructors.find(_.getParameterTypes.toImmutableArraySeq == Seq(classOf[Seq[_]])) val builder = (expressions: Seq[Expression]) => { if (varargCtor.isDefined) { // If there is an apply method that accepts Seq[Expression], use that one. @@ -138,7 +139,7 @@ object FunctionRegistryBase { } else { // Otherwise, find a constructor method that matches the number of arguments, and use that. val params = Seq.fill(expressions.size)(classOf[Expression]) - val f = constructors.find(_.getParameterTypes.toSeq == params).getOrElse { + val f = constructors.find(_.getParameterTypes.toImmutableArraySeq == params).getOrElse { val validParametersCount = constructors .filter(_.getParameterTypes.forall(_ == classOf[Expression])) .map(_.getParameterCount).distinct.sorted diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 88ba156a9cfdf..eff29a78dadd3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.util.TypeUtils.toSQLId import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors} import org.apache.spark.sql.types.{DataType, Metadata, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits._ /** * Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully @@ -562,7 +563,8 @@ case class UnresolvedStarExcept(target: Option[Seq[String]], excepts: Seq[Seq[St col.toAttribute -> nestedExcept.tail }.get } - Alias(CreateStruct(filterColumns(extractedFields.toSeq, newExcepts)), col.name)() + Alias(CreateStruct( + filterColumns(extractedFields.toImmutableArraySeq, newExcepts)), col.name)() // if there are multiple nestedExcepts but one is empty we must have overlapping except // columns. throw an error. case (col, Some(nestedExcepts)) if nestedExcepts.size > 1 => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala index e9a02a243aa5f..b8d52311d9b5b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala @@ -45,6 +45,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.{CaseInsensitiveStringMap, PartitioningUtils} +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils object SessionCatalog { @@ -908,7 +909,7 @@ class SessionCatalog( val viewName = metadata.identifier.unquotedString val viewText = metadata.viewText.get val userSpecifiedColumns = - if (metadata.schema.fieldNames.toSeq == metadata.viewQueryColumnNames) { + if (metadata.schema.fieldNames.toImmutableArraySeq == metadata.viewQueryColumnNames) { " " } else { s" (${metadata.schema.fieldNames.mkString(", ")}) " @@ -947,7 +948,7 @@ class SessionCatalog( val viewColumnNames = if (metadata.viewQueryColumnNames.isEmpty) { // For view created before Spark 2.2.0, the view text is already fully qualified, the plan // output is the same with the view output. - metadata.schema.fieldNames.toSeq + metadata.schema.fieldNames.toImmutableArraySeq } else { assert(metadata.viewQueryColumnNames.length == metadata.schema.length) metadata.viewQueryColumnNames diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala index b34ae6a80f576..2cee8303dc57b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ApplyFunctionExpression.scala @@ -31,7 +31,7 @@ case class ApplyFunctionExpression( override def nullable: Boolean = function.isResultNullable override def name: String = function.name() override def dataType: DataType = function.resultType() - override def inputTypes: Seq[AbstractDataType] = function.inputTypes().toSeq + override def inputTypes: Seq[AbstractDataType] = function.inputTypes().toImmutableArraySeq override lazy val deterministic: Boolean = function.isDeterministic && children.forall(_.deterministic) override def foldable: Boolean = deterministic && children.forall(_.foldable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala index 6fb6333df3b1a..60e457a776bbd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/V2ExpressionUtils.scala @@ -45,7 +45,7 @@ object V2ExpressionUtils extends SQLConfHelper with Logging { case Some(namedExpr) => namedExpr.asInstanceOf[T] case None => - val name = ref.fieldNames.toSeq.quoted + val name = ref.fieldNames.toImmutableArraySeq.quoted val outputString = plan.output.map(_.name).mkString(",") throw QueryCompilationErrors.cannotResolveAttributeError(name, outputString) } @@ -153,7 +153,7 @@ object V2ExpressionUtils extends SQLConfHelper with Logging { def resolveScalarFunction( scalarFunc: ScalarFunction[_], arguments: Seq[Expression]): Expression = { - val declaredInputTypes = scalarFunc.inputTypes().toSeq + val declaredInputTypes = scalarFunc.inputTypes().toImmutableArraySeq val argClasses = declaredInputTypes.map(EncoderUtils.dataTypeJavaClass) findMethod(scalarFunc, MAGIC_METHOD_NAME, argClasses) match { case Some(m) if Modifier.isStatic(m.getModifiers) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala index f3ef8d9088c40..bb94421bc7d40 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/V2Aggregator.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Expression, ImplicitCastInputTypes, UnsafeProjection} import org.apache.spark.sql.connector.catalog.functions.{AggregateFunction => V2AggregateFunction} import org.apache.spark.sql.types.{AbstractDataType, DataType} +import org.apache.spark.util.ArrayImplicits._ case class V2Aggregator[BUF <: java.io.Serializable, OUT]( aggrFunc: V2AggregateFunction[BUF, OUT], @@ -35,7 +36,7 @@ case class V2Aggregator[BUF <: java.io.Serializable, OUT]( override def nullable: Boolean = aggrFunc.isResultNullable override def dataType: DataType = aggrFunc.resultType() - override def inputTypes: Seq[AbstractDataType] = aggrFunc.inputTypes().toSeq + override def inputTypes: Seq[AbstractDataType] = aggrFunc.inputTypes().toImmutableArraySeq override def createAggregationBuffer(): BUF = aggrFunc.newAggregationState() override def update(buffer: BUF, input: InternalRow): BUF = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index 9ee2f2bb41417..ad79fc1047043 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.trees.TernaryLike import org.apache.spark.sql.catalyst.trees.TreePattern.{CASE_WHEN, IF, TreePattern} import org.apache.spark.sql.catalyst.util.TypeUtils.{toSQLExpr, toSQLId, toSQLType} import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ // scalastyle:off line.size.limit @ExpressionDescription( @@ -418,7 +419,7 @@ object CaseKeyWhen { val cases = branches.grouped(2).flatMap { case Seq(cond, value) => Some((EqualTo(key, cond), value)) case Seq(value) => None - }.toArray.toSeq // force materialization to make the seq serializable + }.toArray.toImmutableArraySeq // force materialization to make the seq serializable val elseValue = if (branches.size % 2 != 0) Some(branches.last) else None CaseWhen(cases, elseValue) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala index a5812f376c5ca..eb3568e43f70f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala @@ -903,13 +903,13 @@ case class MapObjects private( case ObjectType(cls) if classOf[scala.collection.Seq[_]].isAssignableFrom(cls) => _.asInstanceOf[scala.collection.Seq[_]].toSeq case ObjectType(cls) if cls.isArray => - _.asInstanceOf[Array[_]].toSeq + _.asInstanceOf[Array[_]].toImmutableArraySeq case ObjectType(cls) if classOf[java.util.List[_]].isAssignableFrom(cls) => _.asInstanceOf[java.util.List[_]].asScala.toSeq case ObjectType(cls) if cls == classOf[Object] => (inputCollection) => { if (inputCollection.getClass.isArray) { - inputCollection.asInstanceOf[Array[_]].toSeq + inputCollection.asInstanceOf[Array[_]].toImmutableArraySeq } else { inputCollection.asInstanceOf[scala.collection.Seq[_]] } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala index 7b8de7d84cf05..cfc9e5520e53b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JsonInferSchema.scala @@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.util.LegacyDateFormats.FAST_DATE_FORMAT import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.internal.{LegacyBehaviorPolicy, SQLConf} import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils class JsonInferSchema(options: JSONOptions) extends Serializable with Logging { @@ -367,9 +368,9 @@ object JsonInferSchema { // Therefore, we can take advantage of the fact that we're merging sorted lists and skip // building a hash map or performing additional sorting. assert(isSorted(fields1), - s"${StructType.simpleString}'s fields were not sorted: ${fields1.toSeq}") + s"${StructType.simpleString}'s fields were not sorted: ${fields1.toImmutableArraySeq}") assert(isSorted(fields2), - s"${StructType.simpleString}'s fields were not sorted: ${fields2.toSeq}") + s"${StructType.simpleString}'s fields were not sorted: ${fields2.toImmutableArraySeq}") val newFields = new java.util.ArrayList[StructField]() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala index 55f7df2237c84..ff638a5abbae3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Window} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern._ import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ /** * We need to take care of special floating numbers (NaN and -0.0) in several places: @@ -133,7 +134,7 @@ object NormalizeFloatingNumbers extends Rule[LogicalPlan] { val fields = expr.dataType.asInstanceOf[StructType].fieldNames.zipWithIndex.map { case (name, i) => Seq(Literal(name), normalize(GetStructField(expr, i))) } - val struct = CreateNamedStruct(fields.flatten.toSeq) + val struct = CreateNamedStruct(fields.flatten.toImmutableArraySeq) KnownFloatingPointNormalized(If(IsNull(expr), Literal(null, struct.dataType), struct)) case _ if expr.dataType.isInstanceOf[ArrayType] => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala index d13bfab6d7028..7e04af190e4aa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/FunctionBuilderBase.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Expression, NamedArgumentExpression} import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.util.ArrayImplicits._ /** * This is a base trait that is used for implementing builder classes that can be used to construct @@ -150,7 +151,7 @@ object NamedParametersSupport { // Check the argument list size against the provided parameter list length. if (parameters.size < args.length) { val validParameterSizes = - Array.range(parameters.count(_.default.isEmpty), parameters.size + 1).toSeq + Array.range(parameters.count(_.default.isEmpty), parameters.size + 1).toImmutableArraySeq throw QueryCompilationErrors.wrongNumArgsError( functionName, validParameterSizes, args.length) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index df70b2b7d3084..c083ee89db6f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ /** * Functionality for working with missing data in `DataFrame`s. @@ -59,7 +60,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def drop(cols: Array[String]): DataFrame = drop(cols.toSeq) + def drop(cols: Array[String]): DataFrame = drop(cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that drops rows containing any null or NaN values @@ -78,7 +79,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def drop(how: String, cols: Array[String]): DataFrame = drop(how, cols.toSeq) + def drop(how: String, cols: Array[String]): DataFrame = drop(how, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that drops rows containing null or NaN values @@ -107,7 +108,8 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def drop(minNonNulls: Int, cols: Array[String]): DataFrame = drop(minNonNulls, cols.toSeq) + def drop(minNonNulls: Int, cols: Array[String]): DataFrame = + drop(minNonNulls, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that drops rows containing less than @@ -145,7 +147,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 2.2.0 */ - def fill(value: Long, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: Long, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * Returns a new `DataFrame` that replaces null or NaN values in specified numeric columns. @@ -153,7 +155,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def fill(value: Double, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: Double, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that replaces null or NaN values in specified @@ -178,7 +180,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 1.3.1 */ - def fill(value: String, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: String, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** * (Scala-specific) Returns a new `DataFrame` that replaces null values in @@ -209,7 +211,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * * @since 2.3.0 */ - def fill(value: Boolean, cols: Array[String]): DataFrame = fill(value, cols.toSeq) + def fill(value: Boolean, cols: Array[String]): DataFrame = fill(value, cols.toImmutableArraySeq) /** @@ -301,7 +303,7 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { * @since 1.3.1 */ def replace[T](cols: Array[String], replacement: java.util.Map[T, T]): DataFrame = { - replace(cols.toSeq, replacement.asScala.toMap) + replace(cols.toImmutableArraySeq, replacement.asScala.toMap) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 9544042465a00..5a7ac49bd4e76 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -40,6 +40,7 @@ import org.apache.spark.sql.internal.SQLConf.PartitionOverwriteMode import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits._ /** * Interface used to write a [[Dataset]] to external storage systems (e.g. file systems, @@ -337,7 +338,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { external = false) runCommand(df.sparkSession) { CreateTableAsSelect( - UnresolvedIdentifier(catalog.name +: ident.namespace.toSeq :+ ident.name), + UnresolvedIdentifier( + catalog.name +: ident.namespace.toImmutableArraySeq :+ ident.name), partitioningAsV2, df.queryExecution.analyzed, tableSpec, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index 0c33f2c87fec2..32d456c691740 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -283,7 +283,8 @@ class Dataset[T] private[sql]( // For array values, replace Seq and Array with square brackets // For cells that are beyond `truncate` characters, replace it with the // first `truncate-3` and "..." - (schema.fieldNames.map(SchemaUtils.escapeMetaCharacters).toSeq +: data.map { row => + (schema.fieldNames + .map(SchemaUtils.escapeMetaCharacters).toImmutableArraySeq +: data.map { row => row.toSeq.map { cell => assert(cell != null, "ToPrettyString is not nullable and should not return null value") // Escapes meta-characters not to break the `showString` format @@ -993,7 +994,7 @@ class Dataset[T] private[sql]( * @since 3.4.0 */ def join(right: Dataset[_], usingColumns: Array[String]): DataFrame = { - join(right, usingColumns.toSeq) + join(right, usingColumns.toImmutableArraySeq) } /** @@ -1062,7 +1063,7 @@ class Dataset[T] private[sql]( * @since 3.4.0 */ def join(right: Dataset[_], usingColumns: Array[String], joinType: String): DataFrame = { - join(right, usingColumns.toSeq, joinType) + join(right, usingColumns.toImmutableArraySeq, joinType) } /** @@ -3144,7 +3145,8 @@ class Dataset[T] private[sql]( * @group typedrel * @since 2.0.0 */ - def dropDuplicates(colNames: Array[String]): Dataset[T] = dropDuplicates(colNames.toSeq) + def dropDuplicates(colNames: Array[String]): Dataset[T] = + dropDuplicates(colNames.toImmutableArraySeq) /** * Returns a new [[Dataset]] with duplicate rows removed, considering only @@ -3231,7 +3233,7 @@ class Dataset[T] private[sql]( * @since 3.5.0 */ def dropDuplicatesWithinWatermark(colNames: Array[String]): Dataset[T] = { - dropDuplicatesWithinWatermark(colNames.toSeq) + dropDuplicatesWithinWatermark(colNames.toImmutableArraySeq) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index bf1b2814270fd..37223a33a2ee7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -37,6 +37,7 @@ import org.apache.spark.sql.errors.{QueryCompilationErrors, QueryExecutionErrors import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.{NumericType, StructType} +import org.apache.spark.util.ArrayImplicits._ /** * A set of methods for aggregations on a `DataFrame`, created by [[Dataset#groupBy groupBy]], @@ -432,7 +433,7 @@ class RelationalGroupedDataset protected[sql]( .sort(pivotColumn) // ensure that the output columns are in a consistent logical order .collect() .map(_.get(0)) - .toSeq + .toImmutableArraySeq if (values.length > maxValues) { throw QueryCompilationErrors.aggregationFunctionAppliedOnNonNumericColumnError( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 49546935da932..d6a46d47c1047 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ /** * For lazy computing, be sure the generator.terminate() called in the very last @@ -186,7 +187,7 @@ case class GenerateExec( case ArrayType(st: StructType, nullable) if e.inline => val row = codeGenAccessor(ctx, data.value, "col", index, st, nullable, checks) val fieldChecks = checks ++ optionalCode(nullable, row.isNull) - val columns = st.fields.toSeq.zipWithIndex.map { case (f, i) => + val columns = st.fields.toImmutableArraySeq.zipWithIndex.map { case (f, i) => codeGenAccessor( ctx, row.value, @@ -258,7 +259,7 @@ case class GenerateExec( val checks = optionalCode(outer, s"!$hasNext") val values = e.dataType match { case ArrayType(st: StructType, nullable) => - st.fields.toSeq.zipWithIndex.map { case (f, i) => + st.fields.toImmutableArraySeq.zipWithIndex.map { case (f, i) => codeGenAccessor(ctx, current, s"st_col${i}", s"$i", f.dataType, f.nullable, checks) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala index f6b5ba15afbd4..95b4b979348d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -74,7 +74,7 @@ object HiveResult { executedPlan.executeCollect().map(_.getString(1)).toImmutableArraySeq case other => val timeFormatters = getTimeFormatters - val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toSeq + val result: Seq[Seq[Any]] = other.executeCollectPublic().map(_.toSeq).toImmutableArraySeq // We need the types so we can output struct field names val types = executedPlan.output.map(_.dataType) // Reformat to match hive tab delimited output. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 32db94b3cdfc0..750f49c25b6d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -112,7 +112,7 @@ class DefaultCachedBatchSerializer extends SimpleMetricsCachedBatchSerializer { } val stats = InternalRow.fromSeq( - columnBuilders.flatMap(_.columnStats.collectedStatistics).toSeq) + columnBuilders.flatMap(_.columnStats.collectedStatistics).toImmutableArraySeq) DefaultCachedBatch(rowCount, columnBuilders.map { builder => JavaUtils.bufferToArray(builder.build()) }, stats) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index a8ec810fab3a2..1283bd8809082 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.CommandExecutionMode import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ /** * A command used to create a data source table. @@ -84,7 +85,7 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo // This is guaranteed in `PreprocessDDL`. assert(table.partitionColumnNames.isEmpty) dataSource match { - case r: HadoopFsRelation => r.partitionSchema.fieldNames.toSeq + case r: HadoopFsRelation => r.partitionSchema.fieldNames.toImmutableArraySeq case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PlanPythonDataSourceScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PlanPythonDataSourceScan.scala index c7eb461450881..ec4c7c188fa07 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PlanPythonDataSourceScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PlanPythonDataSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{Generate, LogicalPlan, Proje import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.PYTHON_DATA_SOURCE import org.apache.spark.sql.execution.python.UserDefinedPythonDataSourceReadRunner +import org.apache.spark.util.ArrayImplicits._ /** * A logical rule to plan reads from a Python data source. @@ -53,7 +54,7 @@ object PlanPythonDataSourceScan extends Rule[LogicalPlan] { val info = new UserDefinedPythonDataSourceReadRunner(dataSource, schema).runInPython() val readerFunc = SimplePythonFunction( - command = info.func.toSeq, + command = info.func.toImmutableArraySeq, envVars = dataSource.envVars, pythonIncludes = dataSource.pythonIncludes, pythonExec = dataSource.pythonExec, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala index c1d02ba5a227d..1cab5eace2bea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilters.scala @@ -41,6 +41,7 @@ import org.apache.spark.sql.catalyst.util.RebaseDateTime.{rebaseGregorianToJulia import org.apache.spark.sql.internal.LegacyBehaviorPolicy import org.apache.spark.sql.sources import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.ArrayImplicits._ /** * Some utility function to convert Spark data source filters to Parquet filters. @@ -98,7 +99,7 @@ class ParquetFilters( val primitiveFields = getPrimitiveFields(schema.getFields.asScala.toSeq).map { field => import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper - (field.fieldNames.toSeq.quoted, field) + (field.fieldNames.toImmutableArraySeq.quoted, field) } if (caseSensitive) { primitiveFields.toMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index ef82a81cb7e00..16889b247f253 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -131,7 +131,7 @@ class DataSourceV2Strategy(session: SparkSession) extends Strategy with Predicat case PhysicalOperation(project, filters, DataSourceV2ScanRelation(_, scan: LocalScan, output, _, _)) => - val localScanExec = LocalTableScanExec(output, scan.rows().toSeq) + val localScanExec = LocalTableScanExec(output, scan.rows().toImmutableArraySeq) withProjectAndFilter(project, filters, localScanExec, needsUnsafeConversion = false) :: Nil case PhysicalOperation(project, filters, relation: DataSourceV2ScanRelation) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala index 36ee01e1c1c4a..02e19dd053f29 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DistributionAndOrderingUtils.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.catalog.functions.ScalarFunction import org.apache.spark.sql.connector.distributions._ import org.apache.spark.sql.connector.write.{RequiresDistributionAndOrdering, Write} import org.apache.spark.sql.errors.QueryCompilationErrors +import org.apache.spark.util.ArrayImplicits._ object DistributionAndOrderingUtils { @@ -44,7 +45,7 @@ object DistributionAndOrderingUtils { .map(e => resolveTransformExpression(e).asInstanceOf[SortOrder]) case d: ClusteredDistribution => d.clustering.map(e => toCatalyst(e, query, funCatalogOpt)) - .map(e => resolveTransformExpression(e)).toSeq + .map(e => resolveTransformExpression(e)).toImmutableArraySeq case _: UnspecifiedDistribution => Seq.empty[Expression] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala index 3b806bc9b487d..f56f9436d9437 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFile import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.util.SchemaUtils +import org.apache.spark.util.ArrayImplicits._ abstract class FileTable( sparkSession: SparkSession, @@ -99,7 +100,8 @@ abstract class FileTable( StructType(fields) } - override def partitioning: Array[Transform] = fileIndex.partitionSchema.names.toSeq.asTransforms + override def partitioning: Array[Transform] = + fileIndex.partitionSchema.names.toImmutableArraySeq.asTransforms override def properties: util.Map[String, String] = options.asCaseSensitiveMap diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala index 3de4692c83b09..34a1adcb6e091 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PushDownUtils.scala @@ -72,7 +72,7 @@ object PushDownUtils { // faster, while the untranslated filters are complicated filters that take more time to // evaluate, so we want to evaluate the postScanFilters filters first. (Left(r.pushedFilters().toImmutableArraySeq), - (postScanFilters ++ untranslatableExprs).toSeq) + (postScanFilters ++ untranslatableExprs).toImmutableArraySeq) case r: SupportsPushDownV2Filters => // A map from translated data source leaf node filters to original catalyst filter @@ -105,7 +105,7 @@ object PushDownUtils { // faster, while the untranslated filters are complicated filters that take more time to // evaluate, so we want to evaluate the postScanFilters filters first. (Right(r.pushedPredicates.toImmutableArraySeq), - (postScanFilters ++ untranslatableExprs).toSeq) + (postScanFilters ++ untranslatableExprs).toImmutableArraySeq) case f: FileScanBuilder => val postScanFilters = f.pushFilters(filters) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala index 933e82a259dbb..6cd7ec403be31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -115,7 +115,8 @@ class V2SessionCatalog(catalog: SessionCatalog) partitions: Array[Transform], properties: util.Map[String, String]): Table = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.TransformHelper - val (partitionColumns, maybeBucketSpec, maybeClusterBySpec) = partitions.toSeq.convertTransforms + val (partitionColumns, maybeBucketSpec, maybeClusterBySpec) = + partitions.toImmutableArraySeq.convertTransforms val provider = properties.getOrDefault(TableCatalog.PROP_PROVIDER, conf.defaultDataSourceName) val tableProperties = properties.asScala val location = Option(properties.get(TableCatalog.PROP_LOCATION)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala index 7e534b92dd92e..319cc1c731577 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2Writes.scala @@ -33,6 +33,7 @@ import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWrite, WriteT import org.apache.spark.sql.internal.connector.SupportsStreamingUpdateAsAppend import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ArrayImplicits._ /** * A rule that constructs logical writes. @@ -91,7 +92,7 @@ object V2Writes extends Rule[LogicalPlan] with PredicateHelper { val writeBuilder = newWriteBuilder(table, writeOptions, query.schema, queryId) val write = buildWriteForMicroBatch(table, writeBuilder, outputMode) val microBatchWrite = new MicroBatchWrite(batchId, write.toStreaming) - val customMetrics = write.supportedCustomMetrics.toSeq + val customMetrics = write.supportedCustomMetrics.toImmutableArraySeq val funCatalogOpt = relation.flatMap(_.funCatalog) val newQuery = DistributionAndOrderingUtils.prepareQuery(write, query, funCatalogOpt) WriteToDataSourceV2(relation, microBatchWrite, newQuery, customMetrics) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala index 264ec09c00b28..8b4e2737744d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StateTable.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.execution.datasources.v2.state.utils.SchemaUtil import org.apache.spark.sql.execution.streaming.state.StateStoreConf import org.apache.spark.sql.types.{DataType, IntegerType, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits._ /** An implementation of [[Table]] with [[SupportsRead]] for State Store data source. */ class StateTable( @@ -67,7 +68,7 @@ class StateTable( override def properties(): util.Map[String, String] = Map.empty[String, String].asJava private def isValidSchema(schema: StructType): Boolean = { - if (schema.fieldNames.toSeq != Seq("key", "value")) { + if (schema.fieldNames.toImmutableArraySeq != Seq("key", "value")) { false } else if (!SchemaUtil.getSchemaAsDataType(schema, "key").isInstanceOf[StructType]) { false diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 1fcd9499b8c9b..a2d9a6705f97f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -39,6 +39,7 @@ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Clock class ContinuousExecution( @@ -95,7 +96,7 @@ class ContinuousExecution( } val streamingWrite = write.toStreaming - val customMetrics = write.supportedCustomMetrics.toSeq + val customMetrics = write.supportedCustomMetrics.toImmutableArraySeq WriteToContinuousDataSource(streamingWrite, _logicalPlan, customMetrics) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index ae342813338c2..654d52bf91652 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -405,8 +405,8 @@ class RocksDBFileManager( val sstDir = new Path(dfsRootDir, RocksDBImmutableFile.SST_FILES_DFS_SUBDIR) val logDir = new Path(dfsRootDir, RocksDBImmutableFile.LOG_FILES_DFS_SUBDIR) - val allSstFiles = if (fm.exists(sstDir)) fm.list(sstDir).toSeq else Seq.empty - val allLogFiles = if (fm.exists(logDir)) fm.list(logDir).toSeq else Seq.empty + val allSstFiles = if (fm.exists(sstDir)) fm.list(sstDir).toImmutableArraySeq else Seq.empty + val allLogFiles = if (fm.exists(logDir)) fm.list(logDir).toImmutableArraySeq else Seq.empty filesToDelete ++= findOrphanFiles(fileToMaxUsedVersion.keys.toSeq, allSstFiles ++ allLogFiles) .map(_ -> -1L) logInfo(s"Deleting ${filesToDelete.size} files not used in versions >= $minVersionToRetain") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala index e40373917c525..68ea33572c858 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/AggregateProcessor.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.util.ArrayImplicits._ /** @@ -124,7 +125,8 @@ private[window] final class AggregateProcessor( private[this] val join = new JoinedRow private[this] val numImperatives = imperatives.length - private[this] val buffer = new SpecificInternalRow(bufferSchema.toSeq.map(_.dataType)) + private[this] val buffer = + new SpecificInternalRow(bufferSchema.toImmutableArraySeq.map(_.dataType)) initialProjection.target(buffer) updateProjection.target(buffer) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index a849c3894f0d6..6cea838311acd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray +import org.apache.spark.util.ArrayImplicits._ /** @@ -110,7 +111,7 @@ abstract class OffsetWindowFunctionFrameBase( protected val projection = { // Collect the expressions and bind them. val boundExpressions = Seq.fill(ordinal)(NoOp) ++ bindReferences( - expressions.toSeq.map(_.input), inputAttrs) + expressions.toImmutableArraySeq.map(_.input), inputAttrs) // Create the projection. newMutableProjection(boundExpressions, Nil).target(target) @@ -119,7 +120,7 @@ abstract class OffsetWindowFunctionFrameBase( /** Create the projection used when the offset row DOES NOT exists. */ protected val fillDefaultValue = { // Collect the expressions and bind them. - val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => + val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toImmutableArraySeq.map { e => if (e.default == null || e.default.foldable && e.default.eval() == null) { // The default value is null. Literal.create(null, e.dataType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index c37f31b9fa670..3eb1b2637ec51 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -377,7 +377,8 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { val columns = sparkSession.sessionState.executePlan(plan).analyzed match { case ResolvedTable(_, _, table, _) => // TODO (SPARK-45787): Support clusterBySpec for listColumns(). - val (partitionColumnNames, bucketSpecOpt, _) = table.partitioning.toSeq.convertTransforms + val (partitionColumnNames, bucketSpecOpt, _) = + table.partitioning.toImmutableArraySeq.convertTransforms val bucketColumnNames = bucketSpecOpt.map(_.bucketColumnNames).getOrElse(Nil) schemaToColumns(table.schema(), partitionColumnNames.contains, bucketColumnNames.contains) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index f4665f8ac6773..59bc3f5d08a4f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -43,6 +43,7 @@ import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Utils, FileDat import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils /** @@ -302,7 +303,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { val cmd = CreateTable( UnresolvedIdentifier(originalMultipartIdentifier), df.schema.asNullable, - partitioningColumns.getOrElse(Nil).asTransforms.toSeq, + partitioningColumns.getOrElse(Nil).asTransforms.toImmutableArraySeq, tableSpec, ignoreIfExists = false) Dataset.ofRows(df.sparkSession, cmd) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala index 5ce7977ab168c..a1bb6029fdc3a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveUtils.scala @@ -47,6 +47,7 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.types._ +import org.apache.spark.util.ArrayImplicits._ import org.apache.spark.util.Utils @@ -384,7 +385,8 @@ private[spark] object HiveUtils extends Logging { logWarning(s"Hive jar path '${file.getPath}' does not exist.") Nil } else { - files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")).map(_.toURI.toURL).toSeq + files.filter(_.getName.toLowerCase(Locale.ROOT).endsWith(".jar")).map(_.toURI.toURL) + .toImmutableArraySeq } } else { file.toURI.toURL :: Nil @@ -468,7 +470,7 @@ private[spark] object HiveUtils extends Logging { version = metaVersion, sparkConf = conf, hadoopConf = hadoopConf, - execJars = jars.toSeq, + execJars = jars.toImmutableArraySeq, config = configurations, isolationOn = true, barrierPrefixes = hiveMetastoreBarrierPrefixes, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index bfc4fac6280da..d72406f094a6b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -171,7 +171,7 @@ class HadoopTableReader( val partColsDelimited: String = partProps.getProperty(META_TABLE_PARTITION_COLUMNS) // Partitioning columns are delimited by "/" - val partCols = partColsDelimited.trim().split("/").toSeq + val partCols = partColsDelimited.trim().split("/").toImmutableArraySeq // 'partValues[i]' contains the value for the partitioning column at 'partCols[i]'. val partValues = if (partSpec == null) { Array.fill(partCols.size)(new String) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index bac5d50bb692e..55b0daf22e91c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -32,6 +32,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.streaming._ import org.apache.spark.streaming.api.java.JavaDStream._ import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.util.ArrayImplicits._ /** * As a workaround for https://issues.scala-lang.org/browse/SI-8905, implementations @@ -146,7 +147,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * an array. */ def glom(): JavaDStream[JList[T]] = - new JavaDStream(dstream.glom().map(_.toSeq.asJava)) + new JavaDStream(dstream.glom().map(_.toImmutableArraySeq.asJava)) /** Return the [[org.apache.spark.streaming.StreamingContext]] associated with this DStream */