Skip to content

Commit 7ae6c8d

Browse files
LuciferYangHyukjinKwon
authored andcommitted
[SPARK-34118][CORE][SQL][2.4] Replaces filter and check for emptiness with exists or forall
### What changes were proposed in this pull request? This pr use `exists` or `forall` to simplify `filter + emptiness check`, it's semantically consistent, but looks simpler. The rule as follow: - `seq.filter(p).size == 0)` -> `!seq.exists(p)` - `seq.filter(p).length > 0` -> `seq.exists(p)` - `seq.filterNot(p).isEmpty` -> `seq.forall(p)` - `seq.filterNot(p).nonEmpty` -> `!seq.forall(p)` ### Why are the changes needed? Code Simpilefications. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? Pass the Jenkins or GitHub Action Closes #31192 from LuciferYang/SPARK-34118-24. Authored-by: yangjie01 <[email protected]> Signed-off-by: HyukjinKwon <[email protected]>
1 parent 63e93a5 commit 7ae6c8d

File tree

4 files changed

+6
-6
lines changed

4 files changed

+6
-6
lines changed

core/src/main/scala/org/apache/spark/api/r/RUtils.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -43,9 +43,9 @@ private[spark] object RUtils {
4343
* Check if SparkR is installed before running tests that use SparkR.
4444
*/
4545
def isSparkRInstalled: Boolean = {
46-
localSparkRPackagePath.filter { pkgDir =>
46+
localSparkRPackagePath.exists { pkgDir =>
4747
new File(Seq(pkgDir, "SparkR").mkString(File.separator)).exists
48-
}.isDefined
48+
}
4949
}
5050

5151
/**

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

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -338,7 +338,7 @@ class FileAppenderSuite extends SparkFunSuite with BeforeAndAfter with Logging {
338338
assert(generatedFiles.size > 1)
339339
if (isCompressed) {
340340
assert(
341-
generatedFiles.filter(_.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)).size > 0)
341+
generatedFiles.exists(_.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)))
342342
}
343343
val allText = generatedFiles.map { file =>
344344
if (file.getName.endsWith(RollingFileAppender.GZIP_LOG_SUFFIX)) {

sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -694,7 +694,7 @@ class JDBCSuite extends QueryTest
694694
test("Remap types via JdbcDialects") {
695695
JdbcDialects.registerDialect(testH2Dialect)
696696
val df = spark.read.jdbc(urlWithUserAndPass, "TEST.PEOPLE", new Properties())
697-
assert(df.schema.filter(_.dataType != org.apache.spark.sql.types.StringType).isEmpty)
697+
assert(!df.schema.exists(_.dataType != org.apache.spark.sql.types.StringType))
698698
val rows = df.collect()
699699
assert(rows(0).get(0).isInstanceOf[String])
700700
assert(rows(0).get(1).isInstanceOf[String])

sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -36,8 +36,8 @@ class DataSourceV2UtilsSuite extends SparkFunSuite {
3636
val cs = classOf[DataSourceV2WithSessionConfig].newInstance()
3737
val confs = DataSourceV2Utils.extractSessionConfigs(cs.asInstanceOf[DataSourceV2], conf)
3838
assert(confs.size == 2)
39-
assert(confs.keySet.filter(_.startsWith("spark.datasource")).size == 0)
40-
assert(confs.keySet.filter(_.startsWith("not.exist.prefix")).size == 0)
39+
assert(!confs.keySet.exists(_.startsWith("spark.datasource")))
40+
assert(!confs.keySet.exists(_.startsWith("not.exist.prefix")))
4141
assert(confs.keySet.contains("foo.bar"))
4242
assert(confs.keySet.contains("whateverConfigName"))
4343
}

0 commit comments

Comments
 (0)