From 9903e057e84a1f72e97dc92ee7b3c7a2ad3ffdb8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 7 Jun 2020 09:16:51 -0700 Subject: [PATCH 1/8] Nested column predicate pushdown for ORC. --- .../datasources/DataSourceStrategy.scala | 2 + .../datasources/orc/OrcFiltersBase.scala | 49 ++- .../datasources/v2/orc/OrcScanBuilder.scala | 7 +- .../NestedColumnPredicateTest.scala | 70 ++++ .../execution/datasources/orc/OrcTest.scala | 12 +- .../datasources/orc/OrcV1FilterSuite.scala | 2 +- .../parquet/ParquetFilterSuite.scala | 43 +- .../datasources/orc/OrcFilters.scala | 26 +- .../datasources/orc/OrcFilterSuite.scala | 331 +++++++++------- .../datasources/orc/OrcFilters.scala | 26 +- .../datasources/orc/OrcFilterSuite.scala | 349 +++++++++------- .../sql/hive/orc/HiveOrcFilterSuite.scala | 374 +++++++++++------- 12 files changed, 779 insertions(+), 512 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala index 23454d7d5e7f3..ada04c2382b72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala @@ -668,6 +668,8 @@ abstract class PushableColumnBase { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper def helper(e: Expression): Option[Seq[String]] = e match { case a: Attribute => + // Attribute that contains dot "." in name is supported only when + // nested predicate pushdown is enabled. if (nestedPredicatePushdownEnabled || !a.name.contains(".")) { Some(Seq(a.name)) } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala index e673309188756..94af19cd428ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.execution.datasources.orc +import java.util.Locale + +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.sources.{And, Filter} -import org.apache.spark.sql.types.{AtomicType, BinaryType, DataType} +import org.apache.spark.sql.types.{AtomicType, BinaryType, DataType, StructField, StructType} /** * Methods that can be shared when upgrading the built-in Hive. @@ -37,12 +40,44 @@ trait OrcFiltersBase { } /** - * Return true if this is a searchable type in ORC. - * Both CharType and VarcharType are cleaned at AstBuilder. + * This method returns a map which contains ORC field name and data type. Each key + * represents a column; `dots` are used as separators for nested columns. If any part + * of the names contains `dots`, it is quoted to avoid confusion. See + * `org.apache.spark.sql.connector.catalog.quote` for implementation details. */ - protected[sql] def isSearchableType(dataType: DataType) = dataType match { - case BinaryType => false - case _: AtomicType => true - case _ => false + protected[sql] def getNameToOrcFieldMap( + schema: StructType, + caseSensitive: Boolean): Map[String, DataType] = { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper + + def getPrimitiveFields( + fields: Seq[StructField], + parentFieldNames: Array[String] = Array.empty): Seq[(String, DataType)] = { + fields.flatMap { f => + f.dataType match { + case st: StructType => + getPrimitiveFields(st.fields.toSeq, parentFieldNames :+ f.name) + case BinaryType => None + case _: AtomicType => + Some(((parentFieldNames :+ f.name).toSeq.quoted, f.dataType)) + case _ => None + } + } + } + + val primitiveFields = getPrimitiveFields(schema.fields) + if (caseSensitive) { + primitiveFields.toMap + } else { + // Don't consider ambiguity here, i.e. more than one field is matched in case insensitive + // mode, just skip pushdown for these fields, they will trigger Exception when reading, + // See: SPARK-25175. + val dedupPrimitiveFields = + primitiveFields + .groupBy(_._1.toLowerCase(Locale.ROOT)) + .filter(_._2.size == 1) + .mapValues(_.head._2) + CaseInsensitiveMap(dedupPrimitiveFields) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 9f40f5faa2e99..6839ee19a5fa2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.orc.OrcFilters import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -60,10 +61,8 @@ case class OrcScanBuilder( // changed `hadoopConf` in executors. OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } - val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap - // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. - val newFilters = filters.filter(!_.containsNestedColumn) - _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, newFilters).toArray + val dataTypeMap = OrcFilters.getNameToOrcFieldMap(schema, SQLConf.get.caseSensitiveAnalysis) + _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, filters).toArray } filters } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala new file mode 100644 index 0000000000000..e7d0b76cb1701 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.functions.struct +import org.apache.spark.sql.types.StructType + +/** + * Defines common stuff for nested column predicate pushdown test, e.g. `ParquetFilterSuite`. + */ +trait NestedColumnPredicateTest { + /** + * Takes single level `inputDF` dataframe to generate multi-level nested + * dataframes as new test data. + * + * This method accepts a function to run test. The given function takes three + * parameters: a DataFrame which ranges from zero-nested to multi-level nested, + * a string of the primitive field name, and a function that produces expected + * result of collected column. + */ + protected def withNestedDataFrame(inputDF: DataFrame) + (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { + assert(inputDF.schema.fields.length == 1) + assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) + val df = inputDF.toDF("temp") + Seq( + ( + df.withColumnRenamed("temp", "a"), + "a", // zero nesting + (x: Any) => x), + ( + df.withColumn("a", struct(df("temp") as "b")).drop("temp"), + "a.b", // one level nesting + (x: Any) => Row(x)), + ( + df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"), + "a.b.c", // two level nesting + (x: Any) => Row(Row(x)) + ), + ( + df.withColumnRenamed("temp", "a.b"), + "`a.b`", // zero nesting with column name containing `dots` + (x: Any) => x + ), + ( + df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"), + "`a.b`.`c.d`", // one level nesting with column names containing `dots` + (x: Any) => Row(x) + ) + ).foreach { case (df, colName, resultFun) => + runTest(df, colName, resultFun) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index e929f904c798d..f0356d9302f25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -78,12 +78,16 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor (f: String => Unit): Unit = withDataSourceFile(data)(f) /** - * Writes `data` to a Orc file and reads it back as a `DataFrame`, + * Writes `df` dataframe to a Orc file and reads it back as a `DataFrame`, * which is then passed to `f`. The Orc file will be deleted after `f` returns. */ - protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag] - (data: Seq[T], testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f) + protected def withOrcDataFrame(df: DataFrame, testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = { + withTempPath { file => + df.write.format(dataSourceName).save(file.getCanonicalPath) + readFile(file.getCanonicalPath, testVectorized)(f) + } + } /** * Writes `data` to a Orc file, reads it back as a `DataFrame` and registers it as a diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala index d0032df488f47..72f01101e194c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.orc import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.{Column, DataFrame, Row} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index d20a07f420e87..06155da45180b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, NestedColumnPredicateTest, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions._ @@ -62,7 +62,8 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} * dependent on this configuration, don't forget you better explicitly set this configuration * within the test. */ -abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSparkSession { +abstract class ParquetFilterSuite + extends QueryTest with ParquetTest with NestedColumnPredicateTest with SharedSparkSession { protected def createParquetFilters( schema: MessageType, @@ -105,44 +106,6 @@ abstract class ParquetFilterSuite extends QueryTest with ParquetTest with Shared checkFilterPredicate(predicate, filterClass, Seq(Row(expected)))(df) } - /** - * Takes single level `inputDF` dataframe to generate multi-level nested - * dataframes as new test data. - */ - private def withNestedDataFrame(inputDF: DataFrame) - (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { - assert(inputDF.schema.fields.length == 1) - assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) - val df = inputDF.toDF("temp") - Seq( - ( - df.withColumnRenamed("temp", "a"), - "a", // zero nesting - (x: Any) => x), - ( - df.withColumn("a", struct(df("temp") as "b")).drop("temp"), - "a.b", // one level nesting - (x: Any) => Row(x)), - ( - df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"), - "a.b.c", // two level nesting - (x: Any) => Row(Row(x)) - ), - ( - df.withColumnRenamed("temp", "a.b"), - "`a.b`", // zero nesting with column name containing `dots` - (x: Any) => x - ), - ( - df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"), - "`a.b`.`c.d`", // one level nesting with column names containing `dots` - (x: Any) => Row(x) - ) - ).foreach { case (df, colName, resultFun) => - runTest(df, colName, resultFun) - } - } - private def testTimestampPushdown(data: Seq[String], java8Api: Boolean): Unit = { implicit class StringToTs(s: String) { def ts: Timestamp = Timestamp.valueOf(s) diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index b68563956c82c..5c912ea384945 100644 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -27,7 +27,7 @@ import org.apache.orc.storage.serde2.io.HiveDecimalWritable import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -68,11 +68,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap + val dataTypeMap = OrcFilters.getNameToOrcFieldMap(schema, SQLConf.get.caseSensitiveAnalysis) // Combines all convertible filters using `And` to produce a single conjunction - // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. - val newFilters = filters.filter(!_.containsNestedColumn) - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, newFilters)) + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) conjunctionOptional.map { conjunction => // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. // The input predicate is fully convertible. There should not be any empty result in the @@ -231,37 +229,37 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters // in order to distinguish predicate pushdown for nested columns. expression match { - case EqualTo(name, value) if isSearchableType(dataTypeMap(name)) => + case EqualTo(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().equals(name, getType(name), castedValue).end()) - case EqualNullSafe(name, value) if isSearchableType(dataTypeMap(name)) => + case EqualNullSafe(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().nullSafeEquals(name, getType(name), castedValue).end()) - case LessThan(name, value) if isSearchableType(dataTypeMap(name)) => + case LessThan(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().lessThan(name, getType(name), castedValue).end()) - case LessThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + case LessThanOrEqual(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThan(name, value) if isSearchableType(dataTypeMap(name)) => + case GreaterThan(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startNot().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + case GreaterThanOrEqual(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startNot().lessThan(name, getType(name), castedValue).end()) - case IsNull(name) if isSearchableType(dataTypeMap(name)) => + case IsNull(name) if dataTypeMap.contains(name) => Some(builder.startAnd().isNull(name, getType(name)).end()) - case IsNotNull(name) if isSearchableType(dataTypeMap(name)) => + case IsNotNull(name) if dataTypeMap.contains(name) => Some(builder.startNot().isNull(name, getType(name)).end()) - case In(name, values) if isSearchableType(dataTypeMap(name)) => + case In(name, values) if dataTypeMap.contains(name) => val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(name))) Some(builder.startAnd().in(name, getType(name), castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 88b4b243b543a..e4392c04940f4 100644 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -27,9 +27,11 @@ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, Column, DataFrame} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.NestedColumnPredicateTest import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf @@ -43,7 +45,7 @@ import org.apache.spark.sql.types._ * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ -class OrcFilterSuite extends OrcTest with SharedSparkSession { +class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedSparkSession { override protected def sparkConf: SparkConf = super @@ -92,155 +94,206 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("filter pushdown - integer") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val intAttr = df(colName).expr + assert(df(colName).expr.dataType === IntegerType) + + checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - long") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toLong))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val longAttr = df(colName).expr + assert(df(colName).expr.dataType === LongType) + + checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - float") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toFloat))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val floatAttr = df(colName).expr + assert(df(colName).expr.dataType === FloatType) + + checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - double") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toDouble))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val doubleAttr = df(colName).expr + assert(df(colName).expr.dataType === DoubleType) + + checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - string") { - withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame((1 to 4).map(i => Tuple1(i.toString)).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val strAttr = df(colName).expr + assert(df(colName).expr.dataType === StringType) + + checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - boolean") { - withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (true :: false :: Nil).map(b => Tuple1.apply(Option(b))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val booleanAttr = df(colName).expr + assert(df(colName).expr.dataType === BooleanType) + + checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> booleanAttr, + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - decimal") { - withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val decimalAttr = df(colName).expr + assert(df(colName).expr.dataType === DecimalType(38, 18)) + + checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) + } } } @@ -284,26 +337,28 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("filter pushdown - combinations with logical operators") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => checkFilterPredicate( - $"_1".isNotNull, + UnresolvedAttribute("_1").isNotNull, "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" ) checkFilterPredicate( - $"_1" =!= 1, + UnresolvedAttribute("_1") =!= 1, "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - !($"_1" < 4), + !(UnresolvedAttribute("_1") < 4), "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - $"_1" < 2 || $"_1" > 3, + UnresolvedAttribute("_1") < 2 || UnresolvedAttribute("_1") > 3, "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + "expr = (or leaf-0 (not leaf-1))" ) checkFilterPredicate( - $"_1" < 2 && $"_1" > 3, + UnresolvedAttribute("_1") < 2 && UnresolvedAttribute("_1") > 3, "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" ) @@ -342,20 +397,22 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("no filter pushdown - non-supported types") { + import testImplicits._ + implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate($"_1".isNull, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i))).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1").isNull, noneSupported = true) } // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b)).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1") <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) + checkNoFilterPredicate(UnresolvedAttribute("_1").isNotNull, noneSupported = true) } } diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 4b642080d25ad..8f48f0aa05688 100644 --- a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -27,7 +27,7 @@ import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.util.DateTimeUtils.{instantToMicros, localDateToDays, toJavaDate, toJavaTimestamp} -import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.quoteIfNeeded +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -68,11 +68,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = schema.map(f => quoteIfNeeded(f.name) -> f.dataType).toMap + val dataTypeMap = OrcFilters.getNameToOrcFieldMap(schema, SQLConf.get.caseSensitiveAnalysis) // Combines all convertible filters using `And` to produce a single conjunction - // TODO (SPARK-25557): ORC doesn't support nested predicate pushdown, so they are removed. - val newFilters = filters.filter(!_.containsNestedColumn) - val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, newFilters)) + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) conjunctionOptional.map { conjunction => // Then tries to build a single ORC `SearchArgument` for the conjunction predicate. // The input predicate is fully convertible. There should not be any empty result in the @@ -231,37 +229,37 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters // in order to distinguish predicate pushdown for nested columns. expression match { - case EqualTo(name, value) if isSearchableType(dataTypeMap(name)) => + case EqualTo(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().equals(name, getType(name), castedValue).end()) - case EqualNullSafe(name, value) if isSearchableType(dataTypeMap(name)) => + case EqualNullSafe(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().nullSafeEquals(name, getType(name), castedValue).end()) - case LessThan(name, value) if isSearchableType(dataTypeMap(name)) => + case LessThan(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().lessThan(name, getType(name), castedValue).end()) - case LessThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + case LessThanOrEqual(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startAnd().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThan(name, value) if isSearchableType(dataTypeMap(name)) => + case GreaterThan(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startNot().lessThanEquals(name, getType(name), castedValue).end()) - case GreaterThanOrEqual(name, value) if isSearchableType(dataTypeMap(name)) => + case GreaterThanOrEqual(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) Some(builder.startNot().lessThan(name, getType(name), castedValue).end()) - case IsNull(name) if isSearchableType(dataTypeMap(name)) => + case IsNull(name) if dataTypeMap.contains(name) => Some(builder.startAnd().isNull(name, getType(name)).end()) - case IsNotNull(name) if isSearchableType(dataTypeMap(name)) => + case IsNotNull(name) if dataTypeMap.contains(name) => Some(builder.startNot().isNull(name, getType(name)).end()) - case In(name, values) if isSearchableType(dataTypeMap(name)) => + case In(name, values) if dataTypeMap.contains(name) => val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(name))) Some(builder.startAnd().in(name, getType(name), castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 2263179515a5f..b6d0dfea7e630 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -26,13 +26,14 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.spark.SparkConf -import org.apache.spark.sql.{AnalysisException, Column, DataFrame} +import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Row} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanRelation} -import org.apache.spark.sql.execution.datasources.v2.orc.{OrcScan, OrcTable} +import org.apache.spark.sql.execution.datasources.NestedColumnPredicateTest +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ @@ -44,7 +45,7 @@ import org.apache.spark.sql.types._ * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ -class OrcFilterSuite extends OrcTest with SharedSparkSession { +class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedSparkSession { override protected def sparkConf: SparkConf = super @@ -74,9 +75,9 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } protected def checkFilterPredicate - (predicate: Predicate, filterOperator: PredicateLeaf.Operator) - (implicit df: DataFrame): Unit = { - def checkComparisonOperator(filter: SearchArgument) = { + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument): Unit = { val operator = filter.getLeaves.asScala assert(operator.map(_.getOperator).contains(filterOperator)) } @@ -86,162 +87,214 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { protected def checkFilterPredicate (predicate: Predicate, stringExpr: String) (implicit df: DataFrame): Unit = { - def checkLogicalOperator(filter: SearchArgument) = { + def checkLogicalOperator(filter: SearchArgument): Unit = { assert(filter.toString == stringExpr) } checkFilterPredicate(df, predicate, checkLogicalOperator) } test("filter pushdown - integer") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i))).toDF("i")) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val intAttr = df(colName).expr + assert(df(colName).expr.dataType === IntegerType) + + checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - long") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toLong))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val longAttr = df(colName).expr + assert(df(colName).expr.dataType === LongType) + + checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - float") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toFloat))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val floatAttr = df(colName).expr + assert(df(colName).expr.dataType === FloatType) + + checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - double") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toDouble))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val doubleAttr = df(colName).expr + assert(df(colName).expr.dataType === DoubleType) + + checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - string") { - withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (1 to 4).map(i => Tuple1(i.toString)).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val strAttr = df(colName).expr + assert(df(colName).expr.dataType === StringType) + + checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - boolean") { - withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame( + (true :: false :: Nil).map(b => Tuple1.apply(Option(b))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val booleanAttr = df(colName).expr + assert(df(colName).expr.dataType === BooleanType) + + checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> booleanAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) + } } } test("filter pushdown - decimal") { - withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + withNestedDataFrame((1 to 4) + .map(i => Tuple1.apply(BigDecimal.valueOf(i))).toDF) { case (inputDF, colName, _) => + withOrcDataFrame(inputDF) { implicit df => + val decimalAttr = df(colName).expr + assert(df(colName).expr.dataType === DecimalType(38, 18)) + + checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) + } } } @@ -285,26 +338,28 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("filter pushdown - combinations with logical operators") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => checkFilterPredicate( - $"_1".isNotNull, + UnresolvedAttribute("_1").isNotNull, "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" ) checkFilterPredicate( - $"_1" =!= 1, + UnresolvedAttribute("_1") =!= 1, "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - !($"_1" < 4), + !(UnresolvedAttribute("_1") < 4), "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - $"_1" < 2 || $"_1" > 3, + UnresolvedAttribute("_1") < 2 || UnresolvedAttribute("_1") > 3, "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + "expr = (or leaf-0 (not leaf-1))" ) checkFilterPredicate( - $"_1" < 2 && $"_1" > 3, + UnresolvedAttribute("_1") < 2 && UnresolvedAttribute("_1") > 3, "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" ) @@ -343,20 +398,22 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("no filter pushdown - non-supported types") { + import testImplicits._ + implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate($"_1".isNull, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i))).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1").isNull, noneSupported = true) } // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b)).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1") <=> 1.b, noneSupported = true) } // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i))).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1").isNotNull, noneSupported = true) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala index 5fc41067f661d..141b4cfacf9ab 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation @@ -123,217 +124,298 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } test("filter pushdown - integer") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong))).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat))).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble))).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { - withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + + withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString)).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(UnresolvedAttribute("_1") === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> "1", + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= "1", + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { - withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + import testImplicits._ + + withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b))).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(UnresolvedAttribute("_1") === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> true, + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > false, + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= false, + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= false, + PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { - withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + import testImplicits._ - checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i))).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") === BigDecimal.valueOf(1), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> BigDecimal.valueOf(1), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(UnresolvedAttribute("_1") < BigDecimal.valueOf(2), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > BigDecimal.valueOf(3), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= BigDecimal.valueOf(1), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= BigDecimal.valueOf(4), + PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) + Literal(BigDecimal.valueOf(1)) === UnresolvedAttribute("_1"), PredicateLeaf.Operator.EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + Literal(BigDecimal.valueOf(1)) <=> UnresolvedAttribute("_1"), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(2)) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(3)) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(1)) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(4)) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - timestamp") { + import testImplicits._ + val timeString = "2015-08-20 14:57:00" val timestamps = (1 to 4).map { i => val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 new Timestamp(milliseconds) } - withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => - checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + withOrcDataFrame(timestamps.map(Tuple1(_)).toDF) { implicit df => + checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") === timestamps(0), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <=> timestamps(0), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> $"_1", + checkFilterPredicate(UnresolvedAttribute("_1") < timestamps(1), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(UnresolvedAttribute("_1") > timestamps(2), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") <= timestamps(0), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(UnresolvedAttribute("_1") >= timestamps(3), + PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === UnresolvedAttribute("_1"), + PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(timestamps(0)) <=> UnresolvedAttribute("_1"), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(1)) > UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= UnresolvedAttribute("_1"), + PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - combinations with logical operators") { - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + import testImplicits._ + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => // Because `ExpressionTree` is not accessible at Hive 1.2.x, this should be checked // in string form in order to check filter creation including logical operators // such as `and`, `or` or `not`. So, this function uses `SearchArgument.toString()` // to produce string expression and then compare it to given string expression below. // This might have to be changed after Hive version is upgraded. checkFilterPredicateWithDiffHiveVersion( - $"_1".isNotNull, + UnresolvedAttribute("_1").isNotNull, """leaf-0 = (IS_NULL _1) |expr = (not leaf-0)""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - $"_1" =!= 1, + UnresolvedAttribute("_1") =!= 1, """leaf-0 = (IS_NULL _1) |leaf-1 = (EQUALS _1 1) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - !($"_1" < 4), + !(UnresolvedAttribute("_1") < 4), """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 4) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - $"_1" < 2 || $"_1" > 3, + UnresolvedAttribute("_1") < 2 || UnresolvedAttribute("_1") > 3, """leaf-0 = (LESS_THAN _1 2) |leaf-1 = (LESS_THAN_EQUALS _1 3) |expr = (or leaf-0 (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - $"_1" < 2 && $"_1" > 3, + UnresolvedAttribute("_1") < 2 && UnresolvedAttribute("_1") > 3, """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 2) |leaf-2 = (LESS_THAN_EQUALS _1 3) @@ -343,27 +425,29 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } test("no filter pushdown - non-supported types") { + import testImplicits._ + implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate($"_1".isNull) + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i))).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1").isNull) } // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate($"_1" <=> 1.b) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b)).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1") <=> 1.b) } // DateType if (!HiveUtils.isHive23) { val stringDate = "2015-01-01" - withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => - checkNoFilterPredicate($"_1" === Date.valueOf(stringDate)) + withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate))).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1") === Date.valueOf(stringDate)) } } // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate($"_1".isNotNull) + withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i))).toDF) { implicit df => + checkNoFilterPredicate(UnresolvedAttribute("_1").isNotNull) } } From 148638284dbb51fa34618c818745a85647c0c02a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jun 2020 17:17:57 -0700 Subject: [PATCH 2/8] Update config doc. --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3a41b0553db54..51712786c0988 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2080,9 +2080,9 @@ object SQLConf { .doc("A comma-separated list of data source short names or fully qualified data source " + "implementation class names for which Spark tries to push down predicates for nested " + "columns and/or names containing `dots` to data sources. This configuration is only " + - "effective with file-based data source in DSv1. Currently, Parquet implements " + - "both optimizations while ORC only supports predicates for names containing `dots`. The " + - "other data sources don't support this feature yet. So the default value is 'parquet,orc'.") + "effective with file-based data source in DSv1. Currently, Parquet and ORC implement " + + "both optimizations. The other data sources don't support this feature yet. So the " + + "default value is 'parquet,orc'.") .version("3.0.0") .stringConf .createWithDefault("parquet,orc") From e12939e16cf3e503b3abe33a4e508252c3f07c75 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Jun 2020 17:35:57 -0700 Subject: [PATCH 3/8] Fix scala style. --- .../spark/sql/execution/datasources/orc/OrcFilterSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index b6d0dfea7e630..cf555489a9589 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -252,7 +252,8 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> booleanAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) <=> booleanAttr, + PredicateLeaf.Operator.NULL_SAFE_EQUALS) checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) From bd691ed16eade2e63c0fdd8d2bbd88282f6c4662 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 9 Jun 2020 23:40:42 -0700 Subject: [PATCH 4/8] Address comments. --- .../main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- .../sql/execution/datasources/orc/OrcFiltersBase.scala | 6 +++--- .../sql/execution/datasources/orc/OrcV1FilterSuite.scala | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 51712786c0988..f08aed6463a79 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2080,7 +2080,7 @@ object SQLConf { .doc("A comma-separated list of data source short names or fully qualified data source " + "implementation class names for which Spark tries to push down predicates for nested " + "columns and/or names containing `dots` to data sources. This configuration is only " + - "effective with file-based data source in DSv1. Currently, Parquet and ORC implement " + + "effective with file-based data sources in DSv1. Currently, Parquet and ORC implement " + "both optimizations. The other data sources don't support this feature yet. So the " + "default value is 'parquet,orc'.") .version("3.0.0") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala index 94af19cd428ef..24514883ed66a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala @@ -52,14 +52,14 @@ trait OrcFiltersBase { def getPrimitiveFields( fields: Seq[StructField], - parentFieldNames: Array[String] = Array.empty): Seq[(String, DataType)] = { + parentFieldNames: Seq[String] = Seq.empty): Seq[(String, DataType)] = { fields.flatMap { f => f.dataType match { case st: StructType => - getPrimitiveFields(st.fields.toSeq, parentFieldNames :+ f.name) + getPrimitiveFields(st.fields, parentFieldNames :+ f.name) case BinaryType => None case _: AtomicType => - Some(((parentFieldNames :+ f.name).toSeq.quoted, f.dataType)) + Some(((parentFieldNames :+ f.name).quoted, f.dataType)) case _ => None } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala index 72f01101e194c..d0032df488f47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution.datasources.orc import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.{Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} import org.apache.spark.sql.catalyst.planning.PhysicalOperation From e76b5f4491b5de95362f7acf8efa02c07d154b81 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sun, 28 Jun 2020 22:27:18 -0700 Subject: [PATCH 5/8] Revert unnecessary change. --- .../sql/execution/datasources/orc/OrcFilterSuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index cf555489a9589..495a444d4b5a0 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -75,9 +75,9 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } protected def checkFilterPredicate - (predicate: Predicate, filterOperator: PredicateLeaf.Operator) - (implicit df: DataFrame): Unit = { - def checkComparisonOperator(filter: SearchArgument): Unit = { + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { val operator = filter.getLeaves.asScala assert(operator.map(_.getOperator).contains(filterOperator)) } @@ -87,7 +87,7 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS protected def checkFilterPredicate (predicate: Predicate, stringExpr: String) (implicit df: DataFrame): Unit = { - def checkLogicalOperator(filter: SearchArgument): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { assert(filter.toString == stringExpr) } checkFilterPredicate(df, predicate, checkLogicalOperator) From 0747fcdef1ffdba5b8ce3cbafdc03cac3559f7d4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 30 Jul 2020 09:16:32 -0700 Subject: [PATCH 6/8] Simplify tests. --- .../datasources/FileBasedDataSourceTest.scala | 40 +- .../NestedColumnPredicateTest.scala | 70 ---- .../execution/datasources/orc/OrcTest.scala | 34 +- .../parquet/ParquetFilterSuite.scala | 34 +- .../datasources/orc/OrcFilterSuite.scala | 375 +++++++++-------- .../datasources/orc/OrcFilterSuite.scala | 382 +++++++++--------- .../sql/hive/orc/HiveOrcFilterSuite.scala | 374 +++++++---------- 7 files changed, 578 insertions(+), 731 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala index bdb161d59a33e..c2dc20b0099a3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala @@ -22,8 +22,10 @@ import java.io.File import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.{DataFrame, Row, SaveMode} +import org.apache.spark.sql.functions.struct import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.types.StructType /** * A helper trait that provides convenient facilities for file-based data source testing. @@ -103,4 +105,40 @@ private[sql] trait FileBasedDataSourceTest extends SQLTestUtils { df: DataFrame, path: File): Unit = { df.write.mode(SaveMode.Overwrite).format(dataSourceName).save(path.getCanonicalPath) } + + /** + * Takes single level `inputDF` dataframe to generate multi-level nested + * dataframes as new test data. It tests both non-nested and nested dataframes + * which are written and read back with specified datasource. + */ + protected def withNestedDataFrame(inputDF: DataFrame): Seq[(DataFrame, String, Any => Any)] = { + assert(inputDF.schema.fields.length == 1) + assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) + val df = inputDF.toDF("temp") + Seq( + ( + df.withColumnRenamed("temp", "a"), + "a", // zero nesting + (x: Any) => x), + ( + df.withColumn("a", struct(df("temp") as "b")).drop("temp"), + "a.b", // one level nesting + (x: Any) => Row(x)), + ( + df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"), + "a.b.c", // two level nesting + (x: Any) => Row(Row(x)) + ), + ( + df.withColumnRenamed("temp", "a.b"), + "`a.b`", // zero nesting with column name containing `dots` + (x: Any) => x + ), + ( + df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"), + "`a.b`.`c.d`", // one level nesting with column names containing `dots` + (x: Any) => Row(x) + ) + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala deleted file mode 100644 index e7d0b76cb1701..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/NestedColumnPredicateTest.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import org.apache.spark.sql.{DataFrame, Row} -import org.apache.spark.sql.functions.struct -import org.apache.spark.sql.types.StructType - -/** - * Defines common stuff for nested column predicate pushdown test, e.g. `ParquetFilterSuite`. - */ -trait NestedColumnPredicateTest { - /** - * Takes single level `inputDF` dataframe to generate multi-level nested - * dataframes as new test data. - * - * This method accepts a function to run test. The given function takes three - * parameters: a DataFrame which ranges from zero-nested to multi-level nested, - * a string of the primitive field name, and a function that produces expected - * result of collected column. - */ - protected def withNestedDataFrame(inputDF: DataFrame) - (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { - assert(inputDF.schema.fields.length == 1) - assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) - val df = inputDF.toDF("temp") - Seq( - ( - df.withColumnRenamed("temp", "a"), - "a", // zero nesting - (x: Any) => x), - ( - df.withColumn("a", struct(df("temp") as "b")).drop("temp"), - "a.b", // one level nesting - (x: Any) => Row(x)), - ( - df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"), - "a.b.c", // two level nesting - (x: Any) => Row(Row(x)) - ), - ( - df.withColumnRenamed("temp", "a.b"), - "`a.b`", // zero nesting with column name containing `dots` - (x: Any) => x - ), - ( - df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"), - "`a.b`.`c.d`", // one level nesting with column names containing `dots` - (x: Any) => Row(x) - ) - ).foreach { case (df, colName, resultFun) => - runTest(df, colName, resultFun) - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index f0356d9302f25..d409fad9cd791 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -78,16 +78,12 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor (f: String => Unit): Unit = withDataSourceFile(data)(f) /** - * Writes `df` dataframe to a Orc file and reads it back as a `DataFrame`, + * Writes `date` dataframe to a Orc file and reads it back as a `DataFrame`, * which is then passed to `f`. The Orc file will be deleted after `f` returns. */ - protected def withOrcDataFrame(df: DataFrame, testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = { - withTempPath { file => - df.write.format(dataSourceName).save(file.getCanonicalPath) - readFile(file.getCanonicalPath, testVectorized)(f) - } - } + protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag] + (data: Seq[T], testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f) /** * Writes `data` to a Orc file, reads it back as a `DataFrame` and registers it as a @@ -147,4 +143,26 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor FileUtils.copyURLToFile(url, file) spark.read.orc(file.getAbsolutePath) } + + /** + * Takes a sequence of products `data` to generate multi-level nested + * dataframes as new test data. It tests both non-nested and nested dataframes + * which are written and read back with Orc datasource. + * + * This is different from [[OrcTest.withOrcDataFrame]] which does not + * test nested cases. + */ + protected def withNestedOrcDataFrame[T <: Product: ClassTag: TypeTag](data: Seq[T]) + (runTest: (DataFrame, String, Any => Any) => Unit): Unit = + withNestedOrcDataFrame(spark.createDataFrame(data))(runTest) + + protected def withNestedOrcDataFrame(inputDF: DataFrame) + (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { + withNestedDataFrame(inputDF).foreach { case (newDF, colName, resultFun) => + withTempPath { file => + newDF.write.format(dataSourceName).save(file.getCanonicalPath) + readFile(file.getCanonicalPath, true) { df => runTest(df, colName, resultFun) } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala index c9bd525ad3af1..5689b9d05d7bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFilterSuite.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath -import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, NestedColumnPredicateTest, PushableColumnAndNestedColumn} +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation, PushableColumnAndNestedColumn} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions._ @@ -65,8 +65,7 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} * dependent on this configuration, don't forget you better explicitly set this configuration * within the test. */ -abstract class ParquetFilterSuite - extends QueryTest with ParquetTest with NestedColumnPredicateTest with SharedSparkSession { +abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSparkSession { protected def createParquetFilters( schema: MessageType, @@ -123,34 +122,7 @@ abstract class ParquetFilterSuite private def withNestedParquetDataFrame(inputDF: DataFrame) (runTest: (DataFrame, String, Any => Any) => Unit): Unit = { - assert(inputDF.schema.fields.length == 1) - assert(!inputDF.schema.fields.head.dataType.isInstanceOf[StructType]) - val df = inputDF.toDF("temp") - Seq( - ( - df.withColumnRenamed("temp", "a"), - "a", // zero nesting - (x: Any) => x), - ( - df.withColumn("a", struct(df("temp") as "b")).drop("temp"), - "a.b", // one level nesting - (x: Any) => Row(x)), - ( - df.withColumn("a", struct(struct(df("temp") as "c") as "b")).drop("temp"), - "a.b.c", // two level nesting - (x: Any) => Row(Row(x)) - ), - ( - df.withColumnRenamed("temp", "a.b"), - "`a.b`", // zero nesting with column name containing `dots` - (x: Any) => x - ), - ( - df.withColumn("a.b", struct(df("temp") as "c.d") ).drop("temp"), - "`a.b`.`c.d`", // one level nesting with column names containing `dots` - (x: Any) => Row(x) - ) - ).foreach { case (newDF, colName, resultFun) => + withNestedDataFrame(inputDF).foreach { case (newDF, colName, resultFun) => withTempPath { file => newDF.write.format(dataSourceName).save(file.getCanonicalPath) readParquetFile(file.getCanonicalPath) { df => runTest(df, colName, resultFun) } diff --git a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index e4392c04940f4..2643196cac561 100644 --- a/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -27,11 +27,9 @@ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.spark.SparkConf import org.apache.spark.sql.{AnalysisException, Column, DataFrame} -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.NestedColumnPredicateTest import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf @@ -45,7 +43,7 @@ import org.apache.spark.sql.types._ * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ -class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedSparkSession { +class OrcFilterSuite extends OrcTest with SharedSparkSession { override protected def sparkConf: SparkConf = super @@ -94,206 +92,199 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } test("filter pushdown - integer") { - import testImplicits._ - withNestedDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val intAttr = df(colName).expr - assert(df(colName).expr.dataType === IntegerType) - - checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val intAttr = df(colName).expr + assert(df(colName).expr.dataType === IntegerType) + + checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toLong))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val longAttr = df(colName).expr - assert(df(colName).expr.dataType === LongType) - - checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toLong)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val longAttr = df(colName).expr + assert(df(colName).expr.dataType === LongType) + + checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toFloat))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val floatAttr = df(colName).expr - assert(df(colName).expr.dataType === FloatType) - - checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toFloat)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val floatAttr = df(colName).expr + assert(df(colName).expr.dataType === FloatType) + + checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toDouble))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val doubleAttr = df(colName).expr - assert(df(colName).expr.dataType === DoubleType) - - checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toDouble)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val doubleAttr = df(colName).expr + assert(df(colName).expr.dataType === DoubleType) + + checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { - import testImplicits._ - withNestedDataFrame((1 to 4).map(i => Tuple1(i.toString)).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val strAttr = df(colName).expr - assert(df(colName).expr.dataType === StringType) - - checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val strAttr = df(colName).expr + assert(df(colName).expr.dataType === StringType) + + checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { - import testImplicits._ - withNestedDataFrame( - (true :: false :: Nil).map(b => Tuple1.apply(Option(b))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val booleanAttr = df(colName).expr - assert(df(colName).expr.dataType === BooleanType) - - checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> booleanAttr, - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val booleanAttr = df(colName).expr + assert(df(colName).expr.dataType === BooleanType) + + checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> booleanAttr, + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val decimalAttr = df(colName).expr - assert(df(colName).expr.dataType === DecimalType(38, 18)) - - checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val decimalAttr = df(colName).expr + assert(df(colName).expr.dataType === DecimalType(38, 18)) + + checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) } } @@ -337,28 +328,26 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } test("filter pushdown - combinations with logical operators") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate( - UnresolvedAttribute("_1").isNotNull, + $"_1".isNotNull, "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" ) checkFilterPredicate( - UnresolvedAttribute("_1") =!= 1, + $"_1" =!= 1, "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - !(UnresolvedAttribute("_1") < 4), + !($"_1" < 4), "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - UnresolvedAttribute("_1") < 2 || UnresolvedAttribute("_1") > 3, + $"_1" < 2 || $"_1" > 3, "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + "expr = (or leaf-0 (not leaf-1))" ) checkFilterPredicate( - UnresolvedAttribute("_1") < 2 && UnresolvedAttribute("_1") > 3, + $"_1" < 2 && $"_1" > 3, "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" ) @@ -397,22 +386,20 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } test("no filter pushdown - non-supported types") { - import testImplicits._ - implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i))).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1").isNull, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => + checkNoFilterPredicate($"_1".isNull, noneSupported = true) } // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b)).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1") <=> 1.b, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => + checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1").isNotNull, noneSupported = true) + checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) } } diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 495a444d4b5a0..86120dd569820 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -26,12 +26,10 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.spark.SparkConf -import org.apache.spark.sql.{AnalysisException, Column, DataFrame, Row} -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.execution.datasources.NestedColumnPredicateTest import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf @@ -45,7 +43,7 @@ import org.apache.spark.sql.types._ * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. */ -class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedSparkSession { +class OrcFilterSuite extends OrcTest with SharedSparkSession { override protected def sparkConf: SparkConf = super @@ -94,208 +92,200 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } test("filter pushdown - integer") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i))).toDF("i")) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val intAttr = df(colName).expr - assert(df(colName).expr.dataType === IntegerType) - - checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val intAttr = df(colName).expr + assert(df(colName).expr.dataType === IntegerType) + + checkFilterPredicate(intAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(intAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(intAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(intAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(intAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(intAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === intAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> intAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > intAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= intAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= intAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toLong))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val longAttr = df(colName).expr - assert(df(colName).expr.dataType === LongType) - - checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toLong)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val longAttr = df(colName).expr + assert(df(colName).expr.dataType === LongType) + + checkFilterPredicate(longAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(longAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(longAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(longAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(longAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(longAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === longAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> longAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > longAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= longAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= longAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toFloat))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val floatAttr = df(colName).expr - assert(df(colName).expr.dataType === FloatType) - - checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toFloat)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val floatAttr = df(colName).expr + assert(df(colName).expr.dataType === FloatType) + + checkFilterPredicate(floatAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(floatAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(floatAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(floatAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(floatAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(floatAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === floatAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> floatAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > floatAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= floatAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= floatAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toDouble))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val doubleAttr = df(colName).expr - assert(df(colName).expr.dataType === DoubleType) - - checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(Option(i.toDouble)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val doubleAttr = df(colName).expr + assert(df(colName).expr.dataType === DoubleType) + + checkFilterPredicate(doubleAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(doubleAttr === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(doubleAttr <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(doubleAttr < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(doubleAttr > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(doubleAttr >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === doubleAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> doubleAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > doubleAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= doubleAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= doubleAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { - import testImplicits._ - withNestedDataFrame( - (1 to 4).map(i => Tuple1(i.toString)).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val strAttr = df(colName).expr - assert(df(colName).expr.dataType === StringType) - - checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1(i.toString))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val strAttr = df(colName).expr + assert(df(colName).expr.dataType === StringType) + + checkFilterPredicate(strAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(strAttr === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(strAttr <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(strAttr < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(strAttr > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(strAttr >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === strAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> strAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > strAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= strAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= strAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { - import testImplicits._ - withNestedDataFrame( - (true :: false :: Nil).map(b => Tuple1.apply(Option(b))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val booleanAttr = df(colName).expr - assert(df(colName).expr.dataType === BooleanType) - - checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> booleanAttr, - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame( + (true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val booleanAttr = df(colName).expr + assert(df(colName).expr.dataType === BooleanType) + + checkFilterPredicate(booleanAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(booleanAttr === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(booleanAttr <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(booleanAttr < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(booleanAttr > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(booleanAttr >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === booleanAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> booleanAttr, + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > booleanAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= booleanAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= booleanAttr, PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { - import testImplicits._ - withNestedDataFrame((1 to 4) - .map(i => Tuple1.apply(BigDecimal.valueOf(i))).toDF) { case (inputDF, colName, _) => - withOrcDataFrame(inputDF) { implicit df => - val decimalAttr = df(colName).expr - assert(df(colName).expr.dataType === DecimalType(38, 18)) - - checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) - } + withNestedOrcDataFrame((1 to 4) + .map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { case (inputDF, colName, _) => + implicit val df: DataFrame = inputDF + + val decimalAttr = df(colName).expr + assert(df(colName).expr.dataType === DecimalType(38, 18)) + + checkFilterPredicate(decimalAttr.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate(decimalAttr === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(decimalAttr <=> BigDecimal.valueOf(1), + PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate(decimalAttr < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(decimalAttr > BigDecimal.valueOf(3), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr <= BigDecimal.valueOf(1), + PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(decimalAttr >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) === decimalAttr, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) <=> decimalAttr, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(2)) > decimalAttr, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(BigDecimal.valueOf(3)) < decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) >= decimalAttr, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(4)) <= decimalAttr, PredicateLeaf.Operator.LESS_THAN) } } @@ -339,28 +329,26 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } test("filter pushdown - combinations with logical operators") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate( - UnresolvedAttribute("_1").isNotNull, + $"_1".isNotNull, "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" ) checkFilterPredicate( - UnresolvedAttribute("_1") =!= 1, + $"_1" =!= 1, "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - !(UnresolvedAttribute("_1") < 4), + !($"_1" < 4), "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" ) checkFilterPredicate( - UnresolvedAttribute("_1") < 2 || UnresolvedAttribute("_1") > 3, + $"_1" < 2 || $"_1" > 3, "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + "expr = (or leaf-0 (not leaf-1))" ) checkFilterPredicate( - UnresolvedAttribute("_1") < 2 && UnresolvedAttribute("_1") > 3, + $"_1" < 2 && $"_1" > 3, "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" ) @@ -399,22 +387,20 @@ class OrcFilterSuite extends OrcTest with NestedColumnPredicateTest with SharedS } test("no filter pushdown - non-supported types") { - import testImplicits._ - implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i))).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1").isNull, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => + checkNoFilterPredicate($"_1".isNull, noneSupported = true) } // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b)).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1") <=> 1.b, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => + checkNoFilterPredicate($"_1" <=> 1.b, noneSupported = true) } // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i))).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1").isNotNull, noneSupported = true) + withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => + checkNoFilterPredicate($"_1".isNotNull, noneSupported = true) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala index 141b4cfacf9ab..5fc41067f661d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala @@ -25,7 +25,6 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.spark.sql.{Column, DataFrame} -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation @@ -124,298 +123,217 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } test("filter pushdown - integer") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - long") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong))).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - float") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat))).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - double") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble))).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(UnresolvedAttribute("_1") === 1, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < 2, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= 4, PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(1) === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(1) <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(2) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(3) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(1) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(4) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - string") { - import testImplicits._ - - withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString)).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(UnresolvedAttribute("_1") === "1", PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> "1", - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < "2", PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= "1", - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= "4", PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal("1") === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal("1") <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal("2") > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal("3") < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("1") >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal("4") <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - boolean") { - import testImplicits._ - - withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b))).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) - - checkFilterPredicate(UnresolvedAttribute("_1") === true, PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> true, - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < true, PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > false, - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= false, - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= false, - PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(false) === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(false) <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(false) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(true) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(true) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate($"_1" === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate($"_1" < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - decimal") { - import testImplicits._ + withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i))).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + checkFilterPredicate($"_1" === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") === BigDecimal.valueOf(1), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> BigDecimal.valueOf(1), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) - - checkFilterPredicate(UnresolvedAttribute("_1") < BigDecimal.valueOf(2), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > BigDecimal.valueOf(3), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= BigDecimal.valueOf(1), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= BigDecimal.valueOf(4), - PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) === UnresolvedAttribute("_1"), PredicateLeaf.Operator.EQUALS) + Literal(BigDecimal.valueOf(1)) === $"_1", PredicateLeaf.Operator.EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) <=> UnresolvedAttribute("_1"), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) + Literal(BigDecimal.valueOf(1)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(2)) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(2)) > $"_1", PredicateLeaf.Operator.LESS_THAN) checkFilterPredicate( - Literal(BigDecimal.valueOf(3)) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(3)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(1)) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) + Literal(BigDecimal.valueOf(1)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) checkFilterPredicate( - Literal(BigDecimal.valueOf(4)) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + Literal(BigDecimal.valueOf(4)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - timestamp") { - import testImplicits._ - val timeString = "2015-08-20 14:57:00" val timestamps = (1 to 4).map { i => val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 new Timestamp(milliseconds) } - withOrcDataFrame(timestamps.map(Tuple1(_)).toDF) { implicit df => - checkFilterPredicate(UnresolvedAttribute("_1").isNull, PredicateLeaf.Operator.IS_NULL) + withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => + checkFilterPredicate($"_1".isNull, PredicateLeaf.Operator.IS_NULL) - checkFilterPredicate(UnresolvedAttribute("_1") === timestamps(0), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <=> timestamps(0), - PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate($"_1" === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate($"_1" <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") < timestamps(1), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(UnresolvedAttribute("_1") > timestamps(2), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") <= timestamps(0), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(UnresolvedAttribute("_1") >= timestamps(3), - PredicateLeaf.Operator.LESS_THAN) - - checkFilterPredicate(Literal(timestamps(0)) === UnresolvedAttribute("_1"), - PredicateLeaf.Operator.EQUALS) - checkFilterPredicate(Literal(timestamps(0)) <=> UnresolvedAttribute("_1"), + checkFilterPredicate($"_1" < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate($"_1" > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate($"_1" >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === $"_1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(timestamps(0)) <=> $"_1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) - checkFilterPredicate(Literal(timestamps(1)) > UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) - checkFilterPredicate(Literal(timestamps(2)) < UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(0)) >= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN_EQUALS) - checkFilterPredicate(Literal(timestamps(3)) <= UnresolvedAttribute("_1"), - PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(1)) > $"_1", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= $"_1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= $"_1", PredicateLeaf.Operator.LESS_THAN) } } test("filter pushdown - combinations with logical operators") { - import testImplicits._ - withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i))).toDF) { implicit df => + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => // Because `ExpressionTree` is not accessible at Hive 1.2.x, this should be checked // in string form in order to check filter creation including logical operators // such as `and`, `or` or `not`. So, this function uses `SearchArgument.toString()` // to produce string expression and then compare it to given string expression below. // This might have to be changed after Hive version is upgraded. checkFilterPredicateWithDiffHiveVersion( - UnresolvedAttribute("_1").isNotNull, + $"_1".isNotNull, """leaf-0 = (IS_NULL _1) |expr = (not leaf-0)""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - UnresolvedAttribute("_1") =!= 1, + $"_1" =!= 1, """leaf-0 = (IS_NULL _1) |leaf-1 = (EQUALS _1 1) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - !(UnresolvedAttribute("_1") < 4), + !($"_1" < 4), """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 4) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - UnresolvedAttribute("_1") < 2 || UnresolvedAttribute("_1") > 3, + $"_1" < 2 || $"_1" > 3, """leaf-0 = (LESS_THAN _1 2) |leaf-1 = (LESS_THAN_EQUALS _1 3) |expr = (or leaf-0 (not leaf-1))""".stripMargin.trim ) checkFilterPredicateWithDiffHiveVersion( - UnresolvedAttribute("_1") < 2 && UnresolvedAttribute("_1") > 3, + $"_1" < 2 && $"_1" > 3, """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 2) |leaf-2 = (LESS_THAN_EQUALS _1 3) @@ -425,29 +343,27 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } test("no filter pushdown - non-supported types") { - import testImplicits._ - implicit class IntToBinary(int: Int) { def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) } // ArrayType - withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i))).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1").isNull) + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => + checkNoFilterPredicate($"_1".isNull) } // BinaryType - withOrcDataFrame((1 to 4).map(i => Tuple1(i.b)).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1") <=> 1.b) + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => + checkNoFilterPredicate($"_1" <=> 1.b) } // DateType if (!HiveUtils.isHive23) { val stringDate = "2015-01-01" - withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate))).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1") === Date.valueOf(stringDate)) + withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => + checkNoFilterPredicate($"_1" === Date.valueOf(stringDate)) } } // MapType - withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i))).toDF) { implicit df => - checkNoFilterPredicate(UnresolvedAttribute("_1").isNotNull) + withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => + checkNoFilterPredicate($"_1".isNotNull) } } From 558db4644446ad9986e825ea5aded7c29ca7ba0b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 6 Aug 2020 22:50:36 -0700 Subject: [PATCH 7/8] Address comments. --- .../execution/datasources/orc/OrcFiltersBase.scala | 11 ++++++----- .../execution/datasources/v2/orc/OrcScanBuilder.scala | 2 +- .../spark/sql/execution/datasources/orc/OrcTest.scala | 4 ++-- .../sql/execution/datasources/orc/OrcFilters.scala | 4 +--- .../sql/execution/datasources/orc/OrcFilters.scala | 4 +--- .../execution/datasources/orc/OrcFilterSuite.scala | 6 +++--- 6 files changed, 14 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala index 24514883ed66a..b4903cb2e1cd8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala @@ -43,9 +43,11 @@ trait OrcFiltersBase { * This method returns a map which contains ORC field name and data type. Each key * represents a column; `dots` are used as separators for nested columns. If any part * of the names contains `dots`, it is quoted to avoid confusion. See - * `org.apache.spark.sql.connector.catalog.quote` for implementation details. + * `org.apache.spark.sql.connector.catalog.quoted` for implementation details. + * + * BinaryType, UserDefinedType, ArrayType and MapType are ignored. */ - protected[sql] def getNameToOrcFieldMap( + protected[sql] def isSearchableType( schema: StructType, caseSensitive: Boolean): Map[String, DataType] = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper @@ -69,11 +71,10 @@ trait OrcFiltersBase { if (caseSensitive) { primitiveFields.toMap } else { - // Don't consider ambiguity here, i.e. more than one field is matched in case insensitive + // Don't consider ambiguity here, i.e. more than one field are matched in case insensitive // mode, just skip pushdown for these fields, they will trigger Exception when reading, // See: SPARK-25175. - val dedupPrimitiveFields = - primitiveFields + val dedupPrimitiveFields = primitiveFields .groupBy(_._1.toLowerCase(Locale.ROOT)) .filter(_._2.size == 1) .mapValues(_.head._2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 6839ee19a5fa2..61d2b3560434d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -61,7 +61,7 @@ case class OrcScanBuilder( // changed `hadoopConf` in executors. OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } - val dataTypeMap = OrcFilters.getNameToOrcFieldMap(schema, SQLConf.get.caseSensitiveAnalysis) + val dataTypeMap = OrcFilters.isSearchableType(schema, SQLConf.get.caseSensitiveAnalysis) _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, filters).toArray } filters diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index d409fad9cd791..aec61acda5444 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -78,7 +78,7 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor (f: String => Unit): Unit = withDataSourceFile(data)(f) /** - * Writes `date` dataframe to a Orc file and reads it back as a `DataFrame`, + * Writes `data` to a Orc file and reads it back as a `DataFrame`, * which is then passed to `f`. The Orc file will be deleted after `f` returns. */ protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag] @@ -149,7 +149,7 @@ abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with Befor * dataframes as new test data. It tests both non-nested and nested dataframes * which are written and read back with Orc datasource. * - * This is different from [[OrcTest.withOrcDataFrame]] which does not + * This is different from [[withOrcDataFrame]] which does not * test nested cases. */ protected def withNestedOrcDataFrame[T <: Product: ClassTag: TypeTag](data: Seq[T]) diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 5c912ea384945..6ef7c82abf0e5 100644 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -68,7 +68,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = OrcFilters.getNameToOrcFieldMap(schema, SQLConf.get.caseSensitiveAnalysis) + val dataTypeMap = OrcFilters.isSearchableType(schema, SQLConf.get.caseSensitiveAnalysis) // Combines all convertible filters using `And` to produce a single conjunction val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) conjunctionOptional.map { conjunction => @@ -226,8 +226,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters - // in order to distinguish predicate pushdown for nested columns. expression match { case EqualTo(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 8f48f0aa05688..39bdbd0320db8 100644 --- a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -68,7 +68,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = OrcFilters.getNameToOrcFieldMap(schema, SQLConf.get.caseSensitiveAnalysis) + val dataTypeMap = OrcFilters.isSearchableType(schema, SQLConf.get.caseSensitiveAnalysis) // Combines all convertible filters using `And` to produce a single conjunction val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) conjunctionOptional.map { conjunction => @@ -226,8 +226,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters - // in order to distinguish predicate pushdown for nested columns. expression match { case EqualTo(name, value) if dataTypeMap.contains(name) => val castedValue = castLiteralValue(value, dataTypeMap(name)) diff --git a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index 86120dd569820..7df9f29b42f63 100644 --- a/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v2.3/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -173,7 +173,7 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { test("filter pushdown - double") { withNestedOrcDataFrame( - (1 to 4).map(i => Tuple1(Option(i.toDouble)))) { case (inputDF, colName, _) => + (1 to 4).map(i => Tuple1(Option(i.toDouble)))) { case (inputDF, colName, _) => implicit val df: DataFrame = inputDF val doubleAttr = df(colName).expr @@ -254,8 +254,8 @@ class OrcFilterSuite extends OrcTest with SharedSparkSession { } test("filter pushdown - decimal") { - withNestedOrcDataFrame((1 to 4) - .map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { case (inputDF, colName, _) => + withNestedOrcDataFrame( + (1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { case (inputDF, colName, _) => implicit val df: DataFrame = inputDF val decimalAttr = df(colName).expr From dc772907844be6fe1b886b45e3e9170fee7793f2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 6 Aug 2020 23:20:27 -0700 Subject: [PATCH 8/8] Rename. --- .../spark/sql/execution/datasources/orc/OrcFiltersBase.scala | 2 +- .../spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala | 2 +- .../apache/spark/sql/execution/datasources/orc/OrcFilters.scala | 2 +- .../apache/spark/sql/execution/datasources/orc/OrcFilters.scala | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala index b4903cb2e1cd8..b277b4da1cf8e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala @@ -47,7 +47,7 @@ trait OrcFiltersBase { * * BinaryType, UserDefinedType, ArrayType and MapType are ignored. */ - protected[sql] def isSearchableType( + protected[sql] def getSearchableTypeMap( schema: StructType, caseSensitive: Boolean): Map[String, DataType] = { import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.MultipartIdentifierHelper diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala index 61d2b3560434d..0330dacffa587 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.scala @@ -61,7 +61,7 @@ case class OrcScanBuilder( // changed `hadoopConf` in executors. OrcInputFormat.setSearchArgument(hadoopConf, f, schema.fieldNames) } - val dataTypeMap = OrcFilters.isSearchableType(schema, SQLConf.get.caseSensitiveAnalysis) + val dataTypeMap = OrcFilters.getSearchableTypeMap(schema, SQLConf.get.caseSensitiveAnalysis) _pushedFilters = OrcFilters.convertibleFilters(schema, dataTypeMap, filters).toArray } filters diff --git a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6ef7c82abf0e5..bc11bb8c1d5d3 100644 --- a/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -68,7 +68,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = OrcFilters.isSearchableType(schema, SQLConf.get.caseSensitiveAnalysis) + val dataTypeMap = OrcFilters.getSearchableTypeMap(schema, SQLConf.get.caseSensitiveAnalysis) // Combines all convertible filters using `And` to produce a single conjunction val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) conjunctionOptional.map { conjunction => diff --git a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 39bdbd0320db8..5273245fae45c 100644 --- a/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -68,7 +68,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { * Create ORC filter as a SearchArgument instance. */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { - val dataTypeMap = OrcFilters.isSearchableType(schema, SQLConf.get.caseSensitiveAnalysis) + val dataTypeMap = OrcFilters.getSearchableTypeMap(schema, SQLConf.get.caseSensitiveAnalysis) // Combines all convertible filters using `And` to produce a single conjunction val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) conjunctionOptional.map { conjunction =>