Skip to content

[SPARK-41636][SQL] Make sure selectFilters returns predicates in deterministic order #42265

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution.datasources

import java.util.Locale

import scala.collection.immutable.ListMap
import scala.collection.mutable

import org.apache.hadoop.fs.Path
Expand Down Expand Up @@ -670,9 +671,10 @@ object DataSourceStrategy
// A map from original Catalyst expressions to corresponding translated data source filters.
// If a predicate is not in this map, it means it cannot be pushed down.
val supportNestedPredicatePushdown = DataSourceUtils.supportNestedPredicatePushdown(relation)
val translatedMap: Map[Expression, Filter] = predicates.flatMap { p =>
// SPARK-41636: we keep the order of the predicates to avoid CodeGenerator cache misses
val translatedMap: Map[Expression, Filter] = ListMap(predicates.flatMap { p =>
translateFilter(p, supportNestedPredicatePushdown).map(f => p -> f)
}.toMap
}: _*)

val pushedFilters: Seq[Filter] = translatedMap.values.toSeq

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -324,4 +324,18 @@ class DataSourceStrategySuite extends PlanTest with SharedSparkSession {
DataSourceStrategy.translateFilter(catalystFilter, true)
}
}

test("SPARK-41636: selectFilters returns predicates in deterministic order") {

val predicates = Seq(EqualTo($"id", 1), EqualTo($"id", 2),
EqualTo($"id", 3), EqualTo($"id", 4), EqualTo($"id", 5), EqualTo($"id", 6))

val (unhandledPredicates, pushedFilters, handledFilters) =
DataSourceStrategy.selectFilters(FakeRelation(), predicates)
assert(unhandledPredicates.equals(predicates))
assert(pushedFilters.zipWithIndex.forall { case (f, i) =>
f.equals(sources.EqualTo("id", i + 1))
})
assert(handledFilters.isEmpty)
}
}