Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -243,7 +243,7 @@ private[kafka010] class KafkaOffsetReaderAdmin(
}

tp -> offset
}.toMap
}
}

private def fetchSpecificOffsets0(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -291,7 +291,7 @@ private[kafka010] class KafkaOffsetReaderConsumer(
}

tp -> offset
}.toMap
}
}

private def fetchSpecificOffsets0(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -99,7 +99,7 @@ object SchemaConverters {

Option(
fd.getMessageType.getFields.asScala
.flatMap(structFieldFor(_, newRecordNames.toSet))
.flatMap(structFieldFor(_, newRecordNames))
.toSeq)
.filter(_.nonEmpty)
.map(StructType.apply)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -568,7 +568,7 @@ private[spark] class ExecutorAllocationManager(
// We don't want to change our target number of executors, because we already did that
// when the task backlog decreased.
if (decommissionEnabled) {
val executorIdsWithoutHostLoss = executorIdsToBeRemoved.toSeq.map(
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No need to toSeq here because executorIdsToBeRemoved(ArrayBuffer).map finally calls toArray

val executorIdsWithoutHostLoss = executorIdsToBeRemoved.map(
id => (id, ExecutorDecommissionInfo("spark scale down"))).toArray
client.decommissionExecutors(
executorIdsWithoutHostLoss,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -424,7 +424,7 @@ class ExecutorPeakMetricsDistributions private[spark](
/** Returns the distributions for the specified metric. */
def getMetricDistribution(metricName: String): IndexedSeq[Double] = {
val sorted = executorMetrics.map(_.getMetricValue(metricName)).sorted
indices.map(i => sorted(i.toInt).toDouble).toIndexedSeq
indices.map(i => sorted(i.toInt).toDouble)
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -555,7 +555,7 @@ object StructType extends AbstractDataType {

def apply(fields: java.util.List[StructField]): StructType = {
import scala.collection.JavaConverters._
StructType(fields.asScala.toSeq)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This represents fields.asScala.toSeq.toArray, seems toSeq is redundant

StructType(fields.asScala.toArray)
}

private[sql] def fromAttributes(attributes: Seq[Attribute]): StructType =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends
s"""
| Registering new PythonUDF:
| name: $name
| command: ${udf.func.command.toSeq}
| command: ${udf.func.command}
| envVars: ${udf.func.envVars}
| pythonIncludes: ${udf.func.pythonIncludes}
| pythonExec: ${udf.func.pythonExec}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -545,7 +545,7 @@ class ResolveSessionCatalog(val catalogManager: CatalogManager)
} else {
CatalogTableType.MANAGED
}
val (partitionColumns, maybeBucketSpec) = partitioning.toSeq.convertTransforms
val (partitionColumns, maybeBucketSpec) = partitioning.convertTransforms

CatalogTable(
identifier = table,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -305,9 +305,9 @@ object ShufflePartitionsUtil extends Logging {
val dataSize = spec.startReducerIndex.until(spec.endReducerIndex)
.map(mapStats.bytesByPartitionId).sum
spec.copy(dataSize = Some(dataSize))
}.toSeq
case None => partitionSpecs.map(_.copy(dataSize = Some(0))).toSeq
}.toSeq
}
case None => partitionSpecs.map(_.copy(dataSize = Some(0)))
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -939,7 +939,7 @@ case class ShowTablePropertiesCommand(
}
case None =>
properties.filterKeys(!_.startsWith(CatalogTable.VIEW_PREFIX))
.toSeq.sortBy(_._1).map(p => Row(p._1, p._2)).toSeq
.toSeq.sortBy(_._1).map(p => Row(p._1, p._2))
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

toSeq has been called previously

}
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -39,7 +39,7 @@ case class ShowFunctionsExec(
pattern: Option[String]) extends V2CommandExec with LeafExecNode {

private def applyPattern(names: Seq[String]): Seq[String] = {
StringUtils.filterPattern(names.toSeq, pattern.getOrElse("*"))
StringUtils.filterPattern(names, pattern.getOrElse("*"))
}

override protected def run(): Seq[InternalRow] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ case class ShowTablePropertiesExec(
}
case None =>
properties.toSeq.sortBy(_._1).map(kv =>
toCatalystRow(kv._1, kv._2)).toSeq
toCatalystRow(kv._1, kv._2))
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

toSeq has been called previously

}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -776,7 +776,7 @@ private[hive] class HiveClientImpl(
assert(s.values.forall(_.nonEmpty), s"partition spec '$s' is invalid")
shim.getPartitionNames(client, table.database, table.identifier.table, s.asJava, -1)
}
hivePartitionNames.sorted.toSeq
hivePartitionNames.sorted
}

override def getPartitionOption(
Expand Down Expand Up @@ -812,7 +812,7 @@ private[hive] class HiveClientImpl(
val parts = shim.getPartitions(client, hiveTable, partSpec.asJava)
.map(fromHivePartition(_, absoluteUri))
HiveCatalogMetrics.incrementFetchedPartitions(parts.length)
parts.toSeq
parts
}

override def getPartitionsByFilter(
Expand Down