From 2527fee5dce3e45ab0f49a203f5bcb3b01a2f2b6 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 3 Jan 2019 13:53:26 +0200 Subject: [PATCH 01/56] WIP: optimize ORC predicate conversion --- .../benchmark/FilterPushdownBenchmark.scala | 384 ++++++++++-------- .../datasources/orc/OrcFilters.scala | 112 ++++- 2 files changed, 315 insertions(+), 181 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index b04024371713..13366c6a9619 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -24,9 +24,9 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.internal.config.UI._ -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.functions.monotonically_increasing_id +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType} @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType * Results will be written to "benchmarks/FilterPushdownBenchmark-results.txt". * }}} */ -object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { + object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { private val conf = new SparkConf() .setAppName(this.getClass.getSimpleName) @@ -125,7 +125,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { Seq(false, true).foreach { pushDownEnabled => val name = s"Native ORC Vectorized ${if (pushDownEnabled) s"(Pushdown)" else ""}" - benchmark.addCase(name) { _ => + benchmark.addCase(name) { _ => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> s"$pushDownEnabled") { spark.sql(s"SELECT $selectExpr FROM orcTable WHERE $whereExpr").collect() } @@ -135,6 +135,25 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { benchmark.run() } + def filterPushDownBenchmarkWithColumn( + values: Int, + title: String, + whereColumn: Column, + selectExpr: String = "*" + ): Unit = { + val benchmark = new Benchmark(title, values, minNumIters = 5, output = output) + val name = s"Native ORC Vectorized (Pushdown)" + benchmark.addCase(name) { _ => + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + spark + .sql(s"SELECT $selectExpr FROM orcTable") + .filter(whereColumn) + .collect() + } + } + benchmark.run() + } + private def runIntBenchmark(numRows: Int, width: Int, mid: Int): Unit = { Seq("value IS NULL", s"$mid < value AND value < $mid").foreach { whereExpr => val title = s"Select 0 int row ($whereExpr)".replace("value AND value", "value") @@ -200,180 +219,205 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { - runBenchmark("Pushdown for many distinct value case") { - withTempPath { dir => - withTempTable("orcTable", "parquetTable") { - Seq(true, false).foreach { useStringForValue => - prepareTable(dir, numRows, width, useStringForValue) - if (useStringForValue) { - runStringBenchmark(numRows, width, mid, "string") - } else { - runIntBenchmark(numRows, width, mid) - } - } - } - } - } - - runBenchmark("Pushdown for few distinct value case (use dictionary encoding)") { - withTempPath { dir => - val numDistinctValues = 200 - - withTempTable("orcTable", "parquetTable") { - prepareStringDictTable(dir, numRows, numDistinctValues, width) - runStringBenchmark(numRows, width, numDistinctValues / 2, "distinct string") - } - } - } - - runBenchmark("Pushdown benchmark for StringStartsWith") { - withTempPath { dir => - withTempTable("orcTable", "parquetTable") { - prepareTable(dir, numRows, width, true) - Seq( - "value like '10%'", - "value like '1000%'", - s"value like '${mid.toString.substring(0, mid.toString.length - 1)}%'" - ).foreach { whereExpr => - val title = s"StringStartsWith filter: ($whereExpr)" - filterPushDownBenchmark(numRows, title, whereExpr) - } - } - } - } - - runBenchmark(s"Pushdown benchmark for ${DecimalType.simpleString}") { - withTempPath { dir => - Seq( - s"decimal(${Decimal.MAX_INT_DIGITS}, 2)", - s"decimal(${Decimal.MAX_LONG_DIGITS}, 2)", - s"decimal(${DecimalType.MAX_PRECISION}, 2)" - ).foreach { dt => - val columns = (1 to width).map(i => s"CAST(id AS string) c$i") - val valueCol = if (dt.equalsIgnoreCase(s"decimal(${Decimal.MAX_INT_DIGITS}, 2)")) { - monotonically_increasing_id() % 9999999 - } else { - monotonically_increasing_id() - } - val df = spark.range(numRows) - .selectExpr(columns: _*).withColumn("value", valueCol.cast(dt)) - withTempTable("orcTable", "parquetTable") { - saveAsTable(df, dir) - - Seq(s"value = $mid").foreach { whereExpr => - val title = s"Select 1 $dt row ($whereExpr)".replace("value AND value", "value") - filterPushDownBenchmark(numRows, title, whereExpr) - } - - val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") - Seq(10, 50, 90).foreach { percent => - filterPushDownBenchmark( - numRows, - s"Select $percent% $dt rows (value < ${numRows * percent / 100})", - s"value < ${numRows * percent / 100}", - selectExpr - ) - } - } - } - } - } - - runBenchmark("Pushdown benchmark for InSet -> InFilters") { - withTempPath { dir => - withTempTable("orcTable", "parquetTable") { - prepareTable(dir, numRows, width, false) - Seq(5, 10, 50, 100).foreach { count => - Seq(10, 50, 90).foreach { distribution => - val filter = - Range(0, count).map(r => scala.util.Random.nextInt(numRows * distribution / 100)) - val whereExpr = s"value in(${filter.mkString(",")})" - val title = s"InSet -> InFilters (values count: $count, distribution: $distribution)" - filterPushDownBenchmark(numRows, title, whereExpr) - } - } - } - } - } - - runBenchmark(s"Pushdown benchmark for ${ByteType.simpleString}") { - withTempPath { dir => - val columns = (1 to width).map(i => s"CAST(id AS string) c$i") - val df = spark.range(numRows).selectExpr(columns: _*) - .withColumn("value", (monotonically_increasing_id() % Byte.MaxValue).cast(ByteType)) - .orderBy("value") - withTempTable("orcTable", "parquetTable") { - saveAsTable(df, dir) - - Seq(s"value = CAST(${Byte.MaxValue / 2} AS ${ByteType.simpleString})") - .foreach { whereExpr => - val title = s"Select 1 ${ByteType.simpleString} row ($whereExpr)" - .replace("value AND value", "value") - filterPushDownBenchmark(numRows, title, whereExpr) - } - - val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") - Seq(10, 50, 90).foreach { percent => - filterPushDownBenchmark( - numRows, - s"Select $percent% ${ByteType.simpleString} rows " + - s"(value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString}))", - s"value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString})", - selectExpr - ) - } - } - } - } - - runBenchmark(s"Pushdown benchmark for Timestamp") { - withTempPath { dir => - withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> true.toString) { - ParquetOutputTimestampType.values.toSeq.map(_.toString).foreach { fileType => - withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fileType) { - val columns = (1 to width).map(i => s"CAST(id AS string) c$i") - val df = spark.range(numRows).selectExpr(columns: _*) - .withColumn("value", monotonically_increasing_id().cast(TimestampType)) - withTempTable("orcTable", "parquetTable") { - saveAsTable(df, dir) - - Seq(s"value = CAST($mid AS timestamp)").foreach { whereExpr => - val title = s"Select 1 timestamp stored as $fileType row ($whereExpr)" - .replace("value AND value", "value") - filterPushDownBenchmark(numRows, title, whereExpr) - } - - val selectExpr = (1 to width) - .map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") - Seq(10, 50, 90).foreach { percent => - filterPushDownBenchmark( - numRows, - s"Select $percent% timestamp stored as $fileType rows " + - s"(value < CAST(${numRows * percent / 100} AS timestamp))", - s"value < CAST(${numRows * percent / 100} as timestamp)", - selectExpr - ) - } - } - } - } - } - } - } - - runBenchmark(s"Pushdown benchmark with many filters") { +// runBenchmark("Pushdown for many distinct value case") { +// withTempPath { dir => +// withTempTable("orcTable", "parquetTable") { +// Seq(true, false).foreach { useStringForValue => +// prepareTable(dir, numRows, width, useStringForValue) +// if (useStringForValue) { +// runStringBenchmark(numRows, width, mid, "string") +// } else { +// runIntBenchmark(numRows, width, mid) +// } +// } +// } +// } +// } +// +// runBenchmark("Pushdown for few distinct value case (use dictionary encoding)") { +// withTempPath { dir => +// val numDistinctValues = 200 +// +// withTempTable("orcTable", "parquetTable") { +// prepareStringDictTable(dir, numRows, numDistinctValues, width) +// runStringBenchmark(numRows, width, numDistinctValues / 2, "distinct string") +// } +// } +// } +// +// runBenchmark("Pushdown benchmark for StringStartsWith") { +// withTempPath { dir => +// withTempTable("orcTable", "parquetTable") { +// prepareTable(dir, numRows, width, true) +// Seq( +// "value like '10%'", +// "value like '1000%'", +// s"value like '${mid.toString.substring(0, mid.toString.length - 1)}%'" +// ).foreach { whereExpr => +// val title = s"StringStartsWith filter: ($whereExpr)" +// filterPushDownBenchmark(numRows, title, whereExpr) +// } +// } +// } +// } +// +// runBenchmark(s"Pushdown benchmark for ${DecimalType.simpleString}") { +// withTempPath { dir => +// Seq( +// s"decimal(${Decimal.MAX_INT_DIGITS}, 2)", +// s"decimal(${Decimal.MAX_LONG_DIGITS}, 2)", +// s"decimal(${DecimalType.MAX_PRECISION}, 2)" +// ).foreach { dt => +// val columns = (1 to width).map(i => s"CAST(id AS string) c$i") +// val valueCol = if (dt.equalsIgnoreCase(s"decimal(${Decimal.MAX_INT_DIGITS}, 2)")) { +// monotonically_increasing_id() % 9999999 +// } else { +// monotonically_increasing_id() +// } +// val df = spark.range(numRows) +// .selectExpr(columns: _*).withColumn("value", valueCol.cast(dt)) +// withTempTable("orcTable", "parquetTable") { +// saveAsTable(df, dir) +// +// Seq(s"value = $mid").foreach { whereExpr => +// val title = s"Select 1 $dt row ($whereExpr)".replace("value AND value", "value") +// filterPushDownBenchmark(numRows, title, whereExpr) +// } +// +// val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") +// Seq(10, 50, 90).foreach { percent => +// filterPushDownBenchmark( +// numRows, +// s"Select $percent% $dt rows (value < ${numRows * percent / 100})", +// s"value < ${numRows * percent / 100}", +// selectExpr +// ) +// } +// } +// } +// } +// } +// +// runBenchmark("Pushdown benchmark for InSet -> InFilters") { +// withTempPath { dir => +// withTempTable("orcTable", "parquetTable") { +// prepareTable(dir, numRows, width, false) +// Seq(5, 10, 50, 100).foreach { count => +// Seq(10, 50, 90).foreach { distribution => +// val filter = +// Range(0, count).map(r => scala.util.Random.nextInt(numRows * distribution / 100)) +// val whereExpr = s"value in(${filter.mkString(",")})" +// val title = s"InSet -> InFilters (values count: $count, distribution: $distribution)" +// filterPushDownBenchmark(numRows, title, whereExpr) +// } +// } +// } +// } +// } +// +// runBenchmark(s"Pushdown benchmark for ${ByteType.simpleString}") { +// withTempPath { dir => +// val columns = (1 to width).map(i => s"CAST(id AS string) c$i") +// val df = spark.range(numRows).selectExpr(columns: _*) +// .withColumn("value", (monotonically_increasing_id() % Byte.MaxValue).cast(ByteType)) +// .orderBy("value") +// withTempTable("orcTable", "parquetTable") { +// saveAsTable(df, dir) +// +// Seq(s"value = CAST(${Byte.MaxValue / 2} AS ${ByteType.simpleString})") +// .foreach { whereExpr => +// val title = s"Select 1 ${ByteType.simpleString} row ($whereExpr)" +// .replace("value AND value", "value") +// filterPushDownBenchmark(numRows, title, whereExpr) +// } +// +// val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") +// Seq(10, 50, 90).foreach { percent => +// filterPushDownBenchmark( +// numRows, +// s"Select $percent% ${ByteType.simpleString} rows " + +// s"(value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString}))", +// s"value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString})", +// selectExpr +// ) +// } +// } +// } +// } +// +// runBenchmark(s"Pushdown benchmark for Timestamp") { +// withTempPath { dir => +// withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> true.toString) { +// ParquetOutputTimestampType.values.toSeq.map(_.toString).foreach { fileType => +// withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fileType) { +// val columns = (1 to width).map(i => s"CAST(id AS string) c$i") +// val df = spark.range(numRows).selectExpr(columns: _*) +// .withColumn("value", monotonically_increasing_id().cast(TimestampType)) +// withTempTable("orcTable", "parquetTable") { +// saveAsTable(df, dir) +// +// Seq(s"value = CAST($mid AS timestamp)").foreach { whereExpr => +// val title = s"Select 1 timestamp stored as $fileType row ($whereExpr)" +// .replace("value AND value", "value") +// filterPushDownBenchmark(numRows, title, whereExpr) +// } +// +// val selectExpr = (1 to width) +// .map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") +// Seq(10, 50, 90).foreach { percent => +// filterPushDownBenchmark( +// numRows, +// s"Select $percent% timestamp stored as $fileType rows " + +// s"(value < CAST(${numRows * percent / 100} AS timestamp))", +// s"value < CAST(${numRows * percent / 100} as timestamp)", +// selectExpr +// ) +// } +// } +// } +// } +// } +// } +// } + +// runBenchmark(s"Pushdown benchmark with many filters") { +// val numRows = 1 +// val width = 500 +// +// withTempPath { dir => +// val columns = (1 to width).map(i => s"id c$i") +// val df = spark.range(1).selectExpr(columns: _*) +// withTempTable("orcTable", "parquetTable") { +// saveAsTable(df, dir) +// Seq(1, 250, 500).foreach { numFilter => +// val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") +// // Note: InferFiltersFromConstraints will add more filters to this given filters +// filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) +// } +// } +// } +// } + + runBenchmark(s"Pushdown benchmark with unbalanced Column") { val numRows = 1 - val width = 500 + val width = 1000 withTempPath { dir => val columns = (1 to width).map(i => s"id c$i") val df = spark.range(1).selectExpr(columns: _*) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - Seq(1, 250, 500).foreach { numFilter => - val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" and ") + (1 to width by 50).foreach { numFilter => + val whereColumn = (1 to numFilter) + .map(i => col(s"c$i") === lit(0)) + .foldLeft(lit(false))(_ || _) +// val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") // Note: InferFiltersFromConstraints will add more filters to this given filters - filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) +// filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) + filterPushDownBenchmarkWithColumn( + numRows, + s"Select 1 row with $numFilter filters", + whereColumn) } } } diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 9e4bf22ff7e3..900f78674a5a 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -185,22 +185,28 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val leftBuilderOption = - createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - val rightBuilderOption = - createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - (leftBuilderOption, rightBuilderOption) match { - case (Some(_), Some(_)) => + val leftIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + left, + canPartialPushDownConjuncts + ) + val rightIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + right, + canPartialPushDownConjuncts + ) + (leftIsConvertible, rightIsConvertible) match { + case (true, true) => for { lhs <- createBuilder(dataTypeMap, left, builder.startAnd(), canPartialPushDownConjuncts) rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) } yield rhs.end() - case (Some(_), None) if canPartialPushDownConjuncts => + case (true, false) if canPartialPushDownConjuncts => createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) - case (None, Some(_)) if canPartialPushDownConjuncts => + case (false, true) if canPartialPushDownConjuncts => createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) case _ => None @@ -218,9 +224,18 @@ private[sql] object OrcFilters extends OrcFiltersBase { // The predicate can be converted as // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). + val leftIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + left, + canPartialPushDownConjuncts + ) + val rightIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + right, + canPartialPushDownConjuncts + ) for { - _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) + _ <- Option(leftIsConvertible && rightIsConvertible).filter(identity) lhs <- createBuilder(dataTypeMap, left, builder.startOr(), canPartialPushDownConjuncts) rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) @@ -228,9 +243,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Not(child) => for { - _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) negate <- createBuilder(dataTypeMap, child, builder.startNot(), canPartialPushDownConjuncts = false) + if isConvertibleToOrcPredicate(dataTypeMap, child, canPartialPushDownConjuncts = false) } yield negate.end() // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` @@ -284,4 +299,79 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => None } } + + private def isConvertibleToOrcPredicate( + dataTypeMap: Map[String, DataType], + expression: Filter, + canPartialPushDownConjuncts: Boolean + ): Boolean = { + import org.apache.spark.sql.sources._ + + expression match { + case And(left, right) => + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + val leftIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + left, + canPartialPushDownConjuncts) + val rightIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + right, + canPartialPushDownConjuncts) + // NOTE: If we can use partial predicates here, we only need one of the children to + // be convertible to be able to convert the parent. Otherwise, we need both to be + // convertible. + if (canPartialPushDownConjuncts) { + leftIsConvertible || rightIsConvertible + } else { + leftIsConvertible && rightIsConvertible + } + + case Or(left, right) => + val leftIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + left, + canPartialPushDownConjuncts) + val rightIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + right, + canPartialPushDownConjuncts + ) + leftIsConvertible && rightIsConvertible + + case Not(child) => + val childIsConvertible = isConvertibleToOrcPredicate( + dataTypeMap, + child, + canPartialPushDownConjuncts = false + ) + childIsConvertible + + // 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`). + + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => true + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => true + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => true + + case _ => false + } + } } From d6c408e05451f2267e5b22a8a3a21467991460f2 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 18 Feb 2019 20:14:39 +0100 Subject: [PATCH 02/56] Implement isConvertibleToOrcPredicate with linear complexity --- .../datasources/orc/OrcFilters.scala | 93 ++++++++++--------- 1 file changed, 49 insertions(+), 44 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 900f78674a5a..661216a95c8d 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -22,6 +22,7 @@ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.orc.storage.serde2.io.HiveDecimalWritable +import scala.collection.mutable import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -150,7 +151,12 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) + createBuilder( + dataTypeMap, + new OrcConvertibilityChecker(dataTypeMap), + expression, + builder, + canPartialPushDownConjuncts = true) } /** @@ -164,6 +170,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { */ private def createBuilder( dataTypeMap: Map[String, DataType], + orcConvertibilityChecker: OrcConvertibilityChecker, expression: Filter, builder: Builder, canPartialPushDownConjuncts: Boolean): Option[Builder] = { @@ -185,29 +192,30 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val leftIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, + val leftIsConvertible = orcConvertibilityChecker.isConvertible( left, canPartialPushDownConjuncts ) - val rightIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, + val rightIsConvertible = orcConvertibilityChecker.isConvertible( right, canPartialPushDownConjuncts ) (leftIsConvertible, rightIsConvertible) match { case (true, true) => for { - lhs <- createBuilder(dataTypeMap, left, + lhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, left, builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) + rhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, right, + lhs, canPartialPushDownConjuncts) } yield rhs.end() case (true, false) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) + createBuilder(dataTypeMap, orcConvertibilityChecker, left, + builder, canPartialPushDownConjuncts) case (false, true) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) + createBuilder(dataTypeMap, orcConvertibilityChecker, right, + builder, canPartialPushDownConjuncts) case _ => None } @@ -224,28 +232,27 @@ private[sql] object OrcFilters extends OrcFiltersBase { // The predicate can be converted as // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). - val leftIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, + val leftIsConvertible = orcConvertibilityChecker.isConvertible( left, canPartialPushDownConjuncts ) - val rightIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, + val rightIsConvertible = orcConvertibilityChecker.isConvertible( right, canPartialPushDownConjuncts ) for { _ <- Option(leftIsConvertible && rightIsConvertible).filter(identity) - lhs <- createBuilder(dataTypeMap, left, + lhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, left, builder.startOr(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) + rhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, right, + lhs, canPartialPushDownConjuncts) } yield rhs.end() case Not(child) => for { - negate <- createBuilder(dataTypeMap, + negate <- createBuilder(dataTypeMap, orcConvertibilityChecker, child, builder.startNot(), canPartialPushDownConjuncts = false) - if isConvertibleToOrcPredicate(dataTypeMap, child, canPartialPushDownConjuncts = false) + if orcConvertibilityChecker.isConvertible(child, canPartialPushDownConjuncts = false) } yield negate.end() // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` @@ -299,13 +306,28 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => None } } +} + +private class OrcConvertibilityChecker( + dataTypeMap: Map[String, DataType] +) { + // TODO(ivan): correctness might be more obvious if this is (Filter, Boolean) -> Boolean instead + // of just Filter -> Boolean. + private val convertibilityCache = new mutable.HashMap[Filter, Boolean] + + def isConvertible(expression: Filter, canPartialPushDownConjuncts: Boolean): Boolean = { + if (!convertibilityCache.contains(expression)) { + convertibilityCache.put(expression, + isConvertibleImpl(expression, canPartialPushDownConjuncts)) + } + convertibilityCache(expression) + } - private def isConvertibleToOrcPredicate( - dataTypeMap: Map[String, DataType], - expression: Filter, - canPartialPushDownConjuncts: Boolean - ): Boolean = { + private def isConvertibleImpl( + expression: Filter, + canPartialPushDownConjuncts: Boolean): Boolean = { import org.apache.spark.sql.sources._ + import OrcFilters._ expression match { case And(left, right) => @@ -320,14 +342,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val leftIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, - left, - canPartialPushDownConjuncts) - val rightIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, - right, - canPartialPushDownConjuncts) + val leftIsConvertible = isConvertible(left, canPartialPushDownConjuncts) + val rightIsConvertible = isConvertible(right, canPartialPushDownConjuncts) // NOTE: If we can use partial predicates here, we only need one of the children to // be convertible to be able to convert the parent. Otherwise, we need both to be // convertible. @@ -338,23 +354,12 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case Or(left, right) => - val leftIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, - left, - canPartialPushDownConjuncts) - val rightIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, - right, - canPartialPushDownConjuncts - ) + val leftIsConvertible = isConvertible(left, canPartialPushDownConjuncts) + val rightIsConvertible = isConvertible(right, canPartialPushDownConjuncts) leftIsConvertible && rightIsConvertible case Not(child) => - val childIsConvertible = isConvertibleToOrcPredicate( - dataTypeMap, - child, - canPartialPushDownConjuncts = false - ) + val childIsConvertible = isConvertible(child, canPartialPushDownConjuncts = false) childIsConvertible // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` From 66f5064b3e0479ffb9eb9828b1b13d489423760e Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 19 Feb 2019 07:20:06 -0800 Subject: [PATCH 03/56] Run FilterPushdownBenchmark with a smaller number of widths --- .../spark/sql/execution/benchmark/FilterPushdownBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 13366c6a9619..e9fc26d7d0dd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -407,7 +407,7 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType val df = spark.range(1).selectExpr(columns: _*) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - (1 to width by 50).foreach { numFilter => + (1 to width by 200).foreach { numFilter => val whereColumn = (1 to numFilter) .map(i => col(s"c$i") === lit(0)) .foldLeft(lit(false))(_ || _) From 1444c3cf5822de74c7eb06c106ae30bb467060a7 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 25 Feb 2019 18:12:58 -0800 Subject: [PATCH 04/56] fix formatting issues from merge --- .../sql/execution/benchmark/FilterPushdownBenchmark.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index e9fc26d7d0dd..4ae5a72e0e90 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -41,7 +41,7 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType * Results will be written to "benchmarks/FilterPushdownBenchmark-results.txt". * }}} */ - object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { +object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { private val conf = new SparkConf() .setAppName(this.getClass.getSimpleName) @@ -125,7 +125,7 @@ import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType Seq(false, true).foreach { pushDownEnabled => val name = s"Native ORC Vectorized ${if (pushDownEnabled) s"(Pushdown)" else ""}" - benchmark.addCase(name) { _ => + benchmark.addCase(name) { _ => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> s"$pushDownEnabled") { spark.sql(s"SELECT $selectExpr FROM orcTable WHERE $whereExpr").collect() } From b662b6056343287d13ad2beddddccf9f8906fb17 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 25 Feb 2019 21:17:15 -0800 Subject: [PATCH 05/56] Use Map.withDefault for the cache, and key it on (Filter, Boolean) --- .../sql/execution/datasources/orc/OrcFilters.scala | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 661216a95c8d..3669b7b2c164 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -311,16 +311,12 @@ private[sql] object OrcFilters extends OrcFiltersBase { private class OrcConvertibilityChecker( dataTypeMap: Map[String, DataType] ) { - // TODO(ivan): correctness might be more obvious if this is (Filter, Boolean) -> Boolean instead - // of just Filter -> Boolean. - private val convertibilityCache = new mutable.HashMap[Filter, Boolean] + + private val convertibilityCache = new mutable.HashMap[(Filter, Boolean), Boolean] + .withDefault(Function.tupled(isConvertibleImpl _)) def isConvertible(expression: Filter, canPartialPushDownConjuncts: Boolean): Boolean = { - if (!convertibilityCache.contains(expression)) { - convertibilityCache.put(expression, - isConvertibleImpl(expression, canPartialPushDownConjuncts)) - } - convertibilityCache(expression) + convertibilityCache((expression, canPartialPushDownConjuncts)) } private def isConvertibleImpl( From 6336415a693f76ff39f3f740bc75b4bb324216b1 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 25 Feb 2019 22:37:50 -0800 Subject: [PATCH 06/56] Use Map.getOrElseUpdate instead of WithDefault for actual caching; use a case class for Map keys --- .../datasources/orc/OrcFilters.scala | 43 ++++++++++++------- 1 file changed, 27 insertions(+), 16 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 3669b7b2c164..48bbc0251f17 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -308,24 +308,39 @@ private[sql] object OrcFilters extends OrcFiltersBase { } } +private case class FilterWithConjunctPushdown( + expression: Filter, + canPartialPushDownConjuncts: Boolean +) + +/** + * Helper class for efficiently checking whether a `Filter` and its children can be converted to + * ORC `SearchArgument`s. + * + * @param dataTypeMap + */ private class OrcConvertibilityChecker( dataTypeMap: Map[String, DataType] ) { - private val convertibilityCache = new mutable.HashMap[(Filter, Boolean), Boolean] - .withDefault(Function.tupled(isConvertibleImpl _)) + private val convertibilityCache = new mutable.HashMap[FilterWithConjunctPushdown, Boolean] def isConvertible(expression: Filter, canPartialPushDownConjuncts: Boolean): Boolean = { - convertibilityCache((expression, canPartialPushDownConjuncts)) + val node = FilterWithConjunctPushdown(expression, canPartialPushDownConjuncts) + convertibilityCache.getOrElseUpdate(node, isConvertibleImpl(node)) } - private def isConvertibleImpl( - expression: Filter, - canPartialPushDownConjuncts: Boolean): Boolean = { + /** + * This method duplicates the logic from `OrcFilters.createBuilder` that is related to checking + * if a given part of the filter is actually convertible to an ORC `SearchArgument`. + * @param node + * @return + */ + private def isConvertibleImpl(node: FilterWithConjunctPushdown): Boolean = { import org.apache.spark.sql.sources._ import OrcFilters._ - expression match { + node.expression match { case And(left, right) => // At here, it is not safe to just convert one side and remove the other side // if we do not understand what the parent filters are. @@ -338,30 +353,26 @@ private class OrcConvertibilityChecker( // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val leftIsConvertible = isConvertible(left, canPartialPushDownConjuncts) - val rightIsConvertible = isConvertible(right, canPartialPushDownConjuncts) + val leftIsConvertible = isConvertible(left, node.canPartialPushDownConjuncts) + val rightIsConvertible = isConvertible(right, node.canPartialPushDownConjuncts) // NOTE: If we can use partial predicates here, we only need one of the children to // be convertible to be able to convert the parent. Otherwise, we need both to be // convertible. - if (canPartialPushDownConjuncts) { + if (node.canPartialPushDownConjuncts) { leftIsConvertible || rightIsConvertible } else { leftIsConvertible && rightIsConvertible } case Or(left, right) => - val leftIsConvertible = isConvertible(left, canPartialPushDownConjuncts) - val rightIsConvertible = isConvertible(right, canPartialPushDownConjuncts) + val leftIsConvertible = isConvertible(left, node.canPartialPushDownConjuncts) + val rightIsConvertible = isConvertible(right, node.canPartialPushDownConjuncts) leftIsConvertible && rightIsConvertible case Not(child) => val childIsConvertible = isConvertible(child, canPartialPushDownConjuncts = false) childIsConvertible - // 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`). - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true From e6b72a0fe83d52e131f5fb4985dddd15276ab7c7 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 26 Feb 2019 01:19:27 -0800 Subject: [PATCH 07/56] Override equals and hashCode to get to actual linear complexity --- .../datasources/orc/OrcFilters.scala | 25 +++++++++++++++---- 1 file changed, 20 insertions(+), 5 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 48bbc0251f17..ee4555b1fe4e 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.orc +import java.util.Objects + import org.apache.orc.storage.common.`type`.HiveDecimal import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder @@ -308,10 +310,25 @@ private[sql] object OrcFilters extends OrcFiltersBase { } } -private case class FilterWithConjunctPushdown( +private sealed case class FilterWithConjunctPushdown( expression: Filter, canPartialPushDownConjuncts: Boolean -) +) { + override def hashCode(): Int = { + Objects.hash( + System.identityHashCode(expression).asInstanceOf[Object], + canPartialPushDownConjuncts.asInstanceOf[Object]) + } + + override def equals(obj: Any): Boolean = { + obj match { + case FilterWithConjunctPushdown(expr, canPushDown) => + Objects.equals(System.identityHashCode(expression), System.identityHashCode(expr)) && + Objects.equals(canPartialPushDownConjuncts, canPushDown) + case _ => false + } + } +} /** * Helper class for efficiently checking whether a `Filter` and its children can be converted to @@ -319,9 +336,7 @@ private case class FilterWithConjunctPushdown( * * @param dataTypeMap */ -private class OrcConvertibilityChecker( - dataTypeMap: Map[String, DataType] -) { +private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { private val convertibilityCache = new mutable.HashMap[FilterWithConjunctPushdown, Boolean] From 8a2c33f5a906ea51751872bc31ecd3a4860db242 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 26 Feb 2019 01:19:52 -0800 Subject: [PATCH 08/56] Add a benchmark that tests only conversion and is able to expose linear vs quadratic behaviour --- .../benchmark/FilterPushdownBenchmark.scala | 43 ++++++++++++++----- 1 file changed, 33 insertions(+), 10 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 4ae5a72e0e90..230a08cb6b52 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -25,11 +25,15 @@ import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{Column, DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, Literal, Or} import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.execution.datasources.DataSourceStrategy +import org.apache.spark.sql.execution.datasources.orc.OrcFilters import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType -import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType} +import org.apache.spark.sql.types._ /** * Benchmark to measure read performance with Filter pushdown. @@ -400,24 +404,43 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { runBenchmark(s"Pushdown benchmark with unbalanced Column") { val numRows = 1 - val width = 1000 + val width = 2000 withTempPath { dir => val columns = (1 to width).map(i => s"id c$i") val df = spark.range(1).selectExpr(columns: _*) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - (1 to width by 200).foreach { numFilter => - val whereColumn = (1 to numFilter) - .map(i => col(s"c$i") === lit(0)) - .foldLeft(lit(false))(_ || _) + (1000 to 15000 by 2000).foreach { numFilter => +// val whereColumn = (1 to numFilter) +// .map(i => col(s"c$i") === lit(0)) +// .foldLeft(lit(false))(_ || _) + val whereExpression = (1 to numFilter) + .map { + i => EqualTo( + Literal(0), + AttributeReference( + s"c1", + IntegerType, + nullable = true)() + ).asInstanceOf[Expression] + } + .foldLeft[Expression](Literal.FalseLiteral)((x, y) => Or(x, y)) + val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", + numRows, minNumIters = 5, output = output) + val name = s"Native ORC Vectorized (Pushdown)" + benchmark.addCase(name) { _ => + OrcFilters.createFilter(df.schema, + DataSourceStrategy.translateFilter(whereExpression).toSeq) + } + benchmark.run() // val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") // Note: InferFiltersFromConstraints will add more filters to this given filters // filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) - filterPushDownBenchmarkWithColumn( - numRows, - s"Select 1 row with $numFilter filters", - whereColumn) +// filterPushDownBenchmarkWithColumn( +// numRows, +// s"Select 1 row with $numFilter filters", +// whereColumn) } } } From 3f095e06f5f944290059b9df40057e1c72feea59 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 1 Mar 2019 19:55:50 +0100 Subject: [PATCH 09/56] Add benchmarks for converting both from Column and from Expression --- .../benchmark/FilterPushdownBenchmark.scala | 52 ++++++++++++------- 1 file changed, 32 insertions(+), 20 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 230a08cb6b52..6df9247d5677 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -402,7 +402,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { // } // } - runBenchmark(s"Pushdown benchmark with unbalanced Column") { + runBenchmark(s"Predicate conversion benchmark with unbalanced Expression") { val numRows = 1 val width = 2000 @@ -411,10 +411,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val df = spark.range(1).selectExpr(columns: _*) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - (1000 to 15000 by 2000).foreach { numFilter => -// val whereColumn = (1 to numFilter) -// .map(i => col(s"c$i") === lit(0)) -// .foldLeft(lit(false))(_ || _) + (1000 to 15000 by 2000).foreach { numFilter => val whereExpression = (1 to numFilter) .map { i => EqualTo( @@ -423,24 +420,39 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { s"c1", IntegerType, nullable = true)() - ).asInstanceOf[Expression] + ).asInstanceOf[Expression] } .foldLeft[Expression](Literal.FalseLiteral)((x, y) => Or(x, y)) - val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", - numRows, minNumIters = 5, output = output) - val name = s"Native ORC Vectorized (Pushdown)" - benchmark.addCase(name) { _ => - OrcFilters.createFilter(df.schema, - DataSourceStrategy.translateFilter(whereExpression).toSeq) + val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", + numRows, minNumIters = 5, output = output) + val name = s"Native ORC Vectorized (Pushdown)" + benchmark.addCase(name) { _ => + OrcFilters.createFilter(df.schema, + DataSourceStrategy.translateFilter(whereExpression).toSeq) + } + benchmark.run() } - benchmark.run() -// val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") - // Note: InferFiltersFromConstraints will add more filters to this given filters -// filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) -// filterPushDownBenchmarkWithColumn( -// numRows, -// s"Select 1 row with $numFilter filters", -// whereColumn) + } + } + } + + runBenchmark(s"Pushdown benchmark with unbalanced Column") { + val numRows = 1 + val width = 200 + + withTempPath { dir => + val columns = (1 to width).map(i => s"id c$i") + val df = spark.range(1).selectExpr(columns: _*) + withTempTable("orcTable", "parquetTable") { + saveAsTable(df, dir) + (1 to 1001 by 200).foreach { numFilter => + val whereColumn = (1 to numFilter) + .map(i => col("c1") === lit(i)) + .foldLeft(lit(false))(_ || _) + filterPushDownBenchmarkWithColumn( + numRows, + s"Select 1 row with $numFilter filters", + whereColumn) } } } From 1f15026a4efad8097419fed9b31009b5f341d820 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 1 Mar 2019 21:18:06 +0100 Subject: [PATCH 10/56] Get rid of unnecessary objects in Expression benchmark --- .../benchmark/FilterPushdownBenchmark.scala | 47 +++++++++---------- 1 file changed, 22 insertions(+), 25 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 6df9247d5677..d4acb6c71c08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -147,6 +147,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { ): Unit = { val benchmark = new Benchmark(title, values, minNumIters = 5, output = output) val name = s"Native ORC Vectorized (Pushdown)" + // TODO(ivan): Consider adding a case for Parquet here as well. benchmark.addCase(name) { _ => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { spark @@ -406,33 +407,29 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val numRows = 1 val width = 2000 - withTempPath { dir => - val columns = (1 to width).map(i => s"id c$i") - val df = spark.range(1).selectExpr(columns: _*) - withTempTable("orcTable", "parquetTable") { - saveAsTable(df, dir) - (1000 to 15000 by 2000).foreach { numFilter => - val whereExpression = (1 to numFilter) - .map { - i => EqualTo( - Literal(0), - AttributeReference( - s"c1", - IntegerType, - nullable = true)() - ).asInstanceOf[Expression] - } - .foldLeft[Expression](Literal.FalseLiteral)((x, y) => Or(x, y)) - val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", - numRows, minNumIters = 5, output = output) - val name = s"Native ORC Vectorized (Pushdown)" - benchmark.addCase(name) { _ => - OrcFilters.createFilter(df.schema, - DataSourceStrategy.translateFilter(whereExpression).toSeq) - } - benchmark.run() + val columns = (1 to width).map(i => s"id c$i") + val df = spark.range(1).selectExpr(columns: _*) + (1000 to 15000 by 2000).foreach { numFilter => + val whereExpression = (1 to numFilter) + .map { + i => + EqualTo( + Literal(0), + AttributeReference( + s"c1", + IntegerType, + nullable = true)() + ).asInstanceOf[Expression] } + .foldLeft[Expression](Literal.FalseLiteral)((x, y) => Or(x, y)) + val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", + numRows, minNumIters = 5, output = output) + val name = s"Native ORC Vectorized (Pushdown)" + benchmark.addCase(name) { _ => + OrcFilters.createFilter(df.schema, + DataSourceStrategy.translateFilter(whereExpression).toSeq) } + benchmark.run() } } From bf5e52147efe50962637562a032ee9c3ac2bdcb1 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 1 Mar 2019 21:32:13 +0100 Subject: [PATCH 11/56] Don't use a raw SQL query in Column pushdown benchmark --- .../execution/benchmark/FilterPushdownBenchmark.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index d4acb6c71c08..531e4ee9fc1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -151,7 +151,8 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { benchmark.addCase(name) { _ => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { spark - .sql(s"SELECT $selectExpr FROM orcTable") + .table("orcTable") + .select(selectExpr) .filter(whereColumn) .collect() } @@ -443,9 +444,9 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) (1 to 1001 by 200).foreach { numFilter => - val whereColumn = (1 to numFilter) - .map(i => col("c1") === lit(i)) - .foldLeft(lit(false))(_ || _) + val whereColumn = (1 to numFilter) + .map(i => col("c1") === lit(i)) + .foldLeft(lit(false))(_ || _) filterPushDownBenchmarkWithColumn( numRows, s"Select 1 row with $numFilter filters", From 15b3e5b454f5848dd453800a3edbf08a78bd8907 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 1 Mar 2019 21:36:59 +0100 Subject: [PATCH 12/56] Add a benchmark case for Parquet Column pushdown --- .../benchmark/FilterPushdownBenchmark.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 531e4ee9fc1b..d3e5196e2206 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -146,9 +146,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { selectExpr: String = "*" ): Unit = { val benchmark = new Benchmark(title, values, minNumIters = 5, output = output) - val name = s"Native ORC Vectorized (Pushdown)" - // TODO(ivan): Consider adding a case for Parquet here as well. - benchmark.addCase(name) { _ => + benchmark.addCase("Native ORC Vectorized (Pushdown)") { _ => withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { spark .table("orcTable") @@ -157,6 +155,15 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { .collect() } } + benchmark.addCase("Native Parquet Vectorized (Pushdown)") { _ => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { + spark + .table("parquetTable") + .select(selectExpr) + .filter(whereColumn) + .collect() + } + } benchmark.run() } From b0f8af24b4c4e6392e87ecb52c9c986c1d9b8ca7 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 8 Mar 2019 11:48:18 +0200 Subject: [PATCH 13/56] Uncomment previously commented benchmarks --- .../benchmark/FilterPushdownBenchmark.scala | 356 +++++++++--------- 1 file changed, 178 insertions(+), 178 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index d3e5196e2206..0046c2bb88f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -232,184 +232,184 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { -// runBenchmark("Pushdown for many distinct value case") { -// withTempPath { dir => -// withTempTable("orcTable", "parquetTable") { -// Seq(true, false).foreach { useStringForValue => -// prepareTable(dir, numRows, width, useStringForValue) -// if (useStringForValue) { -// runStringBenchmark(numRows, width, mid, "string") -// } else { -// runIntBenchmark(numRows, width, mid) -// } -// } -// } -// } -// } -// -// runBenchmark("Pushdown for few distinct value case (use dictionary encoding)") { -// withTempPath { dir => -// val numDistinctValues = 200 -// -// withTempTable("orcTable", "parquetTable") { -// prepareStringDictTable(dir, numRows, numDistinctValues, width) -// runStringBenchmark(numRows, width, numDistinctValues / 2, "distinct string") -// } -// } -// } -// -// runBenchmark("Pushdown benchmark for StringStartsWith") { -// withTempPath { dir => -// withTempTable("orcTable", "parquetTable") { -// prepareTable(dir, numRows, width, true) -// Seq( -// "value like '10%'", -// "value like '1000%'", -// s"value like '${mid.toString.substring(0, mid.toString.length - 1)}%'" -// ).foreach { whereExpr => -// val title = s"StringStartsWith filter: ($whereExpr)" -// filterPushDownBenchmark(numRows, title, whereExpr) -// } -// } -// } -// } -// -// runBenchmark(s"Pushdown benchmark for ${DecimalType.simpleString}") { -// withTempPath { dir => -// Seq( -// s"decimal(${Decimal.MAX_INT_DIGITS}, 2)", -// s"decimal(${Decimal.MAX_LONG_DIGITS}, 2)", -// s"decimal(${DecimalType.MAX_PRECISION}, 2)" -// ).foreach { dt => -// val columns = (1 to width).map(i => s"CAST(id AS string) c$i") -// val valueCol = if (dt.equalsIgnoreCase(s"decimal(${Decimal.MAX_INT_DIGITS}, 2)")) { -// monotonically_increasing_id() % 9999999 -// } else { -// monotonically_increasing_id() -// } -// val df = spark.range(numRows) -// .selectExpr(columns: _*).withColumn("value", valueCol.cast(dt)) -// withTempTable("orcTable", "parquetTable") { -// saveAsTable(df, dir) -// -// Seq(s"value = $mid").foreach { whereExpr => -// val title = s"Select 1 $dt row ($whereExpr)".replace("value AND value", "value") -// filterPushDownBenchmark(numRows, title, whereExpr) -// } -// -// val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") -// Seq(10, 50, 90).foreach { percent => -// filterPushDownBenchmark( -// numRows, -// s"Select $percent% $dt rows (value < ${numRows * percent / 100})", -// s"value < ${numRows * percent / 100}", -// selectExpr -// ) -// } -// } -// } -// } -// } -// -// runBenchmark("Pushdown benchmark for InSet -> InFilters") { -// withTempPath { dir => -// withTempTable("orcTable", "parquetTable") { -// prepareTable(dir, numRows, width, false) -// Seq(5, 10, 50, 100).foreach { count => -// Seq(10, 50, 90).foreach { distribution => -// val filter = -// Range(0, count).map(r => scala.util.Random.nextInt(numRows * distribution / 100)) -// val whereExpr = s"value in(${filter.mkString(",")})" -// val title = s"InSet -> InFilters (values count: $count, distribution: $distribution)" -// filterPushDownBenchmark(numRows, title, whereExpr) -// } -// } -// } -// } -// } -// -// runBenchmark(s"Pushdown benchmark for ${ByteType.simpleString}") { -// withTempPath { dir => -// val columns = (1 to width).map(i => s"CAST(id AS string) c$i") -// val df = spark.range(numRows).selectExpr(columns: _*) -// .withColumn("value", (monotonically_increasing_id() % Byte.MaxValue).cast(ByteType)) -// .orderBy("value") -// withTempTable("orcTable", "parquetTable") { -// saveAsTable(df, dir) -// -// Seq(s"value = CAST(${Byte.MaxValue / 2} AS ${ByteType.simpleString})") -// .foreach { whereExpr => -// val title = s"Select 1 ${ByteType.simpleString} row ($whereExpr)" -// .replace("value AND value", "value") -// filterPushDownBenchmark(numRows, title, whereExpr) -// } -// -// val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") -// Seq(10, 50, 90).foreach { percent => -// filterPushDownBenchmark( -// numRows, -// s"Select $percent% ${ByteType.simpleString} rows " + -// s"(value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString}))", -// s"value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString})", -// selectExpr -// ) -// } -// } -// } -// } -// -// runBenchmark(s"Pushdown benchmark for Timestamp") { -// withTempPath { dir => -// withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> true.toString) { -// ParquetOutputTimestampType.values.toSeq.map(_.toString).foreach { fileType => -// withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fileType) { -// val columns = (1 to width).map(i => s"CAST(id AS string) c$i") -// val df = spark.range(numRows).selectExpr(columns: _*) -// .withColumn("value", monotonically_increasing_id().cast(TimestampType)) -// withTempTable("orcTable", "parquetTable") { -// saveAsTable(df, dir) -// -// Seq(s"value = CAST($mid AS timestamp)").foreach { whereExpr => -// val title = s"Select 1 timestamp stored as $fileType row ($whereExpr)" -// .replace("value AND value", "value") -// filterPushDownBenchmark(numRows, title, whereExpr) -// } -// -// val selectExpr = (1 to width) -// .map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") -// Seq(10, 50, 90).foreach { percent => -// filterPushDownBenchmark( -// numRows, -// s"Select $percent% timestamp stored as $fileType rows " + -// s"(value < CAST(${numRows * percent / 100} AS timestamp))", -// s"value < CAST(${numRows * percent / 100} as timestamp)", -// selectExpr -// ) -// } -// } -// } -// } -// } -// } -// } - -// runBenchmark(s"Pushdown benchmark with many filters") { -// val numRows = 1 -// val width = 500 -// -// withTempPath { dir => -// val columns = (1 to width).map(i => s"id c$i") -// val df = spark.range(1).selectExpr(columns: _*) -// withTempTable("orcTable", "parquetTable") { -// saveAsTable(df, dir) -// Seq(1, 250, 500).foreach { numFilter => -// val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") -// // Note: InferFiltersFromConstraints will add more filters to this given filters -// filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) -// } -// } -// } -// } + runBenchmark("Pushdown for many distinct value case") { + withTempPath { dir => + withTempTable("orcTable", "parquetTable") { + Seq(true, false).foreach { useStringForValue => + prepareTable(dir, numRows, width, useStringForValue) + if (useStringForValue) { + runStringBenchmark(numRows, width, mid, "string") + } else { + runIntBenchmark(numRows, width, mid) + } + } + } + } + } + + runBenchmark("Pushdown for few distinct value case (use dictionary encoding)") { + withTempPath { dir => + val numDistinctValues = 200 + + withTempTable("orcTable", "parquetTable") { + prepareStringDictTable(dir, numRows, numDistinctValues, width) + runStringBenchmark(numRows, width, numDistinctValues / 2, "distinct string") + } + } + } + + runBenchmark("Pushdown benchmark for StringStartsWith") { + withTempPath { dir => + withTempTable("orcTable", "parquetTable") { + prepareTable(dir, numRows, width, true) + Seq( + "value like '10%'", + "value like '1000%'", + s"value like '${mid.toString.substring(0, mid.toString.length - 1)}%'" + ).foreach { whereExpr => + val title = s"StringStartsWith filter: ($whereExpr)" + filterPushDownBenchmark(numRows, title, whereExpr) + } + } + } + } + + runBenchmark(s"Pushdown benchmark for ${DecimalType.simpleString}") { + withTempPath { dir => + Seq( + s"decimal(${Decimal.MAX_INT_DIGITS}, 2)", + s"decimal(${Decimal.MAX_LONG_DIGITS}, 2)", + s"decimal(${DecimalType.MAX_PRECISION}, 2)" + ).foreach { dt => + val columns = (1 to width).map(i => s"CAST(id AS string) c$i") + val valueCol = if (dt.equalsIgnoreCase(s"decimal(${Decimal.MAX_INT_DIGITS}, 2)")) { + monotonically_increasing_id() % 9999999 + } else { + monotonically_increasing_id() + } + val df = spark.range(numRows) + .selectExpr(columns: _*).withColumn("value", valueCol.cast(dt)) + withTempTable("orcTable", "parquetTable") { + saveAsTable(df, dir) + + Seq(s"value = $mid").foreach { whereExpr => + val title = s"Select 1 $dt row ($whereExpr)".replace("value AND value", "value") + filterPushDownBenchmark(numRows, title, whereExpr) + } + + val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") + Seq(10, 50, 90).foreach { percent => + filterPushDownBenchmark( + numRows, + s"Select $percent% $dt rows (value < ${numRows * percent / 100})", + s"value < ${numRows * percent / 100}", + selectExpr + ) + } + } + } + } + } + + runBenchmark("Pushdown benchmark for InSet -> InFilters") { + withTempPath { dir => + withTempTable("orcTable", "parquetTable") { + prepareTable(dir, numRows, width, false) + Seq(5, 10, 50, 100).foreach { count => + Seq(10, 50, 90).foreach { distribution => + val filter = + Range(0, count).map(r => scala.util.Random.nextInt(numRows * distribution / 100)) + val whereExpr = s"value in(${filter.mkString(",")})" + val title = s"InSet -> InFilters (values count: $count, distribution: $distribution)" + filterPushDownBenchmark(numRows, title, whereExpr) + } + } + } + } + } + + runBenchmark(s"Pushdown benchmark for ${ByteType.simpleString}") { + withTempPath { dir => + val columns = (1 to width).map(i => s"CAST(id AS string) c$i") + val df = spark.range(numRows).selectExpr(columns: _*) + .withColumn("value", (monotonically_increasing_id() % Byte.MaxValue).cast(ByteType)) + .orderBy("value") + withTempTable("orcTable", "parquetTable") { + saveAsTable(df, dir) + + Seq(s"value = CAST(${Byte.MaxValue / 2} AS ${ByteType.simpleString})") + .foreach { whereExpr => + val title = s"Select 1 ${ByteType.simpleString} row ($whereExpr)" + .replace("value AND value", "value") + filterPushDownBenchmark(numRows, title, whereExpr) + } + + val selectExpr = (1 to width).map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") + Seq(10, 50, 90).foreach { percent => + filterPushDownBenchmark( + numRows, + s"Select $percent% ${ByteType.simpleString} rows " + + s"(value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString}))", + s"value < CAST(${Byte.MaxValue * percent / 100} AS ${ByteType.simpleString})", + selectExpr + ) + } + } + } + } + + runBenchmark(s"Pushdown benchmark for Timestamp") { + withTempPath { dir => + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> true.toString) { + ParquetOutputTimestampType.values.toSeq.map(_.toString).foreach { fileType => + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fileType) { + val columns = (1 to width).map(i => s"CAST(id AS string) c$i") + val df = spark.range(numRows).selectExpr(columns: _*) + .withColumn("value", monotonically_increasing_id().cast(TimestampType)) + withTempTable("orcTable", "parquetTable") { + saveAsTable(df, dir) + + Seq(s"value = CAST($mid AS timestamp)").foreach { whereExpr => + val title = s"Select 1 timestamp stored as $fileType row ($whereExpr)" + .replace("value AND value", "value") + filterPushDownBenchmark(numRows, title, whereExpr) + } + + val selectExpr = (1 to width) + .map(i => s"MAX(c$i)").mkString("", ",", ", MAX(value)") + Seq(10, 50, 90).foreach { percent => + filterPushDownBenchmark( + numRows, + s"Select $percent% timestamp stored as $fileType rows " + + s"(value < CAST(${numRows * percent / 100} AS timestamp))", + s"value < CAST(${numRows * percent / 100} as timestamp)", + selectExpr + ) + } + } + } + } + } + } + } + + runBenchmark(s"Pushdown benchmark with many filters") { + val numRows = 1 + val width = 500 + + withTempPath { dir => + val columns = (1 to width).map(i => s"id c$i") + val df = spark.range(1).selectExpr(columns: _*) + withTempTable("orcTable", "parquetTable") { + saveAsTable(df, dir) + Seq(1, 250, 500).foreach { numFilter => + val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") + // Note: InferFiltersFromConstraints will add more filters to this given filters + filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) + } + } + } + } runBenchmark(s"Predicate conversion benchmark with unbalanced Expression") { val numRows = 1 From d37ffc0cecf5c0d7235493827c5315b128a7bd81 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 8 Mar 2019 13:16:13 +0200 Subject: [PATCH 14/56] Add comments everywhere --- .../datasources/orc/OrcFilters.scala | 65 +++++++++++++++---- 1 file changed, 53 insertions(+), 12 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index ee4555b1fe4e..6d411b67aad0 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -32,8 +32,8 @@ import org.apache.spark.sql.types._ /** * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. * - * Due to limitation of ORC `SearchArgument` builder, we had to end up with a pretty weird double- - * checking pattern when converting `And`/`Or`/`Not` filters. + * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and + * conversion code paths to make sure we only convert predicates that are known to be convertible. * * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite @@ -42,18 +42,18 @@ import org.apache.spark.sql.types._ * * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and * `startNot()` mutate internal state of the builder instance. This forces us to translate all - * convertible filters with a single builder instance. However, before actually converting a filter, - * we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible filter is - * found, we may already end up with a builder whose internal state is inconsistent. + * convertible filters with a single builder instance. However, if we try to translate a filter + * before checking whether it can be converted or not, we may end up with a builder whose internal + * state is inconsistent in the case of an inconvertible filter. * * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then * try to convert its children. Say we convert `left` child successfully, but find that `right` * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent * now. * - * The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their - * children with brand new builders, and only do the actual conversion with the right builder - * instance when the children are proven to be convertible. + * The workaround employed here is that, for `And`/`Or`/`Not`, we explicitly check if the children + * are convertible, and only do the actual conversion when the children are proven to be + * convertible. * * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of * builder methods mentioned above can only be found in test code, where all tested filters are @@ -310,6 +310,30 @@ private[sql] object OrcFilters extends OrcFiltersBase { } } +/** + * This case class represents a position in a `Filter` tree, paired with information about + * whether `AND` predicates can be partially pushed down. It is used as a key in the + * memoization map for `OrcConvertibilityChecker`. + * + * Because of the way this class is used, there are a few subtleties in its implementation + * and behaviour: + * - The default generated `hashCode` and `equals` methods have linear complexity in the size of + * the `expression` Filter. This is because they end up recursing over the whole `Filter` tree + * in order to check for equality, in order to return true for `Filter`s with different + * identities but the same actual contents. As a result, using this class as a key in a HashMap + * with the default generated methods would end up having a complexity that's linear in the + * size of the `expression`, instead of constnant. This means that using it as a key in the + * memoization map for `OrcConvertibilityChecker` would make the whole convertibility checking + * take time quadratic in the size of the tree instead of linear. + * - For this reason, we override `hashCode` and `equals` to only check the identity of the + * `expression` and not the full expression object. + * - Hashing based on the identity of the expression results in correct behaviour because we + * are indeed only interested in memoizing the results for exact locations in the `Filter` + * tree. If there are expressions that are exactly the same but at different locations, we + * would treat them as different objects and simply compute the results for them twice. + * - Since implementing `equals` and `hashCode` in the presence of subclasses is trickier, + * this class is sealed to guarantee that we don't need to be concerned with subclassing. + */ private sealed case class FilterWithConjunctPushdown( expression: Filter, canPartialPushDownConjuncts: Boolean @@ -332,14 +356,33 @@ private sealed case class FilterWithConjunctPushdown( /** * Helper class for efficiently checking whether a `Filter` and its children can be converted to - * ORC `SearchArgument`s. + * ORC `SearchArgument`s. The `isConvertible` method has a constant amortized complexity for + * checking whether a node from a Filter expression is convertible to an ORC `SearchArgument`. + * The additional memory is O(N) in the size of the whole `Filter` being checked due to the use + * of a hash map that possibly has an entry for each subnode in the filter. * - * @param dataTypeMap + * @param dataTypeMap a map from the attribute name to its data type. */ private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { private val convertibilityCache = new mutable.HashMap[FilterWithConjunctPushdown, Boolean] + /** + * Checks if a given expression from a Filter is convertible. + * + * The result for a given (expression, canPartialPushDownConjuncts) pair within a Filter + * will never change, so we memoize the results in the `convertibilityCache`. This means + * that the overall complexity for checking all nodes in a `Filter` is: + * - a total of one pass across the whole Filter tree is made when nodes are checked for + * convertibility for the first time. + * - when checked for a second, etc. time, the result has already been memoized in the cache + * and thus the check has a constant time. + * + * @param expression the input filter predicates. + * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed + * down safely. Pushing ONLY one side of AND down is safe to + * do at the top level or none of its ancestors is NOT and OR. + */ def isConvertible(expression: Filter, canPartialPushDownConjuncts: Boolean): Boolean = { val node = FilterWithConjunctPushdown(expression, canPartialPushDownConjuncts) convertibilityCache.getOrElseUpdate(node, isConvertibleImpl(node)) @@ -348,8 +391,6 @@ private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { /** * This method duplicates the logic from `OrcFilters.createBuilder` that is related to checking * if a given part of the filter is actually convertible to an ORC `SearchArgument`. - * @param node - * @return */ private def isConvertibleImpl(node: FilterWithConjunctPushdown): Boolean = { import org.apache.spark.sql.sources._ From eb1b0c1b0b57f51927cc104f31842f85f6abeed0 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 12 Mar 2019 12:38:13 +0200 Subject: [PATCH 15/56] Only run benchmarks for a few pre-specified values --- .../sql/execution/benchmark/FilterPushdownBenchmark.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 0046c2bb88f2..6eba8ae4e66e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -417,7 +417,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val columns = (1 to width).map(i => s"id c$i") val df = spark.range(1).selectExpr(columns: _*) - (1000 to 15000 by 2000).foreach { numFilter => + Seq(25, 5000, 15000).foreach { numFilter => val whereExpression = (1 to numFilter) .map { i => @@ -450,7 +450,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val df = spark.range(1).selectExpr(columns: _*) withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) - (1 to 1001 by 200).foreach { numFilter => + Seq(25, 500, 1000).foreach { numFilter => val whereColumn = (1 to numFilter) .map(i => col("c1") === lit(i)) .foldLeft(lit(false))(_ || _) From 1e2651a9c109ce24ec3d4d36a24cbf123acf8deb Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 12 Mar 2019 12:39:41 +0200 Subject: [PATCH 16/56] Add new benchmark results --- .../FilterPushdownBenchmark-results.txt | 1293 +++++++++-------- 1 file changed, 671 insertions(+), 622 deletions(-) diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt index e680ddff53dd..fba9b4e7a495 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt @@ -2,669 +2,718 @@ Pushdown for many distinct value case ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11405 / 11485 1.4 725.1 1.0X -Parquet Vectorized (Pushdown) 675 / 690 23.3 42.9 16.9X -Native ORC Vectorized 7127 / 7170 2.2 453.1 1.6X -Native ORC Vectorized (Pushdown) 519 / 541 30.3 33.0 22.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 string row ('7864320' < value < '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11457 / 11473 1.4 728.4 1.0X -Parquet Vectorized (Pushdown) 656 / 686 24.0 41.7 17.5X -Native ORC Vectorized 7328 / 7342 2.1 465.9 1.6X -Native ORC Vectorized (Pushdown) 539 / 565 29.2 34.2 21.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 string row (value = '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11878 / 11888 1.3 755.2 1.0X -Parquet Vectorized (Pushdown) 630 / 654 25.0 40.1 18.9X -Native ORC Vectorized 7342 / 7362 2.1 466.8 1.6X -Native ORC Vectorized (Pushdown) 519 / 537 30.3 33.0 22.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 string row (value <=> '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11423 / 11440 1.4 726.2 1.0X -Parquet Vectorized (Pushdown) 625 / 643 25.2 39.7 18.3X -Native ORC Vectorized 7315 / 7335 2.2 465.1 1.6X -Native ORC Vectorized (Pushdown) 507 / 520 31.0 32.2 22.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 string row ('7864320' <= value <= '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11440 / 11478 1.4 727.3 1.0X -Parquet Vectorized (Pushdown) 634 / 652 24.8 40.3 18.0X -Native ORC Vectorized 7311 / 7324 2.2 464.8 1.6X -Native ORC Vectorized (Pushdown) 517 / 548 30.4 32.8 22.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 20750 / 20872 0.8 1319.3 1.0X -Parquet Vectorized (Pushdown) 21002 / 21032 0.7 1335.3 1.0X -Native ORC Vectorized 16714 / 16742 0.9 1062.6 1.2X -Native ORC Vectorized (Pushdown) 16926 / 16965 0.9 1076.1 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 int row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10510 / 10532 1.5 668.2 1.0X -Parquet Vectorized (Pushdown) 642 / 665 24.5 40.8 16.4X -Native ORC Vectorized 6609 / 6618 2.4 420.2 1.6X -Native ORC Vectorized (Pushdown) 502 / 512 31.4 31.9 21.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 int row (7864320 < value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10505 / 10514 1.5 667.9 1.0X -Parquet Vectorized (Pushdown) 659 / 673 23.9 41.9 15.9X -Native ORC Vectorized 6634 / 6641 2.4 421.8 1.6X -Native ORC Vectorized (Pushdown) 513 / 526 30.7 32.6 20.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10555 / 10570 1.5 671.1 1.0X -Parquet Vectorized (Pushdown) 651 / 668 24.2 41.4 16.2X -Native ORC Vectorized 6721 / 6728 2.3 427.3 1.6X -Native ORC Vectorized (Pushdown) 508 / 519 31.0 32.3 20.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (value <=> 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10556 / 10566 1.5 671.1 1.0X -Parquet Vectorized (Pushdown) 647 / 654 24.3 41.1 16.3X -Native ORC Vectorized 6716 / 6728 2.3 427.0 1.6X -Native ORC Vectorized (Pushdown) 510 / 521 30.9 32.4 20.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (7864320 <= value <= 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10556 / 10565 1.5 671.1 1.0X -Parquet Vectorized (Pushdown) 649 / 654 24.2 41.3 16.3X -Native ORC Vectorized 6700 / 6712 2.3 426.0 1.6X -Native ORC Vectorized (Pushdown) 509 / 520 30.9 32.3 20.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 int row (7864319 < value < 7864321): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10547 / 10566 1.5 670.5 1.0X -Parquet Vectorized (Pushdown) 649 / 653 24.2 41.3 16.3X -Native ORC Vectorized 6703 / 6713 2.3 426.2 1.6X -Native ORC Vectorized (Pushdown) 510 / 520 30.8 32.5 20.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% int rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11478 / 11525 1.4 729.7 1.0X -Parquet Vectorized (Pushdown) 2576 / 2587 6.1 163.8 4.5X -Native ORC Vectorized 7633 / 7657 2.1 485.3 1.5X -Native ORC Vectorized (Pushdown) 2076 / 2096 7.6 132.0 5.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% int rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14785 / 14802 1.1 940.0 1.0X -Parquet Vectorized (Pushdown) 9971 / 9977 1.6 633.9 1.5X -Native ORC Vectorized 11082 / 11107 1.4 704.6 1.3X -Native ORC Vectorized (Pushdown) 8061 / 8073 2.0 512.5 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% int rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 18174 / 18214 0.9 1155.5 1.0X -Parquet Vectorized (Pushdown) 17387 / 17403 0.9 1105.5 1.0X -Native ORC Vectorized 14465 / 14492 1.1 919.7 1.3X -Native ORC Vectorized (Pushdown) 14024 / 14041 1.1 891.6 1.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all int rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19004 / 19014 0.8 1208.2 1.0X -Parquet Vectorized (Pushdown) 19219 / 19232 0.8 1221.9 1.0X -Native ORC Vectorized 15266 / 15290 1.0 970.6 1.2X -Native ORC Vectorized (Pushdown) 15469 / 15482 1.0 983.5 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all int rows (value > -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19036 / 19052 0.8 1210.3 1.0X -Parquet Vectorized (Pushdown) 19287 / 19306 0.8 1226.2 1.0X -Native ORC Vectorized 15311 / 15371 1.0 973.5 1.2X -Native ORC Vectorized (Pushdown) 15517 / 15590 1.0 986.5 1.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all int rows (value != -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19072 / 19102 0.8 1212.6 1.0X -Parquet Vectorized (Pushdown) 19288 / 19318 0.8 1226.3 1.0X -Native ORC Vectorized 15277 / 15293 1.0 971.3 1.2X -Native ORC Vectorized (Pushdown) 15479 / 15499 1.0 984.1 1.2X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6509 6563 64 2.4 413.8 1.0X +Parquet Vectorized (Pushdown) 451 455 5 34.9 28.7 14.4X +Native ORC Vectorized 4697 4880 311 3.3 298.6 1.4X +Native ORC Vectorized (Pushdown) 572 585 12 27.5 36.3 11.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6597 6624 21 2.4 419.4 1.0X +Parquet Vectorized (Pushdown) 453 456 2 34.7 28.8 14.6X +Native ORC Vectorized 4853 4887 29 3.2 308.5 1.4X +Native ORC Vectorized (Pushdown) 572 582 13 27.5 36.3 11.5X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6597 6648 30 2.4 419.4 1.0X +Parquet Vectorized (Pushdown) 445 448 3 35.4 28.3 14.8X +Native ORC Vectorized 4915 4954 34 3.2 312.5 1.3X +Native ORC Vectorized (Pushdown) 560 574 14 28.1 35.6 11.8X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6598 6664 80 2.4 419.5 1.0X +Parquet Vectorized (Pushdown) 439 442 3 35.8 27.9 15.0X +Native ORC Vectorized 4894 4926 30 3.2 311.1 1.3X +Native ORC Vectorized (Pushdown) 561 572 13 28.0 35.7 11.8X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6607 6634 27 2.4 420.1 1.0X +Parquet Vectorized (Pushdown) 440 444 3 35.8 28.0 15.0X +Native ORC Vectorized 4910 4961 48 3.2 312.2 1.3X +Native ORC Vectorized (Pushdown) 564 575 13 27.9 35.9 11.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 12348 12442 63 1.3 785.1 1.0X +Parquet Vectorized (Pushdown) 12110 12211 96 1.3 769.9 1.0X +Native ORC Vectorized 10689 10772 59 1.5 679.6 1.2X +Native ORC Vectorized (Pushdown) 10926 10971 40 1.4 694.7 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5789 5870 64 2.7 368.1 1.0X +Parquet Vectorized (Pushdown) 356 361 3 44.2 22.6 16.3X +Native ORC Vectorized 4326 4515 303 3.6 275.1 1.3X +Native ORC Vectorized (Pushdown) 547 565 15 28.8 34.8 10.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5791 5837 60 2.7 368.2 1.0X +Parquet Vectorized (Pushdown) 364 373 6 43.2 23.2 15.9X +Native ORC Vectorized 4359 4398 28 3.6 277.1 1.3X +Native ORC Vectorized (Pushdown) 555 569 16 28.3 35.3 10.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5876 5916 25 2.7 373.6 1.0X +Parquet Vectorized (Pushdown) 362 367 4 43.4 23.0 16.2X +Native ORC Vectorized 4393 4453 44 3.6 279.3 1.3X +Native ORC Vectorized (Pushdown) 552 567 16 28.5 35.1 10.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5879 5898 16 2.7 373.8 1.0X +Parquet Vectorized (Pushdown) 359 369 9 43.8 22.8 16.4X +Native ORC Vectorized 4405 4441 30 3.6 280.0 1.3X +Native ORC Vectorized (Pushdown) 548 564 19 28.7 34.8 10.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5846 5930 75 2.7 371.7 1.0X +Parquet Vectorized (Pushdown) 363 372 6 43.4 23.1 16.1X +Native ORC Vectorized 4425 4456 23 3.6 281.3 1.3X +Native ORC Vectorized (Pushdown) 551 572 24 28.6 35.0 10.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5835 5888 43 2.7 370.9 1.0X +Parquet Vectorized (Pushdown) 363 368 3 43.3 23.1 16.1X +Native ORC Vectorized 4426 4445 24 3.6 281.4 1.3X +Native ORC Vectorized (Pushdown) 547 563 16 28.7 34.8 10.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6470 6540 48 2.4 411.4 1.0X +Parquet Vectorized (Pushdown) 1548 1570 16 10.2 98.4 4.2X +Native ORC Vectorized 5078 5106 22 3.1 322.9 1.3X +Native ORC Vectorized (Pushdown) 1625 1641 11 9.7 103.3 4.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8568 8628 45 1.8 544.7 1.0X +Parquet Vectorized (Pushdown) 5826 5891 54 2.7 370.4 1.5X +Native ORC Vectorized 7233 7254 18 2.2 459.8 1.2X +Native ORC Vectorized (Pushdown) 5447 5481 31 2.9 346.3 1.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10653 10701 48 1.5 677.3 1.0X +Parquet Vectorized (Pushdown) 10210 10244 40 1.5 649.1 1.0X +Native ORC Vectorized 9398 9441 32 1.7 597.5 1.1X +Native ORC Vectorized (Pushdown) 9271 9331 56 1.7 589.4 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11098 11146 39 1.4 705.6 1.0X +Parquet Vectorized (Pushdown) 11187 11254 45 1.4 711.2 1.0X +Native ORC Vectorized 9847 9895 43 1.6 626.0 1.1X +Native ORC Vectorized (Pushdown) 10227 12071 623 1.5 650.2 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11873 14239 NaN 1.3 754.8 1.0X +Parquet Vectorized (Pushdown) 11854 11911 36 1.3 753.7 1.0X +Native ORC Vectorized 10197 10482 397 1.5 648.3 1.2X +Native ORC Vectorized (Pushdown) 10450 10471 16 1.5 664.4 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 11715 11776 53 1.3 744.8 1.0X +Parquet Vectorized (Pushdown) 12178 15502 NaN 1.3 774.2 1.0X +Native ORC Vectorized 10196 10256 62 1.5 648.2 1.1X +Native ORC Vectorized (Pushdown) 10448 10479 21 1.5 664.3 1.1X ================================================================================================ Pushdown for few distinct value case (use dictionary encoding) ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 distinct string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10250 / 10274 1.5 651.7 1.0X -Parquet Vectorized (Pushdown) 571 / 576 27.5 36.3 17.9X -Native ORC Vectorized 8651 / 8660 1.8 550.0 1.2X -Native ORC Vectorized (Pushdown) 909 / 933 17.3 57.8 11.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 0 distinct string row ('100' < value < '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10420 / 10426 1.5 662.5 1.0X -Parquet Vectorized (Pushdown) 574 / 579 27.4 36.5 18.2X -Native ORC Vectorized 8973 / 8982 1.8 570.5 1.2X -Native ORC Vectorized (Pushdown) 916 / 955 17.2 58.2 11.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 distinct string row (value = '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10428 / 10441 1.5 663.0 1.0X -Parquet Vectorized (Pushdown) 789 / 809 19.9 50.2 13.2X -Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X -Native ORC Vectorized (Pushdown) 1130 / 1145 13.9 71.8 9.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 distinct string row (value <=> '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10402 / 10416 1.5 661.3 1.0X -Parquet Vectorized (Pushdown) 791 / 806 19.9 50.3 13.2X -Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X -Native ORC Vectorized (Pushdown) 1112 / 1145 14.1 70.7 9.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 distinct string row ('100' <= value <= '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10548 / 10563 1.5 670.6 1.0X -Parquet Vectorized (Pushdown) 790 / 796 19.9 50.2 13.4X -Native ORC Vectorized 9144 / 9153 1.7 581.3 1.2X -Native ORC Vectorized (Pushdown) 1117 / 1148 14.1 71.0 9.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select all distinct string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 20445 / 20469 0.8 1299.8 1.0X -Parquet Vectorized (Pushdown) 20686 / 20699 0.8 1315.2 1.0X -Native ORC Vectorized 18851 / 18953 0.8 1198.5 1.1X -Native ORC Vectorized (Pushdown) 19255 / 19268 0.8 1224.2 1.1X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5997 6029 23 2.6 381.3 1.0X +Parquet Vectorized (Pushdown) 328 336 7 47.9 20.9 18.3X +Native ORC Vectorized 5886 6011 109 2.7 374.2 1.0X +Native ORC Vectorized (Pushdown) 1086 1111 22 14.5 69.1 5.5X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6097 6139 45 2.6 387.6 1.0X +Parquet Vectorized (Pushdown) 331 342 6 47.5 21.1 18.4X +Native ORC Vectorized 6018 6070 33 2.6 382.6 1.0X +Native ORC Vectorized (Pushdown) 1084 1099 14 14.5 68.9 5.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6175 6202 26 2.5 392.6 1.0X +Parquet Vectorized (Pushdown) 474 488 10 33.2 30.1 13.0X +Native ORC Vectorized 6236 6270 41 2.5 396.5 1.0X +Native ORC Vectorized (Pushdown) 1203 1226 18 13.1 76.5 5.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6192 7882 704 2.5 393.7 1.0X +Parquet Vectorized (Pushdown) 511 769 265 30.8 32.5 12.1X +Native ORC Vectorized 6592 7214 441 2.4 419.1 0.9X +Native ORC Vectorized (Pushdown) 1306 1446 124 12.0 83.0 4.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6708 7325 686 2.3 426.5 1.0X +Parquet Vectorized (Pushdown) 482 630 164 32.7 30.6 13.9X +Native ORC Vectorized 6645 6828 109 2.4 422.5 1.0X +Native ORC Vectorized (Pushdown) 1268 1432 208 12.4 80.6 5.3X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 14204 15124 863 1.1 903.1 1.0X +Parquet Vectorized (Pushdown) 13976 15133 1013 1.1 888.6 1.0X +Native ORC Vectorized 14969 15811 1277 1.1 951.7 0.9X +Native ORC Vectorized (Pushdown) 15815 17290 1437 1.0 1005.5 0.9X ================================================================================================ Pushdown benchmark for StringStartsWith ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -StringStartsWith filter: (value like '10%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14265 / 15213 1.1 907.0 1.0X -Parquet Vectorized (Pushdown) 4228 / 4870 3.7 268.8 3.4X -Native ORC Vectorized 10116 / 10977 1.6 643.2 1.4X -Native ORC Vectorized (Pushdown) 10653 / 11376 1.5 677.3 1.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -StringStartsWith filter: (value like '1000%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11499 / 11539 1.4 731.1 1.0X -Parquet Vectorized (Pushdown) 669 / 672 23.5 42.5 17.2X -Native ORC Vectorized 7343 / 7363 2.1 466.8 1.6X -Native ORC Vectorized (Pushdown) 7559 / 7568 2.1 480.6 1.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -StringStartsWith filter: (value like '786432%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11463 / 11468 1.4 728.8 1.0X -Parquet Vectorized (Pushdown) 647 / 651 24.3 41.1 17.7X -Native ORC Vectorized 7322 / 7338 2.1 465.5 1.6X -Native ORC Vectorized (Pushdown) 7533 / 7544 2.1 478.9 1.5X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 10327 10682 540 1.5 656.5 1.0X +Parquet Vectorized (Pushdown) 2991 3328 491 5.3 190.1 3.5X +Native ORC Vectorized 7941 8369 323 2.0 504.9 1.3X +Native ORC Vectorized (Pushdown) 8354 8651 450 1.9 531.1 1.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8599 8687 56 1.8 546.7 1.0X +Parquet Vectorized (Pushdown) 481 492 11 32.7 30.6 17.9X +Native ORC Vectorized 6168 6224 43 2.5 392.2 1.4X +Native ORC Vectorized (Pushdown) 6386 6469 73 2.5 406.0 1.3X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8628 8670 58 1.8 548.6 1.0X +Parquet Vectorized (Pushdown) 463 477 12 34.0 29.4 18.6X +Native ORC Vectorized 6151 6195 38 2.6 391.1 1.4X +Native ORC Vectorized (Pushdown) 6401 6432 30 2.5 407.0 1.3X ================================================================================================ Pushdown benchmark for decimal ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 decimal(9, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5543 / 5564 2.8 352.4 1.0X -Parquet Vectorized (Pushdown) 168 / 174 93.7 10.7 33.0X -Native ORC Vectorized 4992 / 5052 3.2 317.4 1.1X -Native ORC Vectorized (Pushdown) 840 / 850 18.7 53.4 6.6X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% decimal(9, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 7312 / 7358 2.2 464.9 1.0X -Parquet Vectorized (Pushdown) 3008 / 3078 5.2 191.2 2.4X -Native ORC Vectorized 6775 / 6798 2.3 430.7 1.1X -Native ORC Vectorized (Pushdown) 6819 / 6832 2.3 433.5 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% decimal(9, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 13232 / 13241 1.2 841.3 1.0X -Parquet Vectorized (Pushdown) 12555 / 12569 1.3 798.2 1.1X -Native ORC Vectorized 12597 / 12627 1.2 800.9 1.1X -Native ORC Vectorized (Pushdown) 12677 / 12711 1.2 806.0 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% decimal(9, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14725 / 14729 1.1 936.2 1.0X -Parquet Vectorized (Pushdown) 14781 / 14800 1.1 939.7 1.0X -Native ORC Vectorized 15360 / 15453 1.0 976.5 1.0X -Native ORC Vectorized (Pushdown) 15444 / 15466 1.0 981.9 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 decimal(18, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5746 / 5763 2.7 365.3 1.0X -Parquet Vectorized (Pushdown) 166 / 169 94.8 10.6 34.6X -Native ORC Vectorized 5007 / 5023 3.1 318.3 1.1X -Native ORC Vectorized (Pushdown) 2629 / 2640 6.0 167.1 2.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% decimal(18, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6827 / 6864 2.3 434.0 1.0X -Parquet Vectorized (Pushdown) 1809 / 1827 8.7 115.0 3.8X -Native ORC Vectorized 6287 / 6296 2.5 399.7 1.1X -Native ORC Vectorized (Pushdown) 6364 / 6377 2.5 404.6 1.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% decimal(18, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 11315 / 11342 1.4 719.4 1.0X -Parquet Vectorized (Pushdown) 8431 / 8450 1.9 536.0 1.3X -Native ORC Vectorized 11591 / 11611 1.4 736.9 1.0X -Native ORC Vectorized (Pushdown) 11424 / 11475 1.4 726.3 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% decimal(18, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 15703 / 15712 1.0 998.4 1.0X -Parquet Vectorized (Pushdown) 14982 / 15009 1.0 952.5 1.0X -Native ORC Vectorized 16887 / 16955 0.9 1073.7 0.9X -Native ORC Vectorized (Pushdown) 16518 / 16530 1.0 1050.2 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 decimal(38, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 8101 / 8130 1.9 515.1 1.0X -Parquet Vectorized (Pushdown) 184 / 187 85.6 11.7 44.1X -Native ORC Vectorized 4998 / 5027 3.1 317.8 1.6X -Native ORC Vectorized (Pushdown) 165 / 168 95.6 10.5 49.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% decimal(38, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 9405 / 9447 1.7 597.9 1.0X -Parquet Vectorized (Pushdown) 2269 / 2275 6.9 144.2 4.1X -Native ORC Vectorized 6167 / 6203 2.6 392.1 1.5X -Native ORC Vectorized (Pushdown) 1783 / 1787 8.8 113.3 5.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% decimal(38, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14700 / 14707 1.1 934.6 1.0X -Parquet Vectorized (Pushdown) 10699 / 10712 1.5 680.2 1.4X -Native ORC Vectorized 10687 / 10703 1.5 679.5 1.4X -Native ORC Vectorized (Pushdown) 8364 / 8415 1.9 531.8 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% decimal(38, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 19780 / 19894 0.8 1257.6 1.0X -Parquet Vectorized (Pushdown) 19003 / 19025 0.8 1208.1 1.0X -Native ORC Vectorized 15385 / 15404 1.0 978.2 1.3X -Native ORC Vectorized (Pushdown) 15032 / 15060 1.0 955.7 1.3X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3178 3238 56 4.9 202.0 1.0X +Parquet Vectorized (Pushdown) 102 105 3 154.8 6.5 31.3X +Native ORC Vectorized 3224 3270 33 4.9 205.0 1.0X +Native ORC Vectorized (Pushdown) 603 614 9 26.1 38.3 5.3X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4309 4388 54 3.6 274.0 1.0X +Parquet Vectorized (Pushdown) 1894 1904 11 8.3 120.4 2.3X +Native ORC Vectorized 4400 4446 39 3.6 279.7 1.0X +Native ORC Vectorized (Pushdown) 4503 4515 7 3.5 286.3 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8049 8105 54 2.0 511.7 1.0X +Parquet Vectorized (Pushdown) 7630 7665 25 2.1 485.1 1.1X +Native ORC Vectorized 8316 8392 45 1.9 528.7 1.0X +Native ORC Vectorized (Pushdown) 8501 8529 28 1.9 540.5 0.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8846 8921 79 1.8 562.4 1.0X +Parquet Vectorized (Pushdown) 8859 8918 45 1.8 563.2 1.0X +Native ORC Vectorized 9275 9314 28 1.7 589.7 1.0X +Native ORC Vectorized (Pushdown) 9407 9427 13 1.7 598.1 0.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3198 3226 35 4.9 203.3 1.0X +Parquet Vectorized (Pushdown) 101 106 4 155.3 6.4 31.6X +Native ORC Vectorized 3242 3272 30 4.9 206.2 1.0X +Native ORC Vectorized (Pushdown) 1729 1760 19 9.1 109.9 1.8X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3829 3855 23 4.1 243.4 1.0X +Parquet Vectorized (Pushdown) 1013 1027 18 15.5 64.4 3.8X +Native ORC Vectorized 3833 3852 21 4.1 243.7 1.0X +Native ORC Vectorized (Pushdown) 3925 3964 38 4.0 249.5 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6150 6195 48 2.6 391.0 1.0X +Parquet Vectorized (Pushdown) 4568 4609 24 3.4 290.4 1.3X +Native ORC Vectorized 6248 6283 29 2.5 397.2 1.0X +Native ORC Vectorized (Pushdown) 6358 6376 22 2.5 404.2 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8371 8452 46 1.9 532.2 1.0X +Parquet Vectorized (Pushdown) 8109 8134 25 1.9 515.6 1.0X +Native ORC Vectorized 8629 8703 72 1.8 548.6 1.0X +Native ORC Vectorized (Pushdown) 8722 8777 39 1.8 554.6 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4512 4546 23 3.5 286.9 1.0X +Parquet Vectorized (Pushdown) 109 113 3 143.7 7.0 41.2X +Native ORC Vectorized 3297 3319 16 4.8 209.6 1.4X +Native ORC Vectorized (Pushdown) 175 182 6 89.9 11.1 25.8X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5363 5389 25 2.9 340.9 1.0X +Parquet Vectorized (Pushdown) 1320 1342 18 11.9 83.9 4.1X +Native ORC Vectorized 4034 4057 18 3.9 256.5 1.3X +Native ORC Vectorized (Pushdown) 1225 1235 10 12.8 77.9 4.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8240 8267 29 1.9 523.9 1.0X +Parquet Vectorized (Pushdown) 7615 8290 727 2.1 484.1 1.1X +Native ORC Vectorized 8604 8654 53 1.8 547.0 1.0X +Native ORC Vectorized (Pushdown) 6619 6664 27 2.4 420.8 1.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 13919 14010 84 1.1 884.9 1.0X +Parquet Vectorized (Pushdown) 13455 13501 54 1.2 855.4 1.0X +Native ORC Vectorized 12117 12164 72 1.3 770.4 1.1X +Native ORC Vectorized (Pushdown) 11737 11801 37 1.3 746.2 1.2X ================================================================================================ Pushdown benchmark for InSet -> InFilters ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 5, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10521 / 10534 1.5 668.9 1.0X -Parquet Vectorized (Pushdown) 677 / 691 23.2 43.1 15.5X -Native ORC Vectorized 6768 / 6776 2.3 430.3 1.6X -Native ORC Vectorized (Pushdown) 501 / 512 31.4 31.8 21.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 5, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10531 / 10538 1.5 669.5 1.0X -Parquet Vectorized (Pushdown) 677 / 718 23.2 43.0 15.6X -Native ORC Vectorized 6765 / 6773 2.3 430.1 1.6X -Native ORC Vectorized (Pushdown) 499 / 507 31.5 31.7 21.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 5, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10540 / 10553 1.5 670.1 1.0X -Parquet Vectorized (Pushdown) 678 / 710 23.2 43.1 15.5X -Native ORC Vectorized 6787 / 6794 2.3 431.5 1.6X -Native ORC Vectorized (Pushdown) 501 / 509 31.4 31.9 21.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 10, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10551 / 10559 1.5 670.8 1.0X -Parquet Vectorized (Pushdown) 703 / 708 22.4 44.7 15.0X -Native ORC Vectorized 6791 / 6802 2.3 431.7 1.6X -Native ORC Vectorized (Pushdown) 519 / 526 30.3 33.0 20.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 10, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10561 / 10565 1.5 671.4 1.0X -Parquet Vectorized (Pushdown) 711 / 716 22.1 45.2 14.9X -Native ORC Vectorized 6791 / 6806 2.3 431.8 1.6X -Native ORC Vectorized (Pushdown) 529 / 537 29.8 33.6 20.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 10, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10572 / 10590 1.5 672.1 1.0X -Parquet Vectorized (Pushdown) 713 / 716 22.1 45.3 14.8X -Native ORC Vectorized 6808 / 6815 2.3 432.9 1.6X -Native ORC Vectorized (Pushdown) 530 / 541 29.7 33.7 19.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 50, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10871 / 10882 1.4 691.2 1.0X -Parquet Vectorized (Pushdown) 11104 / 11110 1.4 706.0 1.0X -Native ORC Vectorized 7088 / 7104 2.2 450.7 1.5X -Native ORC Vectorized (Pushdown) 665 / 677 23.6 42.3 16.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 50, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10861 / 10867 1.4 690.5 1.0X -Parquet Vectorized (Pushdown) 11094 / 11099 1.4 705.3 1.0X -Native ORC Vectorized 7075 / 7092 2.2 449.8 1.5X -Native ORC Vectorized (Pushdown) 718 / 733 21.9 45.6 15.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 50, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10868 / 10887 1.4 691.0 1.0X -Parquet Vectorized (Pushdown) 11100 / 11106 1.4 705.7 1.0X -Native ORC Vectorized 7087 / 7093 2.2 450.6 1.5X -Native ORC Vectorized (Pushdown) 712 / 731 22.1 45.3 15.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 100, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10850 / 10888 1.4 689.8 1.0X -Parquet Vectorized (Pushdown) 11086 / 11105 1.4 704.9 1.0X -Native ORC Vectorized 7090 / 7101 2.2 450.8 1.5X -Native ORC Vectorized (Pushdown) 867 / 882 18.1 55.1 12.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 100, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10816 / 10819 1.5 687.7 1.0X -Parquet Vectorized (Pushdown) 11052 / 11059 1.4 702.7 1.0X -Native ORC Vectorized 7037 / 7044 2.2 447.4 1.5X -Native ORC Vectorized (Pushdown) 919 / 931 17.1 58.4 11.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -InSet -> InFilters (values count: 100, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10807 / 10815 1.5 687.1 1.0X -Parquet Vectorized (Pushdown) 11047 / 11054 1.4 702.4 1.0X -Native ORC Vectorized 7042 / 7047 2.2 447.7 1.5X -Native ORC Vectorized (Pushdown) 950 / 961 16.6 60.4 11.4X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 7961 8018 63 2.0 506.1 1.0X +Parquet Vectorized (Pushdown) 475 496 18 33.1 30.2 16.7X +Native ORC Vectorized 4700 5828 887 3.3 298.8 1.7X +Native ORC Vectorized (Pushdown) 576 593 19 27.3 36.6 13.8X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5976 6101 93 2.6 380.0 1.0X +Parquet Vectorized (Pushdown) 377 384 5 41.7 24.0 15.9X +Native ORC Vectorized 4535 4573 28 3.5 288.3 1.3X +Native ORC Vectorized (Pushdown) 557 575 19 28.2 35.4 10.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5888 5962 49 2.7 374.3 1.0X +Parquet Vectorized (Pushdown) 373 383 12 42.2 23.7 15.8X +Native ORC Vectorized 4525 4561 31 3.5 287.7 1.3X +Native ORC Vectorized (Pushdown) 563 575 15 27.9 35.8 10.5X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 5883 5916 28 2.7 374.0 1.0X +Parquet Vectorized (Pushdown) 394 402 7 39.9 25.1 14.9X +Native ORC Vectorized 4561 6094 1481 3.4 290.0 1.3X +Native ORC Vectorized (Pushdown) 579 618 34 27.1 36.8 10.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6225 6538 651 2.5 395.8 1.0X +Parquet Vectorized (Pushdown) 415 422 7 37.9 26.4 15.0X +Native ORC Vectorized 4652 4698 41 3.4 295.8 1.3X +Native ORC Vectorized (Pushdown) 581 592 14 27.1 36.9 10.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6226 6249 19 2.5 395.9 1.0X +Parquet Vectorized (Pushdown) 412 418 4 38.1 26.2 15.1X +Native ORC Vectorized 4657 4690 24 3.4 296.1 1.3X +Native ORC Vectorized (Pushdown) 576 588 15 27.3 36.6 10.8X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6459 6476 18 2.4 410.6 1.0X +Parquet Vectorized (Pushdown) 6574 6600 23 2.4 418.0 1.0X +Native ORC Vectorized 4884 4933 37 3.2 310.5 1.3X +Native ORC Vectorized (Pushdown) 688 700 8 22.9 43.7 9.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6432 6504 62 2.4 408.9 1.0X +Parquet Vectorized (Pushdown) 6624 6657 32 2.4 421.1 1.0X +Native ORC Vectorized 4910 4950 28 3.2 312.2 1.3X +Native ORC Vectorized (Pushdown) 696 709 9 22.6 44.3 9.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6453 6496 30 2.4 410.2 1.0X +Parquet Vectorized (Pushdown) 6581 6614 26 2.4 418.4 1.0X +Native ORC Vectorized 4900 4965 53 3.2 311.5 1.3X +Native ORC Vectorized (Pushdown) 706 716 7 22.3 44.9 9.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6421 6450 26 2.4 408.2 1.0X +Parquet Vectorized (Pushdown) 6570 6586 17 2.4 417.7 1.0X +Native ORC Vectorized 4885 4911 29 3.2 310.6 1.3X +Native ORC Vectorized (Pushdown) 765 785 14 20.6 48.6 8.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6171 6261 107 2.5 392.3 1.0X +Parquet Vectorized (Pushdown) 6269 6324 36 2.5 398.6 1.0X +Native ORC Vectorized 4782 4806 18 3.3 304.1 1.3X +Native ORC Vectorized (Pushdown) 829 838 10 19.0 52.7 7.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6142 6190 56 2.6 390.5 1.0X +Parquet Vectorized (Pushdown) 6253 6287 29 2.5 397.6 1.0X +Native ORC Vectorized 4787 4834 43 3.3 304.4 1.3X +Native ORC Vectorized (Pushdown) 813 836 17 19.4 51.7 7.6X ================================================================================================ Pushdown benchmark for tinyint ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 tinyint row (value = CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6034 / 6048 2.6 383.6 1.0X -Parquet Vectorized (Pushdown) 333 / 344 47.2 21.2 18.1X -Native ORC Vectorized 3240 / 3307 4.9 206.0 1.9X -Native ORC Vectorized (Pushdown) 330 / 341 47.6 21.0 18.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6759 / 6800 2.3 429.7 1.0X -Parquet Vectorized (Pushdown) 1533 / 1537 10.3 97.5 4.4X -Native ORC Vectorized 3863 / 3874 4.1 245.6 1.7X -Native ORC Vectorized (Pushdown) 1235 / 1248 12.7 78.5 5.5X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10247 / 10289 1.5 651.5 1.0X -Parquet Vectorized (Pushdown) 7430 / 7453 2.1 472.4 1.4X -Native ORC Vectorized 6995 / 7009 2.2 444.7 1.5X -Native ORC Vectorized (Pushdown) 5561 / 5571 2.8 353.6 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 13949 / 13991 1.1 886.9 1.0X -Parquet Vectorized (Pushdown) 13486 / 13511 1.2 857.4 1.0X -Native ORC Vectorized 10149 / 10186 1.5 645.3 1.4X -Native ORC Vectorized (Pushdown) 9889 / 9905 1.6 628.7 1.4X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3402 3419 14 4.6 216.3 1.0X +Parquet Vectorized (Pushdown) 193 202 12 81.5 12.3 17.6X +Native ORC Vectorized 2106 2131 21 7.5 133.9 1.6X +Native ORC Vectorized (Pushdown) 280 289 8 56.2 17.8 12.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3893 3921 26 4.0 247.5 1.0X +Parquet Vectorized (Pushdown) 966 972 7 16.3 61.4 4.0X +Native ORC Vectorized 2604 2624 22 6.0 165.5 1.5X +Native ORC Vectorized (Pushdown) 927 942 13 17.0 59.0 4.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6048 6081 25 2.6 384.5 1.0X +Parquet Vectorized (Pushdown) 4388 4411 20 3.6 279.0 1.4X +Native ORC Vectorized 4714 4725 16 3.3 299.7 1.3X +Native ORC Vectorized (Pushdown) 4123 4386 338 3.8 262.1 1.5X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8462 9675 925 1.9 538.0 1.0X +Parquet Vectorized (Pushdown) 8124 8156 28 1.9 516.5 1.0X +Native ORC Vectorized 6884 6934 31 2.3 437.7 1.2X +Native ORC Vectorized (Pushdown) 6764 6793 26 2.3 430.1 1.3X ================================================================================================ Pushdown benchmark for Timestamp ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6307 / 6310 2.5 401.0 1.0X -Parquet Vectorized (Pushdown) 6360 / 6397 2.5 404.3 1.0X -Native ORC Vectorized 2912 / 2917 5.4 185.1 2.2X -Native ORC Vectorized (Pushdown) 138 / 141 114.4 8.7 45.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 7225 / 7233 2.2 459.4 1.0X -Parquet Vectorized (Pushdown) 7250 / 7255 2.2 461.0 1.0X -Native ORC Vectorized 3772 / 3783 4.2 239.8 1.9X -Native ORC Vectorized (Pushdown) 1277 / 1282 12.3 81.2 5.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10952 / 10965 1.4 696.3 1.0X -Parquet Vectorized (Pushdown) 10985 / 10998 1.4 698.4 1.0X -Native ORC Vectorized 7178 / 7227 2.2 456.3 1.5X -Native ORC Vectorized (Pushdown) 5825 / 5830 2.7 370.3 1.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14560 / 14583 1.1 925.7 1.0X -Parquet Vectorized (Pushdown) 14608 / 14620 1.1 928.7 1.0X -Native ORC Vectorized 10601 / 10640 1.5 674.0 1.4X -Native ORC Vectorized (Pushdown) 10392 / 10406 1.5 660.7 1.4X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5653 / 5658 2.8 359.4 1.0X -Parquet Vectorized (Pushdown) 165 / 169 95.1 10.5 34.2X -Native ORC Vectorized 2918 / 2921 5.4 185.5 1.9X -Native ORC Vectorized (Pushdown) 137 / 145 114.9 8.7 41.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6540 / 6552 2.4 415.8 1.0X -Parquet Vectorized (Pushdown) 1610 / 1614 9.8 102.3 4.1X -Native ORC Vectorized 3775 / 3788 4.2 240.0 1.7X -Native ORC Vectorized (Pushdown) 1274 / 1277 12.3 81.0 5.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10259 / 10278 1.5 652.3 1.0X -Parquet Vectorized (Pushdown) 7591 / 7601 2.1 482.6 1.4X -Native ORC Vectorized 7185 / 7194 2.2 456.8 1.4X -Native ORC Vectorized (Pushdown) 5828 / 5843 2.7 370.6 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 13850 / 13868 1.1 880.5 1.0X -Parquet Vectorized (Pushdown) 13433 / 13450 1.2 854.0 1.0X -Native ORC Vectorized 10635 / 10669 1.5 676.1 1.3X -Native ORC Vectorized (Pushdown) 10437 / 10448 1.5 663.6 1.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 5884 / 5888 2.7 374.1 1.0X -Parquet Vectorized (Pushdown) 166 / 170 94.7 10.6 35.4X -Native ORC Vectorized 2913 / 2916 5.4 185.2 2.0X -Native ORC Vectorized (Pushdown) 136 / 144 115.4 8.7 43.2X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6763 / 6776 2.3 430.0 1.0X -Parquet Vectorized (Pushdown) 1634 / 1638 9.6 103.9 4.1X -Native ORC Vectorized 3777 / 3785 4.2 240.1 1.8X -Native ORC Vectorized (Pushdown) 1276 / 1279 12.3 81.2 5.3X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 10460 / 10469 1.5 665.0 1.0X -Parquet Vectorized (Pushdown) 7689 / 7698 2.0 488.9 1.4X -Native ORC Vectorized 7190 / 7197 2.2 457.1 1.5X -Native ORC Vectorized (Pushdown) 5820 / 5834 2.7 370.0 1.8X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 14033 / 14039 1.1 892.2 1.0X -Parquet Vectorized (Pushdown) 13608 / 13636 1.2 865.2 1.0X -Native ORC Vectorized 10635 / 10686 1.5 676.2 1.3X -Native ORC Vectorized (Pushdown) 10420 / 10442 1.5 662.5 1.3X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3712 3750 30 4.2 236.0 1.0X +Parquet Vectorized (Pushdown) 3739 3764 30 4.2 237.7 1.0X +Native ORC Vectorized 2064 2087 16 7.6 131.3 1.8X +Native ORC Vectorized (Pushdown) 147 153 7 106.6 9.4 25.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 4337 4376 42 3.6 275.8 1.0X +Parquet Vectorized (Pushdown) 4352 4376 19 3.6 276.7 1.0X +Native ORC Vectorized 2648 2668 16 5.9 168.4 1.6X +Native ORC Vectorized (Pushdown) 919 937 13 17.1 58.4 4.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6703 7302 1119 2.3 426.2 1.0X +Parquet Vectorized (Pushdown) 6687 6709 28 2.4 425.1 1.0X +Native ORC Vectorized 4842 4876 28 3.2 307.8 1.4X +Native ORC Vectorized (Pushdown) 3925 3945 11 4.0 249.6 1.7X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8894 8914 19 1.8 565.5 1.0X +Parquet Vectorized (Pushdown) 8930 8962 29 1.8 567.8 1.0X +Native ORC Vectorized 7086 7115 34 2.2 450.5 1.3X +Native ORC Vectorized (Pushdown) 6973 7018 44 2.3 443.3 1.3X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3282 3321 23 4.8 208.6 1.0X +Parquet Vectorized (Pushdown) 107 112 5 147.4 6.8 30.8X +Native ORC Vectorized 2046 2057 11 7.7 130.1 1.6X +Native ORC Vectorized (Pushdown) 146 152 9 107.5 9.3 22.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3922 3944 21 4.0 249.3 1.0X +Parquet Vectorized (Pushdown) 1033 1040 6 15.2 65.7 3.8X +Native ORC Vectorized 2659 2674 11 5.9 169.0 1.5X +Native ORC Vectorized (Pushdown) 921 938 10 17.1 58.6 4.3X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6254 6287 33 2.5 397.6 1.0X +Parquet Vectorized (Pushdown) 4673 4697 21 3.4 297.1 1.3X +Native ORC Vectorized 4884 4914 27 3.2 310.5 1.3X +Native ORC Vectorized (Pushdown) 3976 3996 14 4.0 252.8 1.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8269 8458 110 1.9 525.8 1.0X +Parquet Vectorized (Pushdown) 7961 7980 21 2.0 506.1 1.0X +Native ORC Vectorized 6993 7026 22 2.2 444.6 1.2X +Native ORC Vectorized (Pushdown) 6883 6928 73 2.3 437.6 1.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3357 3375 17 4.7 213.4 1.0X +Parquet Vectorized (Pushdown) 101 105 5 156.1 6.4 33.3X +Native ORC Vectorized 2003 2031 17 7.9 127.4 1.7X +Native ORC Vectorized (Pushdown) 146 151 8 108.1 9.3 23.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 3983 3997 11 3.9 253.2 1.0X +Parquet Vectorized (Pushdown) 1018 1034 16 15.4 64.7 3.9X +Native ORC Vectorized 2591 2611 15 6.1 164.8 1.5X +Native ORC Vectorized (Pushdown) 911 917 4 17.3 57.9 4.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 6218 6235 17 2.5 395.3 1.0X +Parquet Vectorized (Pushdown) 4571 4620 38 3.4 290.6 1.4X +Native ORC Vectorized 4799 4831 29 3.3 305.1 1.3X +Native ORC Vectorized (Pushdown) 3895 3915 14 4.0 247.7 1.6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 8381 8407 29 1.9 532.9 1.0X +Parquet Vectorized (Pushdown) 8058 8132 84 2.0 512.3 1.0X +Native ORC Vectorized 6990 7043 35 2.3 444.4 1.2X +Native ORC Vectorized (Pushdown) 6859 6897 24 2.3 436.1 1.2X ================================================================================================ Pushdown benchmark with many filters ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 row with 1 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 319 / 323 0.0 318789986.0 1.0X -Parquet Vectorized (Pushdown) 323 / 347 0.0 322755287.0 1.0X -Native ORC Vectorized 316 / 336 0.0 315670745.0 1.0X -Native ORC Vectorized (Pushdown) 317 / 320 0.0 317392594.0 1.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 row with 250 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 2192 / 2218 0.0 2191883823.0 1.0X -Parquet Vectorized (Pushdown) 2675 / 2687 0.0 2675439029.0 0.8X -Native ORC Vectorized 2158 / 2162 0.0 2157646071.0 1.0X -Native ORC Vectorized (Pushdown) 2309 / 2326 0.0 2309096612.0 0.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -Select 1 row with 500 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------- -Parquet Vectorized 6219 / 6248 0.0 6218727737.0 1.0X -Parquet Vectorized (Pushdown) 7376 / 7436 0.0 7375977710.0 0.8X -Native ORC Vectorized 6252 / 6279 0.0 6252473320.0 1.0X -Native ORC Vectorized (Pushdown) 6858 / 6876 0.0 6857854486.0 0.9X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 152 157 4 0.0 151562084.0 1.0X +Parquet Vectorized (Pushdown) 156 160 5 0.0 155523780.0 1.0X +Native ORC Vectorized 145 150 4 0.0 144571223.0 1.0X +Native ORC Vectorized (Pushdown) 145 150 4 0.0 144905986.0 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 250 257 6 0.0 249773194.0 1.0X +Parquet Vectorized (Pushdown) 263 268 5 0.0 262666962.0 1.0X +Native ORC Vectorized 251 263 9 0.0 251438048.0 1.0X +Native ORC Vectorized (Pushdown) 260 263 3 0.0 259811377.0 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Parquet Vectorized 366 377 6 0.0 366122909.0 1.0X +Parquet Vectorized (Pushdown) 395 399 3 0.0 394562758.0 0.9X +Native ORC Vectorized 367 374 4 0.0 366931788.0 1.0X +Native ORC Vectorized (Pushdown) 378 382 4 0.0 377890425.0 1.0X + + +================================================================================================ +Predicate conversion benchmark with unbalanced Expression +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 25 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC Vectorized (Pushdown) 0 0 0 0.0 341119.0 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 5000 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC Vectorized (Pushdown) 5 5 1 0.0 4594703.0 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 15000 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC Vectorized (Pushdown) 16 18 2 0.0 16423418.0 1.0X + + +================================================================================================ +Pushdown benchmark with unbalanced Column +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 25 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC Vectorized (Pushdown) 94 97 3 0.0 94040154.0 1.0X +Native Parquet Vectorized (Pushdown) 88 92 3 0.0 88400127.0 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC Vectorized (Pushdown) 958 964 5 0.0 957697803.0 1.0X +Native Parquet Vectorized (Pushdown) 1021 1034 13 0.0 1021483274.0 0.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 +Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz +Select 1 row with 1000 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Native ORC Vectorized (Pushdown) 5465 5469 6 0.0 5465108758.0 1.0X +Native Parquet Vectorized (Pushdown) 5741 6229 721 0.0 5741040597.0 1.0X From 78ad7b8efc67b96b80661b12da88d7ba78ba3bb7 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 22 Mar 2019 12:42:52 +0200 Subject: [PATCH 17/56] Key memoization table only based on --- .../datasources/orc/OrcFilters.scala | 84 ++++++------------- 1 file changed, 27 insertions(+), 57 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6d411b67aad0..e0ea24b32c33 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -17,14 +17,13 @@ package org.apache.spark.sql.execution.datasources.orc -import java.util.Objects +import java.util import org.apache.orc.storage.common.`type`.HiveDecimal import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.orc.storage.serde2.io.HiveDecimalWritable -import scala.collection.mutable import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ @@ -310,50 +309,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { } } -/** - * This case class represents a position in a `Filter` tree, paired with information about - * whether `AND` predicates can be partially pushed down. It is used as a key in the - * memoization map for `OrcConvertibilityChecker`. - * - * Because of the way this class is used, there are a few subtleties in its implementation - * and behaviour: - * - The default generated `hashCode` and `equals` methods have linear complexity in the size of - * the `expression` Filter. This is because they end up recursing over the whole `Filter` tree - * in order to check for equality, in order to return true for `Filter`s with different - * identities but the same actual contents. As a result, using this class as a key in a HashMap - * with the default generated methods would end up having a complexity that's linear in the - * size of the `expression`, instead of constnant. This means that using it as a key in the - * memoization map for `OrcConvertibilityChecker` would make the whole convertibility checking - * take time quadratic in the size of the tree instead of linear. - * - For this reason, we override `hashCode` and `equals` to only check the identity of the - * `expression` and not the full expression object. - * - Hashing based on the identity of the expression results in correct behaviour because we - * are indeed only interested in memoizing the results for exact locations in the `Filter` - * tree. If there are expressions that are exactly the same but at different locations, we - * would treat them as different objects and simply compute the results for them twice. - * - Since implementing `equals` and `hashCode` in the presence of subclasses is trickier, - * this class is sealed to guarantee that we don't need to be concerned with subclassing. - */ -private sealed case class FilterWithConjunctPushdown( - expression: Filter, - canPartialPushDownConjuncts: Boolean -) { - override def hashCode(): Int = { - Objects.hash( - System.identityHashCode(expression).asInstanceOf[Object], - canPartialPushDownConjuncts.asInstanceOf[Object]) - } - - override def equals(obj: Any): Boolean = { - obj match { - case FilterWithConjunctPushdown(expr, canPushDown) => - Objects.equals(System.identityHashCode(expression), System.identityHashCode(expr)) && - Objects.equals(canPartialPushDownConjuncts, canPushDown) - case _ => false - } - } -} - /** * Helper class for efficiently checking whether a `Filter` and its children can be converted to * ORC `SearchArgument`s. The `isConvertible` method has a constant amortized complexity for @@ -365,14 +320,19 @@ private sealed case class FilterWithConjunctPushdown( */ private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { - private val convertibilityCache = new mutable.HashMap[FilterWithConjunctPushdown, Boolean] + // Here, we only need the hash map to be based on the identity of the `Filter` and not on object + // equality. This ensures that the complexity for accessing the hash map is constant (with the + // only necessary step being to compute the identity hash code of the `Filter`) instead of + // linear (which would be the case if we were computing the `Filter` hash code based on all of + // its children). + private val convertibilityCache = new util.IdentityHashMap[Filter, Boolean] /** * Checks if a given expression from a Filter is convertible. * - * The result for a given (expression, canPartialPushDownConjuncts) pair within a Filter - * will never change, so we memoize the results in the `convertibilityCache`. This means - * that the overall complexity for checking all nodes in a `Filter` is: + * The result for a given expression within a Filter will never change, so we memoize the + * results in the `convertibilityCache`. This means that the overall complexity for checking + * all nodes in a `Filter` is: * - a total of one pass across the whole Filter tree is made when nodes are checked for * convertibility for the first time. * - when checked for a second, etc. time, the result has already been memoized in the cache @@ -384,19 +344,29 @@ private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { * do at the top level or none of its ancestors is NOT and OR. */ def isConvertible(expression: Filter, canPartialPushDownConjuncts: Boolean): Boolean = { - val node = FilterWithConjunctPushdown(expression, canPartialPushDownConjuncts) - convertibilityCache.getOrElseUpdate(node, isConvertibleImpl(node)) + // The value of `canPartialPushDownConjuncts` will always be the same for invocations on the + // `expression`. Thus, we don't need to add that as part of the memoization key. + if (convertibilityCache.containsKey(expression)) { + convertibilityCache.get(expression) + } else { + val result = isConvertibleImpl(expression, canPartialPushDownConjuncts) + convertibilityCache.put(expression, result) + result + } } /** * This method duplicates the logic from `OrcFilters.createBuilder` that is related to checking * if a given part of the filter is actually convertible to an ORC `SearchArgument`. */ - private def isConvertibleImpl(node: FilterWithConjunctPushdown): Boolean = { + private def isConvertibleImpl( + expression: Filter, + canPartialPushDownConjuncts: Boolean + ): Boolean = { import org.apache.spark.sql.sources._ import OrcFilters._ - node.expression match { + expression match { case And(left, right) => // At here, it is not safe to just convert one side and remove the other side // if we do not understand what the parent filters are. @@ -409,12 +379,12 @@ private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. - val leftIsConvertible = isConvertible(left, node.canPartialPushDownConjuncts) - val rightIsConvertible = isConvertible(right, node.canPartialPushDownConjuncts) + val leftIsConvertible = isConvertible(left, canPartialPushDownConjuncts) + val rightIsConvertible = isConvertible(right, canPartialPushDownConjuncts) // NOTE: If we can use partial predicates here, we only need one of the children to // be convertible to be able to convert the parent. Otherwise, we need both to be // convertible. - if (node.canPartialPushDownConjuncts) { + if (canPartialPushDownConjuncts) { leftIsConvertible || rightIsConvertible } else { leftIsConvertible && rightIsConvertible From 56d23cba470c9e90491a16c5c3ccf31884b48f3f Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 7 May 2019 23:48:37 +0300 Subject: [PATCH 18/56] Implement tree trimming --- .../datasources/orc/OrcFilters.scala | 273 +++++------------- 1 file changed, 78 insertions(+), 195 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index e0ea24b32c33..5fc6ea97f2f8 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql.execution.datasources.orc -import java.util - import org.apache.orc.storage.common.`type`.HiveDecimal import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder +import org.apache.orc.storage.ql.io.sarg.SearchArgument.TruthValue import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.orc.storage.serde2.io.HiveDecimalWritable @@ -152,29 +151,14 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - createBuilder( - dataTypeMap, - new OrcConvertibilityChecker(dataTypeMap), - expression, - builder, - canPartialPushDownConjuncts = true) + trimNonConvertibleSubtrees(dataTypeMap, expression, canPartialPushDownConjuncts = true) + .map(createBuilder(dataTypeMap, _, builder)) } - /** - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the builder so far. - */ - private def createBuilder( + private def trimNonConvertibleSubtrees( dataTypeMap: Map[String, DataType], - orcConvertibilityChecker: OrcConvertibilityChecker, expression: Filter, - builder: Builder, - canPartialPushDownConjuncts: Boolean): Option[Builder] = { + canPartialPushDownConjuncts: Boolean): Option[Filter] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -182,43 +166,16 @@ private[sql] object OrcFilters extends OrcFiltersBase { expression match { case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftIsConvertible = orcConvertibilityChecker.isConvertible( - left, - canPartialPushDownConjuncts - ) - val rightIsConvertible = orcConvertibilityChecker.isConvertible( - right, - canPartialPushDownConjuncts - ) - (leftIsConvertible, rightIsConvertible) match { - case (true, true) => - for { - lhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, left, - builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, right, - lhs, canPartialPushDownConjuncts) - } yield rhs.end() - - case (true, false) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, orcConvertibilityChecker, left, - builder, canPartialPushDownConjuncts) - - case (false, true) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, orcConvertibilityChecker, right, - builder, canPartialPushDownConjuncts) - - case _ => None + val lhs = trimNonConvertibleSubtrees(dataTypeMap, left, canPartialPushDownConjuncts = true) + val rhs = trimNonConvertibleSubtrees(dataTypeMap, right, canPartialPushDownConjuncts = true) + if (lhs.isDefined && rhs.isDefined) { + Some(And(lhs.get, rhs.get)) + } else { + if (canPartialPushDownConjuncts && (lhs.isDefined || rhs.isDefined)) { + lhs.orElse(rhs) + } else { + None + } } case Or(left, right) => @@ -233,183 +190,109 @@ private[sql] object OrcFilters extends OrcFiltersBase { // The predicate can be converted as // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). - val leftIsConvertible = orcConvertibilityChecker.isConvertible( - left, - canPartialPushDownConjuncts - ) - val rightIsConvertible = orcConvertibilityChecker.isConvertible( - right, - canPartialPushDownConjuncts - ) for { - _ <- Option(leftIsConvertible && rightIsConvertible).filter(identity) - lhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, left, - builder.startOr(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, orcConvertibilityChecker, right, - lhs, canPartialPushDownConjuncts) - } yield rhs.end() + lhs: Filter <- + trimNonConvertibleSubtrees(dataTypeMap, left, canPartialPushDownConjuncts) + rhs: Filter <- + trimNonConvertibleSubtrees(dataTypeMap, right, canPartialPushDownConjuncts) + } yield Or(lhs, rhs) case Not(child) => - for { - negate <- createBuilder(dataTypeMap, orcConvertibilityChecker, - child, builder.startNot(), canPartialPushDownConjuncts = false) - if orcConvertibilityChecker.isConvertible(child, canPartialPushDownConjuncts = false) - } yield negate.end() + val filteredSubtree = + trimNonConvertibleSubtrees(dataTypeMap, child, canPartialPushDownConjuncts = false) + filteredSubtree.map(Not(_)) - // 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`). + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + + case _ => None + } + } + + private def createBuilder( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Builder = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) + + import org.apache.spark.sql.sources._ + expression match { + case And(left, right) => + builder.startAnd() + createBuilder(dataTypeMap, left, builder) + createBuilder(dataTypeMap, right, builder) + builder.end() + + case Or(left, right) => + builder.startOr() + createBuilder(dataTypeMap, left, builder) + createBuilder(dataTypeMap, right, builder) + builder.end() + + case Not(child) => + builder.startNot() + createBuilder(dataTypeMap, child, builder) + builder.end() case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) - Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + builder.startAnd().isNull(quotedName, getType(attribute)).end() case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) - Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + builder.startNot().isNull(quotedName, getType(attribute)).end() case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(quotedName, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) + builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - case _ => None + case _ => builder.startAnd().literal(TruthValue.YES).end() } } } -/** - * Helper class for efficiently checking whether a `Filter` and its children can be converted to - * ORC `SearchArgument`s. The `isConvertible` method has a constant amortized complexity for - * checking whether a node from a Filter expression is convertible to an ORC `SearchArgument`. - * The additional memory is O(N) in the size of the whole `Filter` being checked due to the use - * of a hash map that possibly has an entry for each subnode in the filter. - * - * @param dataTypeMap a map from the attribute name to its data type. - */ -private class OrcConvertibilityChecker(dataTypeMap: Map[String, DataType]) { - - // Here, we only need the hash map to be based on the identity of the `Filter` and not on object - // equality. This ensures that the complexity for accessing the hash map is constant (with the - // only necessary step being to compute the identity hash code of the `Filter`) instead of - // linear (which would be the case if we were computing the `Filter` hash code based on all of - // its children). - private val convertibilityCache = new util.IdentityHashMap[Filter, Boolean] - - /** - * Checks if a given expression from a Filter is convertible. - * - * The result for a given expression within a Filter will never change, so we memoize the - * results in the `convertibilityCache`. This means that the overall complexity for checking - * all nodes in a `Filter` is: - * - a total of one pass across the whole Filter tree is made when nodes are checked for - * convertibility for the first time. - * - when checked for a second, etc. time, the result has already been memoized in the cache - * and thus the check has a constant time. - * - * @param expression the input filter predicates. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - */ - def isConvertible(expression: Filter, canPartialPushDownConjuncts: Boolean): Boolean = { - // The value of `canPartialPushDownConjuncts` will always be the same for invocations on the - // `expression`. Thus, we don't need to add that as part of the memoization key. - if (convertibilityCache.containsKey(expression)) { - convertibilityCache.get(expression) - } else { - val result = isConvertibleImpl(expression, canPartialPushDownConjuncts) - convertibilityCache.put(expression, result) - result - } - } - - /** - * This method duplicates the logic from `OrcFilters.createBuilder` that is related to checking - * if a given part of the filter is actually convertible to an ORC `SearchArgument`. - */ - private def isConvertibleImpl( - expression: Filter, - canPartialPushDownConjuncts: Boolean - ): Boolean = { - import org.apache.spark.sql.sources._ - import OrcFilters._ - - expression match { - case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftIsConvertible = isConvertible(left, canPartialPushDownConjuncts) - val rightIsConvertible = isConvertible(right, canPartialPushDownConjuncts) - // NOTE: If we can use partial predicates here, we only need one of the children to - // be convertible to be able to convert the parent. Otherwise, we need both to be - // convertible. - if (canPartialPushDownConjuncts) { - leftIsConvertible || rightIsConvertible - } else { - leftIsConvertible && rightIsConvertible - } - - case Or(left, right) => - val leftIsConvertible = isConvertible(left, node.canPartialPushDownConjuncts) - val rightIsConvertible = isConvertible(right, node.canPartialPushDownConjuncts) - leftIsConvertible && rightIsConvertible - - case Not(child) => - val childIsConvertible = isConvertible(child, canPartialPushDownConjuncts = false) - childIsConvertible - - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => true - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => true - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => true - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => true - - case _ => false - } - } -} From 265d08877c78f49785cdb0da46a94d57d4a1ccf7 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 7 May 2019 23:52:45 +0300 Subject: [PATCH 19/56] Extract an inline method --- .../execution/datasources/orc/OrcFilters.scala | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 5fc6ea97f2f8..868724121b5b 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -229,22 +229,22 @@ private[sql] object OrcFilters extends OrcFiltersBase { getPredicateLeafType(dataTypeMap(attribute)) import org.apache.spark.sql.sources._ - expression match { + def updateBuilder(subexpression: Filter): Unit = subexpression match { case And(left, right) => builder.startAnd() - createBuilder(dataTypeMap, left, builder) - createBuilder(dataTypeMap, right, builder) + updateBuilder(left) + updateBuilder(right) builder.end() case Or(left, right) => builder.startOr() - createBuilder(dataTypeMap, left, builder) - createBuilder(dataTypeMap, right, builder) + updateBuilder(left) + updateBuilder(right) builder.end() case Not(child) => builder.startNot() - createBuilder(dataTypeMap, child, builder) + updateBuilder(child) builder.end() case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => @@ -293,6 +293,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => builder.startAnd().literal(TruthValue.YES).end() } + + updateBuilder(expression) + builder } } From a721e80fb1d9c29cf822a52e965bdca42609a023 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 10:49:48 +0300 Subject: [PATCH 20/56] Add a TrimmedFilter value class for better type safety --- .../datasources/orc/OrcFilters.scala | 39 +++++++++++++++---- 1 file changed, 31 insertions(+), 8 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 868724121b5b..888580d08f03 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -144,6 +144,20 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => value } + /** + * A TrimmedFilter is a Filter that has been trimmed such that all the remaining nodes + * are convertible to ORC predicates. + * + * Since nothing in the underlying representation of the Filter is actually different from a + * regular Filter (the only difference is that we might remove some subtrees), this class is just + * a wrapper around a `Filter` value. The main benefits of using this class are readability + * and type safety (to signal that the respective functions only work with already trimmed + * filters). + * + * @param filter The underlying filter representation. + */ + private case class TrimmedFilter(filter: Filter) extends AnyVal + /** * Build a SearchArgument and return the builder so far. */ @@ -151,11 +165,18 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - trimNonConvertibleSubtrees(dataTypeMap, expression, canPartialPushDownConjuncts = true) + trimNonConvertibleSubtrees(dataTypeMap, expression) .map(createBuilder(dataTypeMap, _, builder)) } private def trimNonConvertibleSubtrees( + dataTypeMap: Map[String, DataType], + expression: Filter): Option[TrimmedFilter] = { + trimNonConvertibleSubtreesImpl(dataTypeMap, expression, canPartialPushDownConjuncts = true) + .map(TrimmedFilter) + } + + private def trimNonConvertibleSubtreesImpl( dataTypeMap: Map[String, DataType], expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = { @@ -166,8 +187,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { expression match { case And(left, right) => - val lhs = trimNonConvertibleSubtrees(dataTypeMap, left, canPartialPushDownConjuncts = true) - val rhs = trimNonConvertibleSubtrees(dataTypeMap, right, canPartialPushDownConjuncts = true) + val lhs = + trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) + val rhs = + trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts = true) if (lhs.isDefined && rhs.isDefined) { Some(And(lhs.get, rhs.get)) } else { @@ -192,14 +215,14 @@ private[sql] object OrcFilters extends OrcFiltersBase { // As per the logical in And predicate, we can push down (a1 OR b1). for { lhs: Filter <- - trimNonConvertibleSubtrees(dataTypeMap, left, canPartialPushDownConjuncts) + trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts) rhs: Filter <- - trimNonConvertibleSubtrees(dataTypeMap, right, canPartialPushDownConjuncts) + trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts) } yield Or(lhs, rhs) case Not(child) => val filteredSubtree = - trimNonConvertibleSubtrees(dataTypeMap, child, canPartialPushDownConjuncts = false) + trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) filteredSubtree.map(Not(_)) case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) @@ -223,7 +246,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { private def createBuilder( dataTypeMap: Map[String, DataType], - expression: Filter, + expression: TrimmedFilter, builder: Builder): Builder = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -294,7 +317,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => builder.startAnd().literal(TruthValue.YES).end() } - updateBuilder(expression) + updateBuilder(expression.filter) builder } } From fbe840a3e5bf494927429395c344a7f54ee2cf39 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 11:29:03 +0300 Subject: [PATCH 21/56] Add Javadoc for the trimNonConvertibleFilters methods --- .../datasources/orc/OrcFilters.scala | 24 +++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 888580d08f03..9aa862ba344a 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -49,8 +49,8 @@ import org.apache.spark.sql.types._ * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent * now. * - * The workaround employed here is that, for `And`/`Or`/`Not`, we explicitly check if the children - * are convertible, and only do the actual conversion when the children are proven to be + * The workaround employed here is to trim the Spark filters before trying to convert them. This + * way, we can only do the actual conversion on the part of the Filter that is known to be * convertible. * * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of @@ -169,6 +169,14 @@ private[sql] object OrcFilters extends OrcFiltersBase { .map(createBuilder(dataTypeMap, _, builder)) } + /** + * Transforms a `Filter` by removing all subtrees that are not convertible to an ORC + * SearchArgument. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @return A TrimmedFilter that wraps the transformed `Filter`. + */ private def trimNonConvertibleSubtrees( dataTypeMap: Map[String, DataType], expression: Filter): Option[TrimmedFilter] = { @@ -176,6 +184,18 @@ private[sql] object OrcFilters extends OrcFiltersBase { .map(TrimmedFilter) } + /** + * Internal recursive implementation of the `trimNonConvertibleSubtrees` method. We use two + * separate methods here in order to avoid dealing with the wrapper `TrimmedFilter` classes + * in the recursive implementation here, and only wrap the final result in the outer function. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed + * down safely. Pushing ONLY one side of AND down is safe to + * do at the top level or none of its ancestors is NOT and OR. + * @return the trimmed `Filter`. + */ private def trimNonConvertibleSubtreesImpl( dataTypeMap: Map[String, DataType], expression: Filter, From 5b649dff82ca4cac47a06c64056a8bef6c57a6a2 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 11:46:00 +0300 Subject: [PATCH 22/56] Apply changes to v2.3.4 subtree as well --- .../datasources/orc/OrcFilters.scala | 192 +++++++++++------- 1 file changed, 124 insertions(+), 68 deletions(-) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 632a72a32abd..1b36e950040e 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -29,8 +29,8 @@ import org.apache.spark.sql.types._ /** * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. * - * Due to limitation of ORC `SearchArgument` builder, we had to end up with a pretty weird double- - * checking pattern when converting `And`/`Or`/`Not` filters. + * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and + * conversion code paths to make sure we only convert predicates that are known to be convertible. * * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite @@ -39,18 +39,18 @@ import org.apache.spark.sql.types._ * * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and * `startNot()` mutate internal state of the builder instance. This forces us to translate all - * convertible filters with a single builder instance. However, before actually converting a filter, - * we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible filter is - * found, we may already end up with a builder whose internal state is inconsistent. + * convertible filters with a single builder instance. However, if we try to translate a filter + * before checking whether it can be converted or not, we may end up with a builder whose internal + * state is inconsistent in the case of an inconvertible filter. * * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then * try to convert its children. Say we convert `left` child successfully, but find that `right` * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent * now. * - * The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their - * children with brand new builders, and only do the actual conversion with the right builder - * instance when the children are proven to be convertible. + * The workaround employed here is to trim the Spark filters before trying to convert them. This + * way, we can only do the actual conversion on the part of the Filter that is known to be + * convertible. * * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of * builder methods mentioned above can only be found in test code, where all tested filters are @@ -143,6 +143,20 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => value } + /** + * A TrimmedFilter is a Filter that has been trimmed such that all the remaining nodes + * are convertible to ORC predicates. + * + * Since nothing in the underlying representation of the Filter is actually different from a + * regular Filter (the only difference is that we might remove some subtrees), this class is just + * a wrapper around a `Filter` value. The main benefits of using this class are readability + * and type safety (to signal that the respective functions only work with already trimmed + * filters). + * + * @param filter The underlying filter representation. + */ + private case class TrimmedFilter(filter: Filter) extends AnyVal + /** * Build a SearchArgument and return the builder so far. */ @@ -150,23 +164,41 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) + trimNonConvertibleSubtrees(dataTypeMap, expression) + .map(createBuilder(dataTypeMap, _, builder)) } /** + * Transforms a `Filter` by removing all subtrees that are not convertible to an ORC + * SearchArgument. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @return A TrimmedFilter that wraps the transformed `Filter`. + */ + private def trimNonConvertibleSubtrees( + dataTypeMap: Map[String, DataType], + expression: Filter): Option[TrimmedFilter] = { + trimNonConvertibleSubtreesImpl(dataTypeMap, expression, canPartialPushDownConjuncts = true) + .map(TrimmedFilter) + } + + /** + * Internal recursive implementation of the `trimNonConvertibleSubtrees` method. We use two + * separate methods here in order to avoid dealing with the wrapper `TrimmedFilter` classes + * in the recursive implementation here, and only wrap the final result in the outer function. + * * @param dataTypeMap a map from the attribute name to its data type. * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed * down safely. Pushing ONLY one side of AND down is safe to * do at the top level or none of its ancestors is NOT and OR. - * @return the builder so far. + * @return the trimmed `Filter`. */ - private def createBuilder( + private def trimNonConvertibleSubtreesImpl( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder, - canPartialPushDownConjuncts: Boolean): Option[Builder] = { + canPartialPushDownConjuncts: Boolean): Option[Filter] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -174,36 +206,18 @@ private[sql] object OrcFilters extends OrcFiltersBase { expression match { case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftBuilderOption = - createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - val rightBuilderOption = - createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - (leftBuilderOption, rightBuilderOption) match { - case (Some(_), Some(_)) => - for { - lhs <- createBuilder(dataTypeMap, left, - builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() - - case (Some(_), None) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) - - case (None, Some(_)) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) - - case _ => None + val lhs = + trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) + val rhs = + trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts = true) + if (lhs.isDefined && rhs.isDefined) { + Some(And(lhs.get, rhs.get)) + } else { + if (canPartialPushDownConjuncts && (lhs.isDefined || rhs.isDefined)) { + lhs.orElse(rhs) + } else { + None + } } case Or(left, right) => @@ -219,68 +233,110 @@ private[sql] object OrcFilters extends OrcFiltersBase { // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). for { - _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - lhs <- createBuilder(dataTypeMap, left, builder.startOr(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() + lhs: Filter <- + trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts) + rhs: Filter <- + trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts) + } yield Or(lhs, rhs) case Not(child) => - for { - _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) - negate <- createBuilder(dataTypeMap, - child, builder.startNot(), canPartialPushDownConjuncts = false) - } yield negate.end() + val filteredSubtree = + trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) + filteredSubtree.map(Not(_)) - // 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`). + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + Some(expression) + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => Some(expression) + + case _ => None + } + } + + private def createBuilder( + dataTypeMap: Map[String, DataType], + expression: TrimmedFilter, + builder: Builder): Builder = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) + + import org.apache.spark.sql.sources._ + def updateBuilder(subexpression: Filter): Unit = subexpression match { + case And(left, right) => + builder.startAnd() + updateBuilder(left) + updateBuilder(right) + builder.end() + + case Or(left, right) => + builder.startOr() + updateBuilder(left) + updateBuilder(right) + builder.end() + + case Not(child) => + builder.startNot() + updateBuilder(child) + builder.end() case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) - Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + builder.startAnd().isNull(quotedName, getType(attribute)).end() case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) - Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + builder.startNot().isNull(quotedName, getType(attribute)).end() case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(quotedName, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) + builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - case _ => None + case _ => builder.startAnd().literal(TruthValue.YES).end() } + + updateBuilder(expression.filter) + builder } } From 9fe61d4401a868ac6457cf5bbaaaf6078534f028 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 12:11:14 +0300 Subject: [PATCH 23/56] Add comments --- .../datasources/orc/OrcFilters.scala | 32 +++++++++++++++++++ .../datasources/orc/OrcFilters.scala | 32 +++++++++++++++++++ 2 files changed, 64 insertions(+) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 9aa862ba344a..14d11dd984b8 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -189,6 +189,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { * separate methods here in order to avoid dealing with the wrapper `TrimmedFilter` classes * in the recursive implementation here, and only wrap the final result in the outer function. * + * NOTE: If you change the set of supported `Filter` types here, you need to modify + * `createBuilder` accordingly! + * * @param dataTypeMap a map from the attribute name to its data type. * @param expression the input filter predicates. * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed @@ -207,6 +210,17 @@ private[sql] object OrcFilters extends OrcFiltersBase { expression match { case And(left, right) => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. val lhs = trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) val rhs = @@ -245,6 +259,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) filteredSubtree.map(Not(_)) + // 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`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) @@ -264,6 +281,19 @@ private[sql] object OrcFilters extends OrcFiltersBase { } } + /** + * Build a SearchArgument for a Filter that has already been trimmed so as to only contain + * expressions that are convertible to a `SearchArgument`. This allows for a more efficient and + * more readable implementation since there's no need to check every node before converting it. + * + * NOTE: If you change the set of supported `Filter` types here, you need to modify + * `trimNonConvertibleSubtreesImpl` accordingly! + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the trimmed input filter predicates. + * @param builder the builder so far. + * @return + */ private def createBuilder( dataTypeMap: Map[String, DataType], expression: TrimmedFilter, @@ -334,6 +364,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { builder.startAnd().in(quotedName, getType(attribute), castedValues.map(_.asInstanceOf[AnyRef]): _*).end() + // This case should never happen since this function only expects fully convertible filters. + // However, we return true as a safety measure in case something goes wrong. case _ => builder.startAnd().literal(TruthValue.YES).end() } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 1b36e950040e..d5c66c18961c 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -188,6 +188,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { * separate methods here in order to avoid dealing with the wrapper `TrimmedFilter` classes * in the recursive implementation here, and only wrap the final result in the outer function. * + * NOTE: If you change the set of supported `Filter` types here, you need to modify + * `createBuilder` accordingly! + * * @param dataTypeMap a map from the attribute name to its data type. * @param expression the input filter predicates. * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed @@ -206,6 +209,17 @@ private[sql] object OrcFilters extends OrcFiltersBase { expression match { case And(left, right) => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. val lhs = trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) val rhs = @@ -244,6 +258,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) filteredSubtree.map(Not(_)) + // 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`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) @@ -263,6 +280,19 @@ private[sql] object OrcFilters extends OrcFiltersBase { } } + /** + * Build a SearchArgument for a Filter that has already been trimmed so as to only contain + * expressions that are convertible to a `SearchArgument`. This allows for a more efficient and + * more readable implementation since there's no need to check every node before converting it. + * + * NOTE: If you change the set of supported `Filter` types here, you need to modify + * `trimNonConvertibleSubtreesImpl` accordingly! + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the trimmed input filter predicates. + * @param builder the builder so far. + * @return + */ private def createBuilder( dataTypeMap: Map[String, DataType], expression: TrimmedFilter, @@ -333,6 +363,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { builder.startAnd().in(quotedName, getType(attribute), castedValues.map(_.asInstanceOf[AnyRef]): _*).end() + // This case should never happen since this function only expects fully convertible filters. + // However, we return true as a safety measure in case something goes wrong. case _ => builder.startAnd().literal(TruthValue.YES).end() } From c41421f8de24a252d12fdc3c9420b366ab5d17be Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 12:16:30 +0300 Subject: [PATCH 24/56] Replace if-else for And predicate with a pattern match --- .../sql/execution/datasources/orc/OrcFilters.scala | 13 +++++-------- .../sql/execution/datasources/orc/OrcFilters.scala | 13 +++++-------- 2 files changed, 10 insertions(+), 16 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 14d11dd984b8..ecf945019ed7 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -225,14 +225,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) val rhs = trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts = true) - if (lhs.isDefined && rhs.isDefined) { - Some(And(lhs.get, rhs.get)) - } else { - if (canPartialPushDownConjuncts && (lhs.isDefined || rhs.isDefined)) { - lhs.orElse(rhs) - } else { - None - } + (lhs, rhs) match { + case (Some(l), Some(r)) => Some(And(l, r)) + case (Some(_), None) if canPartialPushDownConjuncts => lhs + case (None, Some(_)) if canPartialPushDownConjuncts => rhs + case _ => None } case Or(left, right) => diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index d5c66c18961c..7de5ab53ca2f 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -224,14 +224,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) val rhs = trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts = true) - if (lhs.isDefined && rhs.isDefined) { - Some(And(lhs.get, rhs.get)) - } else { - if (canPartialPushDownConjuncts && (lhs.isDefined || rhs.isDefined)) { - lhs.orElse(rhs) - } else { - None - } + (lhs, rhs) match { + case (Some(l), Some(r)) => Some(And(l, r)) + case (Some(_), None) if canPartialPushDownConjuncts => lhs + case (None, Some(_)) if canPartialPushDownConjuncts => rhs + case _ => None } case Or(left, right) => From c3174e9aa3130bd6e79a195b2f1f08b78e951f05 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 12:17:52 +0300 Subject: [PATCH 25/56] Minor formatting and style changes --- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 7de5ab53ca2f..8870816008ab 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -251,9 +251,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } yield Or(lhs, rhs) case Not(child) => - val filteredSubtree = - trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) - filteredSubtree.map(Not(_)) + trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) + .map(Not) // 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 From 9af949975d9d967a0d52767865467dea5d2a1680 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 8 May 2019 19:27:04 +0300 Subject: [PATCH 26/56] Code review comments --- .../benchmark/FilterPushdownBenchmark.scala | 19 ++++---------- .../datasources/orc/OrcFilters.scala | 25 ++++++++++--------- .../datasources/orc/OrcFilters.scala | 25 ++++++++++--------- 3 files changed, 31 insertions(+), 38 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 6eba8ae4e66e..5e4ae561a6f2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -320,7 +320,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val filter = Range(0, count).map(r => scala.util.Random.nextInt(numRows * distribution / 100)) val whereExpr = s"value in(${filter.mkString(",")})" - val title = s"InSet -> InFilters (values count: $count, distribution: $distribution)" + val title = s"InSet -> InFilters (values count: $count, distribution: $distribution)" filterPushDownBenchmark(numRows, title, whereExpr) } } @@ -418,24 +418,15 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val columns = (1 to width).map(i => s"id c$i") val df = spark.range(1).selectExpr(columns: _*) Seq(25, 5000, 15000).foreach { numFilter => - val whereExpression = (1 to numFilter) - .map { - i => - EqualTo( - Literal(0), - AttributeReference( - s"c1", - IntegerType, - nullable = true)() - ).asInstanceOf[Expression] - } - .foldLeft[Expression](Literal.FalseLiteral)((x, y) => Or(x, y)) + val whereColumn = (1 to numFilter) + .map(i => col("c1") === lit(i)) + .foldLeft(lit(false))(_ || _) val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", numRows, minNumIters = 5, output = output) val name = s"Native ORC Vectorized (Pushdown)" benchmark.addCase(name) { _ => OrcFilters.createFilter(df.schema, - DataSourceStrategy.translateFilter(whereExpression).toSeq) + DataSourceStrategy.translateFilter(whereColumn.expr).toSeq) } benchmark.run() } diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index ecf945019ed7..13ad5c9f655b 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -256,9 +256,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) filteredSubtree.map(Not(_)) - // 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`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) @@ -317,45 +314,49 @@ private[sql] object OrcFilters extends OrcFiltersBase { updateBuilder(child) builder.end() - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + // 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`). + + case EqualTo(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case EqualNullSafe(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case LessThan(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case LessThanOrEqual(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case GreaterThan(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case GreaterThanOrEqual(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + case IsNull(attribute) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + case IsNotNull(attribute) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + case In(attribute, values) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) builder.startAnd().in(quotedName, getType(attribute), diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 8870816008ab..10f226a93903 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -254,9 +254,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) .map(Not) - // 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`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) @@ -315,45 +312,49 @@ private[sql] object OrcFilters extends OrcFiltersBase { updateBuilder(child) builder.end() - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + // 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`). + + case EqualTo(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case EqualNullSafe(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case LessThan(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case LessThanOrEqual(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case GreaterThan(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + case GreaterThanOrEqual(attribute, value) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + case IsNull(attribute) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + case IsNotNull(attribute) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + case In(attribute, values) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) builder.startAnd().in(quotedName, getType(attribute), From 88d418154b94f5ee7f74c0e3295d6c4002afc7e3 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 9 May 2019 11:45:29 +0300 Subject: [PATCH 27/56] Update Benchmark comments --- .../execution/benchmark/FilterPushdownBenchmark.scala | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 5e4ae561a6f2..650c443e67cb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -412,6 +412,13 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } runBenchmark(s"Predicate conversion benchmark with unbalanced Expression") { + // This benchmark tests a very isolated part of the predicate pushdown process - specifically, + // the individual action of converting a Spark `Expression` to an ORC `SearchArgument`. + // This results in more granular numbers that can help highlight small performance + // differences in this part of the code that would be hidden by slower components that + // get run when a full Spark job is executed. + // The benchmark below runs a more complete, end-to-end test which covers the whole pipeline + // and can uncover high-level performance problems, but is bad at discriminating details. val numRows = 1 val width = 2000 @@ -421,7 +428,8 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { val whereColumn = (1 to numFilter) .map(i => col("c1") === lit(i)) .foldLeft(lit(false))(_ || _) - val benchmark = new Benchmark(s"Select 1 row with $numFilter filters", + val benchmark = new Benchmark( + s"Convert a filter with $numFilter columns to ORC filter", numRows, minNumIters = 5, output = output) val name = s"Native ORC Vectorized (Pushdown)" benchmark.addCase(name) { _ => From 5aa1502e0ba4153af503143be09e16610c0711be Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 9 May 2019 11:46:33 +0300 Subject: [PATCH 28/56] Rename Expression -> Column in benchmark name --- .../spark/sql/execution/benchmark/FilterPushdownBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 650c443e67cb..fe0ce329a771 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -411,7 +411,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } } - runBenchmark(s"Predicate conversion benchmark with unbalanced Expression") { + runBenchmark(s"Predicate conversion benchmark with unbalanced Column") { // This benchmark tests a very isolated part of the predicate pushdown process - specifically, // the individual action of converting a Spark `Expression` to an ORC `SearchArgument`. // This results in more granular numbers that can help highlight small performance From c6dacf8a17cae304caa9cc1b2575a0c0e73ba509 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 15 May 2019 09:48:15 +0300 Subject: [PATCH 29/56] Revert existing benchmark --- .../spark/sql/execution/benchmark/FilterPushdownBenchmark.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index fe0ce329a771..236253704823 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -403,7 +403,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { withTempTable("orcTable", "parquetTable") { saveAsTable(df, dir) Seq(1, 250, 500).foreach { numFilter => - val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" or ") + val whereExpr = (1 to numFilter).map(i => s"c$i = 0").mkString(" and ") // Note: InferFiltersFromConstraints will add more filters to this given filters filterPushDownBenchmark(numRows, s"Select 1 row with $numFilter filters", whereExpr) } From d350bed043592bfe33e4e5769583307796ae6597 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 21 May 2019 11:13:34 +0300 Subject: [PATCH 30/56] Initial implementation with filter-and-build in the same place --- .../datasources/orc/OrcFilters.scala | 211 +++++++++++++++++- 1 file changed, 209 insertions(+), 2 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 13ad5c9f655b..638e9e1fabf3 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -24,8 +24,17 @@ import org.apache.orc.storage.ql.io.sarg.SearchArgument.TruthValue import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.orc.storage.serde2.io.HiveDecimalWritable +import org.apache.spark.sql.sources.And +import org.apache.spark.sql.sources.EqualNullSafe +import org.apache.spark.sql.sources.EqualTo import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.GreaterThan +import org.apache.spark.sql.sources.LessThan +import org.apache.spark.sql.sources.LessThanOrEqual +import org.apache.spark.sql.sources.Not +import org.apache.spark.sql.sources.Or import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.DecimalType.Expression /** * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. @@ -165,8 +174,206 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - trimNonConvertibleSubtrees(dataTypeMap, expression) - .map(createBuilder(dataTypeMap, _, builder)) + filterAndBuild(dataTypeMap, expression, builder) +// trimNonConvertibleSubtrees(dataTypeMap, expression) +// .map(createBuilder(dataTypeMap, _, builder)) + } + +// case class ExpressionFunctions( +// filterFn: (trimFn: (Filter => Option[Filter])) => Option[Filter], +// convertFn: (SearchArgument.Builder) => Unit +// ) + + sealed trait ActionType { + type T + } + case object FilterAction extends ActionType { + override type T = Option[Filter] + } + case object BuildAction extends ActionType { + override type T = Unit + } + + private def filterAndBuild( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder + ): Option[Builder] = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) + + import org.apache.spark.sql.sources._ + + def performAction( + actionType: ActionType, + expression: Filter, + canPartialPushDownConjuncts: Boolean): Either[Option[Filter], Unit] = { + expression match { + case And(left, right) => + actionType match { + case FilterAction => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported + // predicate can be safely removed. + val lhs = + performFilter(left, canPartialPushDownConjuncts = true) + val rhs = + performFilter(right, canPartialPushDownConjuncts = true) + (lhs, rhs) match { + case (Some(l), Some(r)) => Left(Some(And(l, r))) + case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) + case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) + case _ => Left(None) + } + case BuildAction => + builder.startAnd() + updateBuilder(left) + updateBuilder(right) + builder.end() + Right(Unit) + } + + case Or(left, right) => + actionType match { + case FilterAction => + Left(for { + lhs: Filter <- performFilter(left, canPartialPushDownConjuncts = false) + rhs: Filter <- performFilter(right, canPartialPushDownConjuncts = false) + } yield Or(lhs, rhs)) + case BuildAction => + builder.startOr() + updateBuilder(left) + updateBuilder(right) + builder.end() + Right(Unit) + } + + case Not(child) => + actionType match { + case FilterAction => + val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) + Left(filteredSubtree.map(Not(_))) + case BuildAction => + builder.startNot() + updateBuilder(child) + builder.end() + Right(Unit) + } + + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startAnd().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startNot().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end() + Right(Unit) + } + + case _ => + actionType match { + case FilterAction => Left(None) + case BuildAction => + throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") + } + } + } + + def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean) = + performAction(FilterAction, expression, canPartialPushDownConjuncts).left.get + + def updateBuilder(expression: Filter) = + performAction(BuildAction, expression, canPartialPushDownConjuncts = true).right.get + + + val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) + filteredExpression.foreach(updateBuilder) + filteredExpression.map(_ => builder) +// if (filteredExpression.isDefined) { +// updateBuilder(filteredExpression.get) +// Some(builder) +// } else { +// None +// } } /** From 0f62282c159c3cc0f4a7794febb646e5b7c88a15 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 21 May 2019 11:15:58 +0300 Subject: [PATCH 31/56] Remove commented out code --- .../datasources/orc/OrcFilters.scala | 219 +----------------- 1 file changed, 2 insertions(+), 217 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 638e9e1fabf3..67479c6f0ca8 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -175,15 +175,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { expression: Filter, builder: Builder): Option[Builder] = { filterAndBuild(dataTypeMap, expression, builder) -// trimNonConvertibleSubtrees(dataTypeMap, expression) -// .map(createBuilder(dataTypeMap, _, builder)) } -// case class ExpressionFunctions( -// filterFn: (trimFn: (Filter => Option[Filter])) => Option[Filter], -// convertFn: (SearchArgument.Builder) => Unit -// ) - sealed trait ActionType { type T } @@ -245,8 +238,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { actionType match { case FilterAction => Left(for { - lhs: Filter <- performFilter(left, canPartialPushDownConjuncts = false) - rhs: Filter <- performFilter(right, canPartialPushDownConjuncts = false) + lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) + rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) } yield Or(lhs, rhs)) case BuildAction => builder.startOr() @@ -368,214 +361,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) filteredExpression.foreach(updateBuilder) filteredExpression.map(_ => builder) -// if (filteredExpression.isDefined) { -// updateBuilder(filteredExpression.get) -// Some(builder) -// } else { -// None -// } - } - - /** - * Transforms a `Filter` by removing all subtrees that are not convertible to an ORC - * SearchArgument. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @return A TrimmedFilter that wraps the transformed `Filter`. - */ - private def trimNonConvertibleSubtrees( - dataTypeMap: Map[String, DataType], - expression: Filter): Option[TrimmedFilter] = { - trimNonConvertibleSubtreesImpl(dataTypeMap, expression, canPartialPushDownConjuncts = true) - .map(TrimmedFilter) - } - - /** - * Internal recursive implementation of the `trimNonConvertibleSubtrees` method. We use two - * separate methods here in order to avoid dealing with the wrapper `TrimmedFilter` classes - * in the recursive implementation here, and only wrap the final result in the outer function. - * - * NOTE: If you change the set of supported `Filter` types here, you need to modify - * `createBuilder` accordingly! - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the trimmed `Filter`. - */ - private def trimNonConvertibleSubtreesImpl( - dataTypeMap: Map[String, DataType], - expression: Filter, - canPartialPushDownConjuncts: Boolean): Option[Filter] = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute)) - - import org.apache.spark.sql.sources._ - - expression match { - case And(left, right) => - // At here, it is not safe to just keep one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val lhs = - trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) - val rhs = - trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts = true) - (lhs, rhs) match { - case (Some(l), Some(r)) => Some(And(l, r)) - case (Some(_), None) if canPartialPushDownConjuncts => lhs - case (None, Some(_)) if canPartialPushDownConjuncts => rhs - case _ => None - } - - case Or(left, right) => - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - for { - lhs: Filter <- - trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts) - rhs: Filter <- - trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts) - } yield Or(lhs, rhs) - - case Not(child) => - val filteredSubtree = - trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) - filteredSubtree.map(Not(_)) - - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - - case _ => None - } - } - - /** - * Build a SearchArgument for a Filter that has already been trimmed so as to only contain - * expressions that are convertible to a `SearchArgument`. This allows for a more efficient and - * more readable implementation since there's no need to check every node before converting it. - * - * NOTE: If you change the set of supported `Filter` types here, you need to modify - * `trimNonConvertibleSubtreesImpl` accordingly! - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the trimmed input filter predicates. - * @param builder the builder so far. - * @return - */ - private def createBuilder( - dataTypeMap: Map[String, DataType], - expression: TrimmedFilter, - builder: Builder): Builder = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute)) - - import org.apache.spark.sql.sources._ - def updateBuilder(subexpression: Filter): Unit = subexpression match { - case And(left, right) => - builder.startAnd() - updateBuilder(left) - updateBuilder(right) - builder.end() - - case Or(left, right) => - builder.startOr() - updateBuilder(left) - updateBuilder(right) - builder.end() - - case Not(child) => - builder.startNot() - updateBuilder(child) - builder.end() - - // 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`). - - case EqualTo(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - - case EqualNullSafe(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - - case LessThan(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - - case LessThanOrEqual(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - - case GreaterThan(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - - case GreaterThanOrEqual(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - - case IsNull(attribute) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startAnd().isNull(quotedName, getType(attribute)).end() - - case IsNotNull(attribute) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startNot().isNull(quotedName, getType(attribute)).end() - - case In(attribute, values) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - builder.startAnd().in(quotedName, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - - // This case should never happen since this function only expects fully convertible filters. - // However, we return true as a safety measure in case something goes wrong. - case _ => builder.startAnd().literal(TruthValue.YES).end() - } - - updateBuilder(expression.filter) - builder } } From 90a781a56fadd0c58dbf6ace2ed05b4158c16a47 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 21 May 2019 11:35:54 +0300 Subject: [PATCH 32/56] Get rid of the type member in ActionType --- .../sql/execution/datasources/orc/OrcFilters.scala | 12 +++--------- 1 file changed, 3 insertions(+), 9 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 67479c6f0ca8..e47f3c5ef6d7 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -177,15 +177,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { filterAndBuild(dataTypeMap, expression, builder) } - sealed trait ActionType { - type T - } - case object FilterAction extends ActionType { - override type T = Option[Filter] - } - case object BuildAction extends ActionType { - override type T = Unit - } + sealed trait ActionType + case object FilterAction extends ActionType + case object BuildAction extends ActionType private def filterAndBuild( dataTypeMap: Map[String, DataType], From 77af5b795e7ca11004e3bd9be857691d0c82e8b9 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 22 May 2019 17:50:59 +0300 Subject: [PATCH 33/56] Remove unused imports --- .../sql/execution/datasources/orc/OrcFilters.scala | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index e47f3c5ef6d7..0c505909719e 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -20,21 +20,11 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.orc.storage.common.`type`.HiveDecimal import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder -import org.apache.orc.storage.ql.io.sarg.SearchArgument.TruthValue import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder import org.apache.orc.storage.serde2.io.HiveDecimalWritable -import org.apache.spark.sql.sources.And -import org.apache.spark.sql.sources.EqualNullSafe -import org.apache.spark.sql.sources.EqualTo import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.sources.GreaterThan -import org.apache.spark.sql.sources.LessThan -import org.apache.spark.sql.sources.LessThanOrEqual -import org.apache.spark.sql.sources.Not -import org.apache.spark.sql.sources.Or import org.apache.spark.sql.types._ -import org.apache.spark.sql.types.DecimalType.Expression /** * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. From 2b311d2876b1eb575a46c461b82df7b2ef955495 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 09:36:09 +0300 Subject: [PATCH 34/56] Add comments to new functions --- .../datasources/orc/OrcFilters.scala | 34 ++++++++++++++++++- 1 file changed, 33 insertions(+), 1 deletion(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 0c505909719e..ba3e6471a811 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -171,6 +171,20 @@ private[sql] object OrcFilters extends OrcFiltersBase { case object FilterAction extends ActionType case object BuildAction extends ActionType + /** + * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then + * only building the remaining convertible nodes. + * + * Doing the conversion in this way avoids the computational complexity problems introduced by + * checking whether a node is convertible while building it. The approach implemented here has + * complexity that's linear in the size of the Filter tree - O(number of Filter nodes) - we run + * a single pass over the tree to trim it, and then another pass on the trimmed tree to convert + * the remaining nodes. + * + * The alternative approach of checking-while-building can (and did) result + * in exponential complexity in the height of the tree, causing perf problems with Filters with + * as few as ~35 nodes if they were skewed. + */ private def filterAndBuild( dataTypeMap: Map[String, DataType], expression: Filter, @@ -181,6 +195,25 @@ private[sql] object OrcFilters extends OrcFiltersBase { import org.apache.spark.sql.sources._ + // The performAction method can run both the filtering and building operations for a given + // node - we signify which one we want with the `actionType` parameter. + // + // There are a couple of benefits to coupling the two operations like this: + // 1. All the logic for a given predicate is grouped logically in the same place. You don't + // have to scroll across the whole file to see what the filter action for an And is while + // you're looking at the build action. + // 2. It's much easier to keep the implementations of the two operations up-to-date with + // each other. If the `filter` and `build` operations are implemented as separate case-matches + // in different methods, it's very easy to change one without appropriately updating the + // other. For example, if we add a new supported node type to `filter`, it would be very + // easy to forget to update `build` to support it too, thus leading to conversion errors. + // + // Doing things this way does have some annoying side effects: + // - We need to return an `Either`, with one action type always returning a Left and the other + // always returning a Right. + // - We always need to pass the canPartialPushDownConjuncts parameter even though the build + // action doesn't need it (because by the time we run the `build` operation, we know all + // remaining nodes are convertible). def performAction( actionType: ActionType, expression: Filter, @@ -341,7 +374,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { def updateBuilder(expression: Filter) = performAction(BuildAction, expression, canPartialPushDownConjuncts = true).right.get - val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) filteredExpression.foreach(updateBuilder) filteredExpression.map(_ => builder) From 1f2d5ec3bb36968030ba12cdb4154050fcb722c1 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 09:39:08 +0300 Subject: [PATCH 35/56] Remove now unused TrimmedFilter class --- .../sql/execution/datasources/orc/OrcFilters.scala | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index ba3e6471a811..f0f1a49b773b 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -143,20 +143,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => value } - /** - * A TrimmedFilter is a Filter that has been trimmed such that all the remaining nodes - * are convertible to ORC predicates. - * - * Since nothing in the underlying representation of the Filter is actually different from a - * regular Filter (the only difference is that we might remove some subtrees), this class is just - * a wrapper around a `Filter` value. The main benefits of using this class are readability - * and type safety (to signal that the respective functions only work with already trimmed - * filters). - * - * @param filter The underlying filter representation. - */ - private case class TrimmedFilter(filter: Filter) extends AnyVal - /** * Build a SearchArgument and return the builder so far. */ From ee25809d552150d2f3b442cb64fcc154a8f20dff Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 09:44:02 +0300 Subject: [PATCH 36/56] Apply changes to 2.3.4 version of the file --- .../datasources/orc/OrcFilters.scala | 394 +++++++++--------- 1 file changed, 195 insertions(+), 199 deletions(-) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 10f226a93903..6f19a0593446 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -143,20 +143,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => value } - /** - * A TrimmedFilter is a Filter that has been trimmed such that all the remaining nodes - * are convertible to ORC predicates. - * - * Since nothing in the underlying representation of the Filter is actually different from a - * regular Filter (the only difference is that we might remove some subtrees), this class is just - * a wrapper around a `Filter` value. The main benefits of using this class are readability - * and type safety (to signal that the respective functions only work with already trimmed - * filters). - * - * @param filter The underlying filter representation. - */ - private case class TrimmedFilter(filter: Filter) extends AnyVal - /** * Build a SearchArgument and return the builder so far. */ @@ -164,208 +150,218 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - trimNonConvertibleSubtrees(dataTypeMap, expression) - .map(createBuilder(dataTypeMap, _, builder)) + filterAndBuild(dataTypeMap, expression, builder) } - /** - * Transforms a `Filter` by removing all subtrees that are not convertible to an ORC - * SearchArgument. - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @return A TrimmedFilter that wraps the transformed `Filter`. - */ - private def trimNonConvertibleSubtrees( - dataTypeMap: Map[String, DataType], - expression: Filter): Option[TrimmedFilter] = { - trimNonConvertibleSubtreesImpl(dataTypeMap, expression, canPartialPushDownConjuncts = true) - .map(TrimmedFilter) - } + sealed trait ActionType + case object FilterAction extends ActionType + case object BuildAction extends ActionType /** - * Internal recursive implementation of the `trimNonConvertibleSubtrees` method. We use two - * separate methods here in order to avoid dealing with the wrapper `TrimmedFilter` classes - * in the recursive implementation here, and only wrap the final result in the outer function. + * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then + * only building the remaining convertible nodes. * - * NOTE: If you change the set of supported `Filter` types here, you need to modify - * `createBuilder` accordingly! + * Doing the conversion in this way avoids the computational complexity problems introduced by + * checking whether a node is convertible while building it. The approach implemented here has + * complexity that's linear in the size of the Filter tree - O(number of Filter nodes) - we run + * a single pass over the tree to trim it, and then another pass on the trimmed tree to convert + * the remaining nodes. * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the trimmed `Filter`. + * The alternative approach of checking-while-building can (and did) result + * in exponential complexity in the height of the tree, causing perf problems with Filters with + * as few as ~35 nodes if they were skewed. */ - private def trimNonConvertibleSubtreesImpl( + private def filterAndBuild( dataTypeMap: Map[String, DataType], expression: Filter, - canPartialPushDownConjuncts: Boolean): Option[Filter] = { + builder: Builder + ): Option[Builder] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) import org.apache.spark.sql.sources._ - expression match { - case And(left, right) => - // At here, it is not safe to just keep one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val lhs = - trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts = true) - val rhs = - trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts = true) - (lhs, rhs) match { - case (Some(l), Some(r)) => Some(And(l, r)) - case (Some(_), None) if canPartialPushDownConjuncts => lhs - case (None, Some(_)) if canPartialPushDownConjuncts => rhs - case _ => None - } - - case Or(left, right) => - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - for { - lhs: Filter <- - trimNonConvertibleSubtreesImpl(dataTypeMap, left, canPartialPushDownConjuncts) - rhs: Filter <- - trimNonConvertibleSubtreesImpl(dataTypeMap, right, canPartialPushDownConjuncts) - } yield Or(lhs, rhs) - - case Not(child) => - trimNonConvertibleSubtreesImpl(dataTypeMap, child, canPartialPushDownConjuncts = false) - .map(Not) - - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(expression) - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => Some(expression) - - case _ => None + // The performAction method can run both the filtering and building operations for a given + // node - we signify which one we want with the `actionType` parameter. + // + // There are a couple of benefits to coupling the two operations like this: + // 1. All the logic for a given predicate is grouped logically in the same place. You don't + // have to scroll across the whole file to see what the filter action for an And is while + // you're looking at the build action. + // 2. It's much easier to keep the implementations of the two operations up-to-date with + // each other. If the `filter` and `build` operations are implemented as separate case-matches + // in different methods, it's very easy to change one without appropriately updating the + // other. For example, if we add a new supported node type to `filter`, it would be very + // easy to forget to update `build` to support it too, thus leading to conversion errors. + // + // Doing things this way does have some annoying side effects: + // - We need to return an `Either`, with one action type always returning a Left and the other + // always returning a Right. + // - We always need to pass the canPartialPushDownConjuncts parameter even though the build + // action doesn't need it (because by the time we run the `build` operation, we know all + // remaining nodes are convertible). + def performAction( + actionType: ActionType, + expression: Filter, + canPartialPushDownConjuncts: Boolean): Either[Option[Filter], Unit] = { + expression match { + case And(left, right) => + actionType match { + case FilterAction => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported + // predicate can be safely removed. + val lhs = + performFilter(left, canPartialPushDownConjuncts = true) + val rhs = + performFilter(right, canPartialPushDownConjuncts = true) + (lhs, rhs) match { + case (Some(l), Some(r)) => Left(Some(And(l, r))) + case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) + case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) + case _ => Left(None) + } + case BuildAction => + builder.startAnd() + updateBuilder(left) + updateBuilder(right) + builder.end() + Right(Unit) + } + + case Or(left, right) => + actionType match { + case FilterAction => + Left(for { + lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) + rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) + } yield Or(lhs, rhs)) + case BuildAction => + builder.startOr() + updateBuilder(left) + updateBuilder(right) + builder.end() + Right(Unit) + } + + case Not(child) => + actionType match { + case FilterAction => + val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) + Left(filteredSubtree.map(Not(_))) + case BuildAction => + builder.startNot() + updateBuilder(child) + builder.end() + Right(Unit) + } + + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startAnd().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startNot().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case FilterAction => Left(Some(expression)) + case BuildAction => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end() + Right(Unit) + } + + case _ => + actionType match { + case FilterAction => Left(None) + case BuildAction => + throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") + } + } } - } - /** - * Build a SearchArgument for a Filter that has already been trimmed so as to only contain - * expressions that are convertible to a `SearchArgument`. This allows for a more efficient and - * more readable implementation since there's no need to check every node before converting it. - * - * NOTE: If you change the set of supported `Filter` types here, you need to modify - * `trimNonConvertibleSubtreesImpl` accordingly! - * - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the trimmed input filter predicates. - * @param builder the builder so far. - * @return - */ - private def createBuilder( - dataTypeMap: Map[String, DataType], - expression: TrimmedFilter, - builder: Builder): Builder = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute)) - - import org.apache.spark.sql.sources._ - def updateBuilder(subexpression: Filter): Unit = subexpression match { - case And(left, right) => - builder.startAnd() - updateBuilder(left) - updateBuilder(right) - builder.end() - - case Or(left, right) => - builder.startOr() - updateBuilder(left) - updateBuilder(right) - builder.end() - - case Not(child) => - builder.startNot() - updateBuilder(child) - builder.end() - - // 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`). + def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean) = + performAction(FilterAction, expression, canPartialPushDownConjuncts).left.get - case EqualTo(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - - case EqualNullSafe(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - - case LessThan(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - - case LessThanOrEqual(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - - case GreaterThan(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - - case GreaterThanOrEqual(attribute, value) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - - case IsNull(attribute) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startAnd().isNull(quotedName, getType(attribute)).end() - - case IsNotNull(attribute) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startNot().isNull(quotedName, getType(attribute)).end() - - case In(attribute, values) => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - builder.startAnd().in(quotedName, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - - // This case should never happen since this function only expects fully convertible filters. - // However, we return true as a safety measure in case something goes wrong. - case _ => builder.startAnd().literal(TruthValue.YES).end() - } + def updateBuilder(expression: Filter) = + performAction(BuildAction, expression, canPartialPushDownConjuncts = true).right.get - updateBuilder(expression.filter) - builder + val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) + filteredExpression.foreach(updateBuilder) + filteredExpression.map(_ => builder) } } From 2befb28b3ab2522cdab3a9a0e269138776a9513d Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 10:41:26 +0300 Subject: [PATCH 37/56] Don't filter predicates in convertibleFilters since they are already filtered when building --- .../datasources/orc/OrcFilters.scala | 40 ++----------------- .../datasources/orc/OrcFilters.scala | 40 ++----------------- 2 files changed, 8 insertions(+), 72 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index f0f1a49b773b..94d180991877 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -75,42 +75,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { schema: StructType, dataTypeMap: Map[String, DataType], filters: Seq[Filter]): Seq[Filter] = { - import org.apache.spark.sql.sources._ - - def convertibleFiltersHelper( - filter: Filter, - canPartialPushDown: Boolean): Option[Filter] = filter match { - case And(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - (leftResultOptional, rightResultOptional) match { - case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) - case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) - case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) - case _ => None - } - - case Or(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - if (leftResultOptional.isEmpty || rightResultOptional.isEmpty) { - None - } else { - Some(Or(leftResultOptional.get, rightResultOptional.get)) - } - case Not(pred) => - val resultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) - resultOptional.map(Not) - case other => - if (buildSearchArgument(dataTypeMap, other, newBuilder()).isDefined) { - Some(other) - } else { - None - } - } - filters.flatMap { filter => - convertibleFiltersHelper(filter, true) - } + for { + filter <- filters + _ <- buildSearchArgument(dataTypeMap, filter, newBuilder()) + } yield filter } /** diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6f19a0593446..f69b5cf46c54 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -75,42 +75,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { schema: StructType, dataTypeMap: Map[String, DataType], filters: Seq[Filter]): Seq[Filter] = { - import org.apache.spark.sql.sources._ - - def convertibleFiltersHelper( - filter: Filter, - canPartialPushDown: Boolean): Option[Filter] = filter match { - case And(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - (leftResultOptional, rightResultOptional) match { - case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) - case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) - case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) - case _ => None - } - - case Or(left, right) => - val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) - val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) - if (leftResultOptional.isEmpty || rightResultOptional.isEmpty) { - None - } else { - Some(Or(leftResultOptional.get, rightResultOptional.get)) - } - case Not(pred) => - val resultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) - resultOptional.map(Not) - case other => - if (buildSearchArgument(dataTypeMap, other, newBuilder()).isDefined) { - Some(other) - } else { - None - } - } - filters.flatMap { filter => - convertibleFiltersHelper(filter, true) - } + for { + filter <- filters + _ <- buildSearchArgument(dataTypeMap, filter, newBuilder()) + } yield filter } /** From 9ea2a6fe5dba780a880baf18a4dc66c8141ea863 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 10:43:34 +0300 Subject: [PATCH 38/56] Add OR partial pushdown comment back --- .../sql/execution/datasources/orc/OrcFilters.scala | 11 +++++++++++ .../sql/execution/datasources/orc/OrcFilters.scala | 11 +++++++++++ 2 files changed, 22 insertions(+) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 94d180991877..42740dd9a74b 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -208,6 +208,17 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Or(left, right) => actionType match { case FilterAction => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). Left(for { lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index f69b5cf46c54..431c9a0444e2 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -208,6 +208,17 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Or(left, right) => actionType match { case FilterAction => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). Left(for { lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) From b19792c4bb24a1366930beff37e172cc7f3c883f Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 10:49:20 +0300 Subject: [PATCH 39/56] Don't explicitly set canPartialPushDownPredicates = true when filtering ANDs --- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 6 ++---- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 6 ++---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 42740dd9a74b..176f4a434c5d 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -187,10 +187,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported // predicate can be safely removed. - val lhs = - performFilter(left, canPartialPushDownConjuncts = true) - val rhs = - performFilter(right, canPartialPushDownConjuncts = true) + val lhs = performFilter(left, canPartialPushDownConjuncts) + val rhs = performFilter(right, canPartialPushDownConjuncts) (lhs, rhs) match { case (Some(l), Some(r)) => Left(Some(And(l, r))) case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 431c9a0444e2..2e1cd81c1087 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -187,10 +187,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported // predicate can be safely removed. - val lhs = - performFilter(left, canPartialPushDownConjuncts = true) - val rhs = - performFilter(right, canPartialPushDownConjuncts = true) + val lhs = performFilter(left, canPartialPushDownConjuncts) + val rhs = performFilter(right, canPartialPushDownConjuncts) (lhs, rhs) match { case (Some(l), Some(r)) => Left(Some(And(l, r))) case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) From 3d1b28e993c99bd8b49a70fa88dd567de55e1166 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Mon, 27 May 2019 10:52:24 +0300 Subject: [PATCH 40/56] Add back comment about wrapping leaf predicates in an AND --- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 4 ++++ .../spark/sql/execution/datasources/orc/OrcFilters.scala | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 176f4a434c5d..6b56266dbce8 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -241,6 +241,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { Right(Unit) } + // 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`). + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { case FilterAction => Left(Some(expression)) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 2e1cd81c1087..c7a8d7d5ae95 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -241,6 +241,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { Right(Unit) } + // 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`). + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { case FilterAction => Left(Some(expression)) From 44d258303f585113c4e27952a1701fc31fc69caa Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 11:38:47 +0300 Subject: [PATCH 41/56] Code review comments --- .../datasources/orc/OrcFilters.scala | 70 +++++++++---------- .../datasources/orc/OrcFilters.scala | 70 +++++++++---------- 2 files changed, 68 insertions(+), 72 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6b56266dbce8..7ea197229394 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -122,8 +122,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } sealed trait ActionType - case object FilterAction extends ActionType - case object BuildAction extends ActionType + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType {} + case object BuildSearchArgument extends ActionType /** * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then @@ -142,8 +142,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { private def filterAndBuild( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder - ): Option[Builder] = { + builder: Builder): Option[Builder] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -170,12 +169,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { // remaining nodes are convertible). def performAction( actionType: ActionType, - expression: Filter, - canPartialPushDownConjuncts: Boolean): Either[Option[Filter], Unit] = { + expression: Filter): Either[Option[Filter], Unit] = { expression match { case And(left, right) => actionType match { - case FilterAction => + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // At here, it is not safe to just keep one side and remove the other side // if we do not understand what the parent filters are. // @@ -195,7 +193,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) case _ => Left(None) } - case BuildAction => + case BuildSearchArgument => builder.startAnd() updateBuilder(left) updateBuilder(right) @@ -205,7 +203,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Or(left, right) => actionType match { - case FilterAction => + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // The Or predicate is convertible when both of its children can be pushed down. // That is to say, if one/both of the children can be partially pushed down, the Or // predicate can be partially pushed down as well. @@ -221,7 +219,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) } yield Or(lhs, rhs)) - case BuildAction => + case BuildSearchArgument => builder.startOr() updateBuilder(left) updateBuilder(right) @@ -231,10 +229,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Not(child) => actionType match { - case FilterAction => + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) Left(filteredSubtree.map(Not(_))) - case BuildAction => + case BuildSearchArgument => builder.startNot() updateBuilder(child) builder.end() @@ -247,8 +245,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() @@ -256,8 +254,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() @@ -265,8 +263,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() @@ -274,8 +272,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() @@ -283,8 +281,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() @@ -292,8 +290,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() @@ -301,24 +299,24 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() Right(Unit) } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() Right(Unit) } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) builder.startAnd().in(quotedName, getType(attribute), @@ -328,18 +326,18 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => actionType match { - case FilterAction => Left(None) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(None) + case BuildSearchArgument => throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") } } } - def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean) = - performAction(FilterAction, expression, canPartialPushDownConjuncts).left.get + def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression).left.get - def updateBuilder(expression: Filter) = - performAction(BuildAction, expression, canPartialPushDownConjuncts = true).right.get + def updateBuilder(expression: Filter): Unit = + performAction(BuildSearchArgument, expression).right.get val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) filteredExpression.foreach(updateBuilder) diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index c7a8d7d5ae95..f4e301fdaeb9 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -122,8 +122,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } sealed trait ActionType - case object FilterAction extends ActionType - case object BuildAction extends ActionType + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType {} + case object BuildSearchArgument extends ActionType /** * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then @@ -142,8 +142,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { private def filterAndBuild( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder - ): Option[Builder] = { + builder: Builder): Option[Builder] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -170,12 +169,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { // remaining nodes are convertible). def performAction( actionType: ActionType, - expression: Filter, - canPartialPushDownConjuncts: Boolean): Either[Option[Filter], Unit] = { + expression: Filter): Either[Option[Filter], Unit] = { expression match { case And(left, right) => actionType match { - case FilterAction => + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // At here, it is not safe to just keep one side and remove the other side // if we do not understand what the parent filters are. // @@ -195,7 +193,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) case _ => Left(None) } - case BuildAction => + case BuildSearchArgument => builder.startAnd() updateBuilder(left) updateBuilder(right) @@ -205,7 +203,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Or(left, right) => actionType match { - case FilterAction => + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // The Or predicate is convertible when both of its children can be pushed down. // That is to say, if one/both of the children can be partially pushed down, the Or // predicate can be partially pushed down as well. @@ -221,7 +219,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) } yield Or(lhs, rhs)) - case BuildAction => + case BuildSearchArgument => builder.startOr() updateBuilder(left) updateBuilder(right) @@ -231,10 +229,10 @@ private[sql] object OrcFilters extends OrcFiltersBase { case Not(child) => actionType match { - case FilterAction => + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) Left(filteredSubtree.map(Not(_))) - case BuildAction => + case BuildSearchArgument => builder.startNot() updateBuilder(child) builder.end() @@ -247,8 +245,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() @@ -256,8 +254,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() @@ -265,8 +263,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() @@ -274,8 +272,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() @@ -283,8 +281,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() @@ -292,8 +290,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() @@ -301,24 +299,24 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() Right(Unit) } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() Right(Unit) } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case FilterAction => Left(Some(expression)) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) builder.startAnd().in(quotedName, getType(attribute), @@ -328,18 +326,18 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => actionType match { - case FilterAction => Left(None) - case BuildAction => + case TrimUnconvertibleFilters(_) => Left(None) + case BuildSearchArgument => throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") } } } - def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean) = - performAction(FilterAction, expression, canPartialPushDownConjuncts).left.get + def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression).left.get - def updateBuilder(expression: Filter) = - performAction(BuildAction, expression, canPartialPushDownConjuncts = true).right.get + def updateBuilder(expression: Filter): Unit = + performAction(BuildSearchArgument, expression).right.get val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) filteredExpression.foreach(updateBuilder) From 9a6fd4d47d4e56851114d0ec8a05556b49b78d7f Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 12:02:54 +0300 Subject: [PATCH 42/56] Initial version that passes partial OR pushdown test --- .../datasources/orc/OrcFilters.scala | 24 ++++++++++++++----- .../datasources/orc/OrcFilters.scala | 24 ++++++++++++++----- 2 files changed, 36 insertions(+), 12 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 7ea197229394..2757cf0cad67 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -77,8 +77,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { filters: Seq[Filter]): Seq[Filter] = { for { filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, newBuilder()) - } yield filter + trimmedFilter <- trimUnconvertibleFilters(dataTypeMap, filter, newBuilder()) + } yield trimmedFilter } /** @@ -111,6 +111,13 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => value } + private def trimUnconvertibleFilters( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Option[Filter] = { + filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = false).left.get + } + /** * Build a SearchArgument and return the builder so far. */ @@ -118,7 +125,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - filterAndBuild(dataTypeMap, expression, builder) + filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = true).right.get } sealed trait ActionType @@ -142,7 +149,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { private def filterAndBuild( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder): Option[Builder] = { + builder: Builder, + shouldBuildSearchArgument: Boolean): Either[Option[Filter], Option[Builder]] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -340,8 +348,12 @@ private[sql] object OrcFilters extends OrcFiltersBase { performAction(BuildSearchArgument, expression).right.get val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) - filteredExpression.foreach(updateBuilder) - filteredExpression.map(_ => builder) + if (shouldBuildSearchArgument) { + filteredExpression.foreach(updateBuilder) + Right(filteredExpression.map(_ => builder)) + } else { + Left(filteredExpression) + } } } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index f4e301fdaeb9..2a22404e638d 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -77,8 +77,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { filters: Seq[Filter]): Seq[Filter] = { for { filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, newBuilder()) - } yield filter + trimmedFilter <- trimUnconvertibleFilters(dataTypeMap, filter, newBuilder()) + } yield trimmedFilter } /** @@ -111,6 +111,13 @@ private[sql] object OrcFilters extends OrcFiltersBase { case _ => value } + private def trimUnconvertibleFilters( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Option[Filter] = { + filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = false).left.get + } + /** * Build a SearchArgument and return the builder so far. */ @@ -118,7 +125,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Builder] = { - filterAndBuild(dataTypeMap, expression, builder) + filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = true).right.get } sealed trait ActionType @@ -142,7 +149,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { private def filterAndBuild( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder): Option[Builder] = { + builder: Builder, + shouldBuildSearchArgument: Boolean): Either[Option[Filter], Option[Builder]] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -340,7 +348,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { performAction(BuildSearchArgument, expression).right.get val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) - filteredExpression.foreach(updateBuilder) - filteredExpression.map(_ => builder) + if (shouldBuildSearchArgument) { + filteredExpression.foreach(updateBuilder) + Right(filteredExpression.map(_ => builder)) + } else { + Left(filteredExpression) + } } } From aa8a629f7a463245e02f0b43692a6887b2a710cc Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 12:23:19 +0300 Subject: [PATCH 43/56] Make interface for filtering and building nicer --- .../datasources/orc/OrcFilters.scala | 420 +++++++++--------- .../datasources/orc/OrcFilters.scala | 420 +++++++++--------- 2 files changed, 418 insertions(+), 422 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 2757cf0cad67..8187fd1e11d3 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -115,23 +115,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Filter] = { - filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = false).left.get + performFilter(dataTypeMap, expression, canPartialPushDownConjuncts = true) } - /** - * Build a SearchArgument and return the builder so far. - */ - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder): Option[Builder] = { - filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = true).right.get - } - - sealed trait ActionType - case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType {} - case object BuildSearchArgument extends ActionType - /** * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then * only building the remaining convertible nodes. @@ -146,214 +132,226 @@ private[sql] object OrcFilters extends OrcFiltersBase { * in exponential complexity in the height of the tree, causing perf problems with Filters with * as few as ~35 nodes if they were skewed. */ - private def filterAndBuild( + private def buildSearchArgument( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder, - shouldBuildSearchArgument: Boolean): Either[Option[Filter], Option[Builder]] = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute)) - - import org.apache.spark.sql.sources._ + builder: Builder): Option[Builder] = { + val filteredExpression: Option[Filter] = performFilter( + dataTypeMap, + expression, + canPartialPushDownConjuncts = true) + filteredExpression.foreach(updateBuilder(dataTypeMap, _, builder)) + filteredExpression.map(_ => builder) + } - // The performAction method can run both the filtering and building operations for a given - // node - we signify which one we want with the `actionType` parameter. - // - // There are a couple of benefits to coupling the two operations like this: - // 1. All the logic for a given predicate is grouped logically in the same place. You don't - // have to scroll across the whole file to see what the filter action for an And is while - // you're looking at the build action. - // 2. It's much easier to keep the implementations of the two operations up-to-date with - // each other. If the `filter` and `build` operations are implemented as separate case-matches - // in different methods, it's very easy to change one without appropriately updating the - // other. For example, if we add a new supported node type to `filter`, it would be very - // easy to forget to update `build` to support it too, thus leading to conversion errors. - // - // Doing things this way does have some annoying side effects: - // - We need to return an `Either`, with one action type always returning a Left and the other - // always returning a Right. - // - We always need to pass the canPartialPushDownConjuncts parameter even though the build - // action doesn't need it (because by the time we run the `build` operation, we know all - // remaining nodes are convertible). - def performAction( - actionType: ActionType, - expression: Filter): Either[Option[Filter], Unit] = { - expression match { - case And(left, right) => - actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - // At here, it is not safe to just keep one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported - // predicate can be safely removed. - val lhs = performFilter(left, canPartialPushDownConjuncts) - val rhs = performFilter(right, canPartialPushDownConjuncts) - (lhs, rhs) match { - case (Some(l), Some(r)) => Left(Some(And(l, r))) - case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) - case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) - case _ => Left(None) - } - case BuildSearchArgument => - builder.startAnd() - updateBuilder(left) - updateBuilder(right) - builder.end() - Right(Unit) - } + import org.apache.spark.sql.sources._ - case Or(left, right) => - actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - Left(for { - lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) - rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) - } yield Or(lhs, rhs)) - case BuildSearchArgument => - builder.startOr() - updateBuilder(left) - updateBuilder(right) - builder.end() - Right(Unit) - } + sealed trait ActionType + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType + case class BuildSearchArgument(builder: Builder) extends ActionType - case Not(child) => - actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) - Left(filteredSubtree.map(Not(_))) - case BuildSearchArgument => - builder.startNot() - updateBuilder(child) - builder.end() - Right(Unit) - } + // The performAction method can run both the filtering and building operations for a given + // node - we signify which one we want with the `actionType` parameter. + // + // There are a couple of benefits to coupling the two operations like this: + // 1. All the logic for a given predicate is grouped logically in the same place. You don't + // have to scroll across the whole file to see what the filter action for an And is while + // you're looking at the build action. + // 2. It's much easier to keep the implementations of the two operations up-to-date with + // each other. If the `filter` and `build` operations are implemented as separate case-matches + // in different methods, it's very easy to change one without appropriately updating the + // other. For example, if we add a new supported node type to `filter`, it would be very + // easy to forget to update `build` to support it too, thus leading to conversion errors. + // + // Doing things this way does have some annoying side effects: + // - We need to return an `Either`, with one action type always returning a Left and the other + // always returning a Right. + // - We always need to pass the canPartialPushDownConjuncts parameter even though the build + // action doesn't need it (because by the time we run the `build` operation, we know all + // remaining nodes are convertible). + def performAction( + actionType: ActionType, + dataTypeMap: Map[String, DataType], + expression: Filter): Either[Option[Filter], Unit] = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) - // 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`). + expression match { + case And(left, right) => + actionType match { + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported + // predicate can be safely removed. + val lhs = performFilter(dataTypeMap, left, canPartialPushDownConjuncts) + val rhs = performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + (lhs, rhs) match { + case (Some(l), Some(r)) => Left(Some(And(l, r))) + case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) + case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) + case _ => Left(None) + } + case BuildSearchArgument(builder) => + builder.startAnd() + updateBuilder(dataTypeMap, left, builder) + updateBuilder(dataTypeMap, right, builder) + builder.end() + Right(Unit) + } - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startAnd().isNull(quotedName, getType(attribute)).end() - Right(Unit) - } - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startNot().isNull(quotedName, getType(attribute)).end() - Right(Unit) - } - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - builder.startAnd().in(quotedName, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - Right(Unit) - } + case Or(left, right) => + actionType match { + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + Left(for { + lhs: Filter <- performFilter(dataTypeMap, left, canPartialPushDownConjuncts) + rhs: Filter <- performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + } yield Or(lhs, rhs)) + case BuildSearchArgument(builder) => + builder.startOr() + updateBuilder(dataTypeMap, left, builder) + updateBuilder(dataTypeMap, right, builder) + builder.end() + Right(Unit) + } - case _ => - actionType match { - case TrimUnconvertibleFilters(_) => Left(None) - case BuildSearchArgument => - throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") - } - } - } + case Not(child) => + actionType match { + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + val filteredSubtree = + performFilter(dataTypeMap, child, canPartialPushDownConjuncts = false) + Left(filteredSubtree.map(Not(_))) + case BuildSearchArgument(builder) => + builder.startNot() + updateBuilder(dataTypeMap, child, builder) + builder.end() + Right(Unit) + } - def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = - performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression).left.get + // 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`). - def updateBuilder(expression: Filter): Unit = - performAction(BuildSearchArgument, expression).right.get + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startAnd().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startNot().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end() + Right(Unit) + } - val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) - if (shouldBuildSearchArgument) { - filteredExpression.foreach(updateBuilder) - Right(filteredExpression.map(_ => builder)) - } else { - Left(filteredExpression) + case _ => + actionType match { + case TrimUnconvertibleFilters(_) => Left(None) + case BuildSearchArgument(builder) => + throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") + } } } + + private def performFilter( + dataTypeMap: Map[String, DataType], + expression: Filter, + canPartialPushDownConjuncts: Boolean): Option[Filter] = + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), dataTypeMap, expression) + .left + .get + + private def updateBuilder( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Unit = + performAction(BuildSearchArgument(builder), dataTypeMap, expression).right.get } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 2a22404e638d..97701ed25869 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -115,23 +115,9 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Option[Filter] = { - filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = false).left.get + performFilter(dataTypeMap, expression, canPartialPushDownConjuncts = true) } - /** - * Build a SearchArgument and return the builder so far. - */ - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder): Option[Builder] = { - filterAndBuild(dataTypeMap, expression, builder, shouldBuildSearchArgument = true).right.get - } - - sealed trait ActionType - case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType {} - case object BuildSearchArgument extends ActionType - /** * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then * only building the remaining convertible nodes. @@ -146,213 +132,225 @@ private[sql] object OrcFilters extends OrcFiltersBase { * in exponential complexity in the height of the tree, causing perf problems with Filters with * as few as ~35 nodes if they were skewed. */ - private def filterAndBuild( + private def buildSearchArgument( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder, - shouldBuildSearchArgument: Boolean): Either[Option[Filter], Option[Builder]] = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute)) - - import org.apache.spark.sql.sources._ + builder: Builder): Option[Builder] = { + val filteredExpression: Option[Filter] = performFilter( + dataTypeMap, + expression, + canPartialPushDownConjuncts = true) + filteredExpression.foreach(updateBuilder(dataTypeMap, _, builder)) + filteredExpression.map(_ => builder) + } - // The performAction method can run both the filtering and building operations for a given - // node - we signify which one we want with the `actionType` parameter. - // - // There are a couple of benefits to coupling the two operations like this: - // 1. All the logic for a given predicate is grouped logically in the same place. You don't - // have to scroll across the whole file to see what the filter action for an And is while - // you're looking at the build action. - // 2. It's much easier to keep the implementations of the two operations up-to-date with - // each other. If the `filter` and `build` operations are implemented as separate case-matches - // in different methods, it's very easy to change one without appropriately updating the - // other. For example, if we add a new supported node type to `filter`, it would be very - // easy to forget to update `build` to support it too, thus leading to conversion errors. - // - // Doing things this way does have some annoying side effects: - // - We need to return an `Either`, with one action type always returning a Left and the other - // always returning a Right. - // - We always need to pass the canPartialPushDownConjuncts parameter even though the build - // action doesn't need it (because by the time we run the `build` operation, we know all - // remaining nodes are convertible). - def performAction( - actionType: ActionType, - expression: Filter): Either[Option[Filter], Unit] = { - expression match { - case And(left, right) => - actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - // At here, it is not safe to just keep one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported - // predicate can be safely removed. - val lhs = performFilter(left, canPartialPushDownConjuncts) - val rhs = performFilter(right, canPartialPushDownConjuncts) - (lhs, rhs) match { - case (Some(l), Some(r)) => Left(Some(And(l, r))) - case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) - case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) - case _ => Left(None) - } - case BuildSearchArgument => - builder.startAnd() - updateBuilder(left) - updateBuilder(right) - builder.end() - Right(Unit) - } + import org.apache.spark.sql.sources._ - case Or(left, right) => - actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - Left(for { - lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) - rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) - } yield Or(lhs, rhs)) - case BuildSearchArgument => - builder.startOr() - updateBuilder(left) - updateBuilder(right) - builder.end() - Right(Unit) - } + sealed trait ActionType + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType + case class BuildSearchArgument(builder: Builder) extends ActionType - case Not(child) => - actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) - Left(filteredSubtree.map(Not(_))) - case BuildSearchArgument => - builder.startNot() - updateBuilder(child) - builder.end() - Right(Unit) - } + // The performAction method can run both the filtering and building operations for a given + // node - we signify which one we want with the `actionType` parameter. + // + // There are a couple of benefits to coupling the two operations like this: + // 1. All the logic for a given predicate is grouped logically in the same place. You don't + // have to scroll across the whole file to see what the filter action for an And is while + // you're looking at the build action. + // 2. It's much easier to keep the implementations of the two operations up-to-date with + // each other. If the `filter` and `build` operations are implemented as separate case-matches + // in different methods, it's very easy to change one without appropriately updating the + // other. For example, if we add a new supported node type to `filter`, it would be very + // easy to forget to update `build` to support it too, thus leading to conversion errors. + // + // Doing things this way does have some annoying side effects: + // - We need to return an `Either`, with one action type always returning a Left and the other + // always returning a Right. + // - We always need to pass the canPartialPushDownConjuncts parameter even though the build + // action doesn't need it (because by the time we run the `build` operation, we know all + // remaining nodes are convertible). + def performAction( + actionType: ActionType, + dataTypeMap: Map[String, DataType], + expression: Filter): Either[Option[Filter], Unit] = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) - // 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`). + expression match { + case And(left, right) => + actionType match { + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported + // predicate can be safely removed. + val lhs = performFilter(dataTypeMap, left, canPartialPushDownConjuncts) + val rhs = performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + (lhs, rhs) match { + case (Some(l), Some(r)) => Left(Some(And(l, r))) + case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) + case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) + case _ => Left(None) + } + case BuildSearchArgument(builder) => + builder.startAnd() + updateBuilder(dataTypeMap, left, builder) + updateBuilder(dataTypeMap, right, builder) + builder.end() + Right(Unit) + } - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) - } - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startAnd().isNull(quotedName, getType(attribute)).end() - Right(Unit) - } - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - builder.startNot().isNull(quotedName, getType(attribute)).end() - Right(Unit) - } - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) - case BuildSearchArgument => - val quotedName = quoteAttributeNameIfNeeded(attribute) - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - builder.startAnd().in(quotedName, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - Right(Unit) - } + case Or(left, right) => + actionType match { + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + Left(for { + lhs: Filter <- performFilter(dataTypeMap, left, canPartialPushDownConjuncts) + rhs: Filter <- performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + } yield Or(lhs, rhs)) + case BuildSearchArgument(builder) => + builder.startOr() + updateBuilder(dataTypeMap, left, builder) + updateBuilder(dataTypeMap, right, builder) + builder.end() + Right(Unit) + } - case _ => - actionType match { - case TrimUnconvertibleFilters(_) => Left(None) - case BuildSearchArgument => - throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") - } - } - } + case Not(child) => + actionType match { + case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + val filteredSubtree = + performFilter(dataTypeMap, child, canPartialPushDownConjuncts = false) + Left(filteredSubtree.map(Not(_))) + case BuildSearchArgument(builder) => + builder.startNot() + updateBuilder(dataTypeMap, child, builder) + builder.end() + Right(Unit) + } - def performFilter(expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = - performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression).left.get + // 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`). - def updateBuilder(expression: Filter): Unit = - performAction(BuildSearchArgument, expression).right.get + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() + Right(Unit) + } + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startAnd().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + builder.startNot().isNull(quotedName, getType(attribute)).end() + Right(Unit) + } + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + actionType match { + case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case BuildSearchArgument(builder) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end() + Right(Unit) + } - val filteredExpression = performFilter(expression, canPartialPushDownConjuncts = true) - if (shouldBuildSearchArgument) { - filteredExpression.foreach(updateBuilder) - Right(filteredExpression.map(_ => builder)) - } else { - Left(filteredExpression) + case _ => + actionType match { + case TrimUnconvertibleFilters(_) => Left(None) + case BuildSearchArgument(builder) => + throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") + } } } + + private def performFilter( + dataTypeMap: Map[String, DataType], + expression: Filter, + canPartialPushDownConjuncts: Boolean): Option[Filter] = + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), dataTypeMap, expression) + .left + .get + + private def updateBuilder( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Unit = + performAction(BuildSearchArgument(builder), dataTypeMap, expression).right.get } From fa9bf7093978826928ead83d1f25ac9ac64747f8 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 12:36:09 +0300 Subject: [PATCH 44/56] Move actual conversion functionality into a separate class --- .../datasources/orc/OrcFiltersBase.scala | 4 +- .../datasources/orc/OrcFilters.scala | 54 ++++++++++--------- .../datasources/orc/OrcFilters.scala | 54 ++++++++++--------- 3 files changed, 58 insertions(+), 54 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 8d4898a6b85c..0b5658715377 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 @@ -38,7 +38,7 @@ trait 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. - protected def quoteAttributeNameIfNeeded(name: String) : String = { + protected[sql] def quoteAttributeNameIfNeeded(name: String) : String = { if (!name.contains("`") && name.contains(".")) { s"`$name`" } else { @@ -50,7 +50,7 @@ trait OrcFiltersBase { * Return true if this is a searchable type in ORC. * Both CharType and VarcharType are cleaned at AstBuilder. */ - protected def isSearchableType(dataType: DataType) = dataType match { + protected[sql] def isSearchableType(dataType: DataType) = dataType match { case BinaryType => false case _: AtomicType => true case _ => false diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 8187fd1e11d3..753a4d9a68f0 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -63,11 +63,12 @@ private[sql] object OrcFilters extends OrcFiltersBase { */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { // Combines all convertible filters using `And` to produce a single conjunction conjunction <- buildTree(convertibleFilters(schema, dataTypeMap, filters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate - builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) + builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() } @@ -75,12 +76,19 @@ private[sql] object OrcFilters extends OrcFiltersBase { schema: StructType, dataTypeMap: Map[String, DataType], filters: Seq[Filter]): Seq[Filter] = { + val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { filter <- filters - trimmedFilter <- trimUnconvertibleFilters(dataTypeMap, filter, newBuilder()) + trimmedFilter <- orcFilterConverter.trimUnconvertibleFilters(filter, newBuilder()) } yield trimmedFilter } +} + +private class OrcFilterConverter( + val dataTypeMap: Map[String, DataType] +) { + /** * Get PredicateLeafType which is corresponding to the given DataType. */ @@ -110,12 +118,13 @@ private[sql] object OrcFilters extends OrcFiltersBase { new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) case _ => value } + import org.apache.spark.sql.sources._ + import OrcFilters._ - private def trimUnconvertibleFilters( - dataTypeMap: Map[String, DataType], + private[sql] def trimUnconvertibleFilters( expression: Filter, builder: Builder): Option[Filter] = { - performFilter(dataTypeMap, expression, canPartialPushDownConjuncts = true) + performFilter(expression, canPartialPushDownConjuncts = true) } /** @@ -132,20 +141,16 @@ private[sql] object OrcFilters extends OrcFiltersBase { * in exponential complexity in the height of the tree, causing perf problems with Filters with * as few as ~35 nodes if they were skewed. */ - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], + private[sql] def buildSearchArgument( expression: Filter, builder: Builder): Option[Builder] = { val filteredExpression: Option[Filter] = performFilter( - dataTypeMap, expression, canPartialPushDownConjuncts = true) - filteredExpression.foreach(updateBuilder(dataTypeMap, _, builder)) + filteredExpression.foreach(updateBuilder(_, builder)) filteredExpression.map(_ => builder) } - import org.apache.spark.sql.sources._ - sealed trait ActionType case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType case class BuildSearchArgument(builder: Builder) extends ActionType @@ -171,7 +176,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { // remaining nodes are convertible). def performAction( actionType: ActionType, - dataTypeMap: Map[String, DataType], expression: Filter): Either[Option[Filter], Unit] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -191,8 +195,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported // predicate can be safely removed. - val lhs = performFilter(dataTypeMap, left, canPartialPushDownConjuncts) - val rhs = performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + val lhs = performFilter(left, canPartialPushDownConjuncts) + val rhs = performFilter(right, canPartialPushDownConjuncts) (lhs, rhs) match { case (Some(l), Some(r)) => Left(Some(And(l, r))) case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) @@ -201,8 +205,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case BuildSearchArgument(builder) => builder.startAnd() - updateBuilder(dataTypeMap, left, builder) - updateBuilder(dataTypeMap, right, builder) + updateBuilder(left, builder) + updateBuilder(right, builder) builder.end() Right(Unit) } @@ -222,13 +226,13 @@ private[sql] object OrcFilters extends OrcFiltersBase { // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). Left(for { - lhs: Filter <- performFilter(dataTypeMap, left, canPartialPushDownConjuncts) - rhs: Filter <- performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) + rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) } yield Or(lhs, rhs)) case BuildSearchArgument(builder) => builder.startOr() - updateBuilder(dataTypeMap, left, builder) - updateBuilder(dataTypeMap, right, builder) + updateBuilder(left, builder) + updateBuilder(right, builder) builder.end() Right(Unit) } @@ -237,11 +241,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { actionType match { case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => val filteredSubtree = - performFilter(dataTypeMap, child, canPartialPushDownConjuncts = false) + performFilter(child, canPartialPushDownConjuncts = false) Left(filteredSubtree.map(Not(_))) case BuildSearchArgument(builder) => builder.startNot() - updateBuilder(dataTypeMap, child, builder) + updateBuilder(child, builder) builder.end() Right(Unit) } @@ -341,17 +345,15 @@ private[sql] object OrcFilters extends OrcFiltersBase { } private def performFilter( - dataTypeMap: Map[String, DataType], expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = - performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), dataTypeMap, expression) + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression) .left .get private def updateBuilder( - dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Unit = - performAction(BuildSearchArgument(builder), dataTypeMap, expression).right.get + performAction(BuildSearchArgument(builder), expression).right.get } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 97701ed25869..6f1e79ba760e 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -63,11 +63,12 @@ private[sql] object OrcFilters extends OrcFiltersBase { */ def createFilter(schema: StructType, filters: Seq[Filter]): Option[SearchArgument] = { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap + val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { // Combines all convertible filters using `And` to produce a single conjunction conjunction <- buildTree(convertibleFilters(schema, dataTypeMap, filters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate - builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) + builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() } @@ -75,12 +76,19 @@ private[sql] object OrcFilters extends OrcFiltersBase { schema: StructType, dataTypeMap: Map[String, DataType], filters: Seq[Filter]): Seq[Filter] = { + val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { filter <- filters - trimmedFilter <- trimUnconvertibleFilters(dataTypeMap, filter, newBuilder()) + trimmedFilter <- orcFilterConverter.trimUnconvertibleFilters(filter, newBuilder()) } yield trimmedFilter } +} + +private class OrcFilterConverter( + val dataTypeMap: Map[String, DataType] +) { + /** * Get PredicateLeafType which is corresponding to the given DataType. */ @@ -110,12 +118,13 @@ private[sql] object OrcFilters extends OrcFiltersBase { new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) case _ => value } + import org.apache.spark.sql.sources._ + import OrcFilters._ - private def trimUnconvertibleFilters( - dataTypeMap: Map[String, DataType], + private[sql] def trimUnconvertibleFilters( expression: Filter, builder: Builder): Option[Filter] = { - performFilter(dataTypeMap, expression, canPartialPushDownConjuncts = true) + performFilter(expression, canPartialPushDownConjuncts = true) } /** @@ -132,20 +141,16 @@ private[sql] object OrcFilters extends OrcFiltersBase { * in exponential complexity in the height of the tree, causing perf problems with Filters with * as few as ~35 nodes if they were skewed. */ - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], + private[sql] def buildSearchArgument( expression: Filter, builder: Builder): Option[Builder] = { val filteredExpression: Option[Filter] = performFilter( - dataTypeMap, expression, canPartialPushDownConjuncts = true) - filteredExpression.foreach(updateBuilder(dataTypeMap, _, builder)) + filteredExpression.foreach(updateBuilder(_, builder)) filteredExpression.map(_ => builder) } - import org.apache.spark.sql.sources._ - sealed trait ActionType case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType case class BuildSearchArgument(builder: Builder) extends ActionType @@ -171,7 +176,6 @@ private[sql] object OrcFilters extends OrcFiltersBase { // remaining nodes are convertible). def performAction( actionType: ActionType, - dataTypeMap: Map[String, DataType], expression: Filter): Either[Option[Filter], Unit] = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -191,8 +195,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported // predicate can be safely removed. - val lhs = performFilter(dataTypeMap, left, canPartialPushDownConjuncts) - val rhs = performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + val lhs = performFilter(left, canPartialPushDownConjuncts) + val rhs = performFilter(right, canPartialPushDownConjuncts) (lhs, rhs) match { case (Some(l), Some(r)) => Left(Some(And(l, r))) case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) @@ -201,8 +205,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { } case BuildSearchArgument(builder) => builder.startAnd() - updateBuilder(dataTypeMap, left, builder) - updateBuilder(dataTypeMap, right, builder) + updateBuilder(left, builder) + updateBuilder(right, builder) builder.end() Right(Unit) } @@ -222,13 +226,13 @@ private[sql] object OrcFilters extends OrcFiltersBase { // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). Left(for { - lhs: Filter <- performFilter(dataTypeMap, left, canPartialPushDownConjuncts) - rhs: Filter <- performFilter(dataTypeMap, right, canPartialPushDownConjuncts) + lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) + rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) } yield Or(lhs, rhs)) case BuildSearchArgument(builder) => builder.startOr() - updateBuilder(dataTypeMap, left, builder) - updateBuilder(dataTypeMap, right, builder) + updateBuilder(left, builder) + updateBuilder(right, builder) builder.end() Right(Unit) } @@ -237,11 +241,11 @@ private[sql] object OrcFilters extends OrcFiltersBase { actionType match { case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => val filteredSubtree = - performFilter(dataTypeMap, child, canPartialPushDownConjuncts = false) + performFilter(child, canPartialPushDownConjuncts = false) Left(filteredSubtree.map(Not(_))) case BuildSearchArgument(builder) => builder.startNot() - updateBuilder(dataTypeMap, child, builder) + updateBuilder(child, builder) builder.end() Right(Unit) } @@ -341,16 +345,14 @@ private[sql] object OrcFilters extends OrcFiltersBase { } private def performFilter( - dataTypeMap: Map[String, DataType], expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = - performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), dataTypeMap, expression) + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression) .left .get private def updateBuilder( - dataTypeMap: Map[String, DataType], expression: Filter, builder: Builder): Unit = - performAction(BuildSearchArgument(builder), dataTypeMap, expression).right.get + performAction(BuildSearchArgument(builder), expression).right.get } From caf013d6453085e6abb0196185ad448d133b182a Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 12:39:35 +0300 Subject: [PATCH 45/56] Improve filter interface --- .../sql/execution/datasources/orc/OrcFilters.scala | 10 +++------- .../sql/execution/datasources/orc/OrcFilters.scala | 10 +++------- 2 files changed, 6 insertions(+), 14 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 753a4d9a68f0..67494be65716 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -77,10 +77,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], filters: Seq[Filter]): Seq[Filter] = { val orcFilterConverter = new OrcFilterConverter(dataTypeMap) - for { - filter <- filters - trimmedFilter <- orcFilterConverter.trimUnconvertibleFilters(filter, newBuilder()) - } yield trimmedFilter + filters.flatMap(orcFilterConverter.trimUnconvertibleFilters) } } @@ -118,12 +115,11 @@ private class OrcFilterConverter( new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) case _ => value } + import org.apache.spark.sql.sources._ import OrcFilters._ - private[sql] def trimUnconvertibleFilters( - expression: Filter, - builder: Builder): Option[Filter] = { + private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { performFilter(expression, canPartialPushDownConjuncts = true) } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6f1e79ba760e..747c5df3a2a5 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -77,10 +77,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { dataTypeMap: Map[String, DataType], filters: Seq[Filter]): Seq[Filter] = { val orcFilterConverter = new OrcFilterConverter(dataTypeMap) - for { - filter <- filters - trimmedFilter <- orcFilterConverter.trimUnconvertibleFilters(filter, newBuilder()) - } yield trimmedFilter + filters.flatMap(orcFilterConverter.trimUnconvertibleFilters) } } @@ -118,12 +115,11 @@ private class OrcFilterConverter( new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) case _ => value } + import org.apache.spark.sql.sources._ import OrcFilters._ - private[sql] def trimUnconvertibleFilters( - expression: Filter, - builder: Builder): Option[Filter] = { + private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { performFilter(expression, canPartialPushDownConjuncts = true) } From 323b4e27401d6ab1a71e43a74f94c13e7e627dc5 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 12:40:41 +0300 Subject: [PATCH 46/56] Fix comment --- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 3 ++- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 67494be65716..6226d6da8918 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -30,7 +30,8 @@ import org.apache.spark.sql.types._ * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. * * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and - * conversion code paths to make sure we only convert predicates that are known to be convertible. + * conversion passes through the Filter to make sure we only convert predicates that are known + * to be convertible. * * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 747c5df3a2a5..35c77e03d15a 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -30,7 +30,8 @@ import org.apache.spark.sql.types._ * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. * * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and - * conversion code paths to make sure we only convert predicates that are known to be convertible. + * conversion passes through the Filter to make sure we only convert predicates that are known + * to be convertible. * * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite From aeaf8a147189bd8f77ebfec8f40249afdfc67782 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 28 May 2019 12:44:01 +0300 Subject: [PATCH 47/56] Add comment to updateBuilder --- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 4 ++++ .../spark/sql/execution/datasources/orc/OrcFilters.scala | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6226d6da8918..40a4ed1b04d0 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -348,6 +348,10 @@ private class OrcFilterConverter( .left .get + /** + * Builds a SearchArgument for the given Filter. This method should only be called on Filters + * that have previously been trimmed to remove unsupported sub-Filters! + */ private def updateBuilder( expression: Filter, builder: Builder): Unit = diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 35c77e03d15a..6cfefb4e03fa 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -348,6 +348,10 @@ private class OrcFilterConverter( .left .get + /** + * Builds a SearchArgument for the given Filter. This method should only be called on Filters + * that have previously been trimmed to remove unsupported sub-Filters! + */ private def updateBuilder( expression: Filter, builder: Builder): Unit = From c12ed1150318d3bff6d1a9d22bd87c43869f631f Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 29 May 2019 13:21:17 +0300 Subject: [PATCH 48/56] Code review comments --- .../datasources/orc/OrcFilters.scala | 88 ++++++++----------- .../datasources/orc/OrcFilters.scala | 88 ++++++++----------- 2 files changed, 78 insertions(+), 98 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 40a4ed1b04d0..07b06f10c1c0 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -67,7 +67,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters(schema, dataTypeMap, filters)) + conjunction <- buildTree(filters.flatMap(orcFilterConverter.trimUnconvertibleFilters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() @@ -83,9 +83,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { } -private class OrcFilterConverter( - val dataTypeMap: Map[String, DataType] -) { +private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { /** * Get PredicateLeafType which is corresponding to the given DataType. @@ -148,9 +146,10 @@ private class OrcFilterConverter( filteredExpression.map(_ => builder) } - sealed trait ActionType - case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType - case class BuildSearchArgument(builder: Builder) extends ActionType + sealed trait ActionType[ReturnType] + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) + extends ActionType[Option[Filter]] + case class BuildSearchArgument(builder: Builder) extends ActionType[Unit] // The performAction method can run both the filtering and building operations for a given // node - we signify which one we want with the `actionType` parameter. @@ -164,16 +163,9 @@ private class OrcFilterConverter( // in different methods, it's very easy to change one without appropriately updating the // other. For example, if we add a new supported node type to `filter`, it would be very // easy to forget to update `build` to support it too, thus leading to conversion errors. - // - // Doing things this way does have some annoying side effects: - // - We need to return an `Either`, with one action type always returning a Left and the other - // always returning a Right. - // - We always need to pass the canPartialPushDownConjuncts parameter even though the build - // action doesn't need it (because by the time we run the `build` operation, we know all - // remaining nodes are convertible). - def performAction( - actionType: ActionType, - expression: Filter): Either[Option[Filter], Unit] = { + private def performAction[ReturnType]( + actionType: ActionType[ReturnType], + expression: Filter): ReturnType = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -195,17 +187,17 @@ private class OrcFilterConverter( val lhs = performFilter(left, canPartialPushDownConjuncts) val rhs = performFilter(right, canPartialPushDownConjuncts) (lhs, rhs) match { - case (Some(l), Some(r)) => Left(Some(And(l, r))) - case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) - case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) - case _ => Left(None) + case (Some(l), Some(r)) => Some(And(l, r)) + case (Some(_), None) if canPartialPushDownConjuncts => lhs + case (None, Some(_)) if canPartialPushDownConjuncts => rhs + case _ => None } case BuildSearchArgument(builder) => builder.startAnd() updateBuilder(left, builder) updateBuilder(right, builder) builder.end() - Right(Unit) + () } case Or(left, right) => @@ -222,16 +214,16 @@ private class OrcFilterConverter( // The predicate can be converted as // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). - Left(for { + for { lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) - } yield Or(lhs, rhs)) + } yield Or(lhs, rhs) case BuildSearchArgument(builder) => builder.startOr() updateBuilder(left, builder) updateBuilder(right, builder) builder.end() - Right(Unit) + () } case Not(child) => @@ -239,12 +231,12 @@ private class OrcFilterConverter( case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) - Left(filteredSubtree.map(Not(_))) + filteredSubtree.map(Not(_)) case BuildSearchArgument(builder) => builder.startNot() updateBuilder(child, builder) builder.end() - Right(Unit) + () } // NOTE: For all case branches dealing with leaf predicates below, the additional @@ -253,88 +245,88 @@ private class OrcFilterConverter( case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() - Right(Unit) + () } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() - Right(Unit) + () } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) builder.startAnd().in(quotedName, getType(attribute), castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - Right(Unit) + () } case _ => actionType match { - case TrimUnconvertibleFilters(_) => Left(None) + case TrimUnconvertibleFilters(_) => None case BuildSearchArgument(builder) => throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") } @@ -345,8 +337,6 @@ private class OrcFilterConverter( expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression) - .left - .get /** * Builds a SearchArgument for the given Filter. This method should only be called on Filters @@ -355,6 +345,6 @@ private class OrcFilterConverter( private def updateBuilder( expression: Filter, builder: Builder): Unit = - performAction(BuildSearchArgument(builder), expression).right.get + performAction(BuildSearchArgument(builder), expression) } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 6cfefb4e03fa..ccaf243dbe29 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -67,7 +67,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters(schema, dataTypeMap, filters)) + conjunction <- buildTree(filters.flatMap(orcFilterConverter.trimUnconvertibleFilters)) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() @@ -83,9 +83,7 @@ private[sql] object OrcFilters extends OrcFiltersBase { } -private class OrcFilterConverter( - val dataTypeMap: Map[String, DataType] -) { +private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { /** * Get PredicateLeafType which is corresponding to the given DataType. @@ -148,9 +146,10 @@ private class OrcFilterConverter( filteredExpression.map(_ => builder) } - sealed trait ActionType - case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType - case class BuildSearchArgument(builder: Builder) extends ActionType + sealed trait ActionType[ReturnType] + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) + extends ActionType[Option[Filter]] + case class BuildSearchArgument(builder: Builder) extends ActionType[Unit] // The performAction method can run both the filtering and building operations for a given // node - we signify which one we want with the `actionType` parameter. @@ -164,16 +163,9 @@ private class OrcFilterConverter( // in different methods, it's very easy to change one without appropriately updating the // other. For example, if we add a new supported node type to `filter`, it would be very // easy to forget to update `build` to support it too, thus leading to conversion errors. - // - // Doing things this way does have some annoying side effects: - // - We need to return an `Either`, with one action type always returning a Left and the other - // always returning a Right. - // - We always need to pass the canPartialPushDownConjuncts parameter even though the build - // action doesn't need it (because by the time we run the `build` operation, we know all - // remaining nodes are convertible). - def performAction( - actionType: ActionType, - expression: Filter): Either[Option[Filter], Unit] = { + private def performAction[ReturnType]( + actionType: ActionType[ReturnType], + expression: Filter): ReturnType = { def getType(attribute: String): PredicateLeaf.Type = getPredicateLeafType(dataTypeMap(attribute)) @@ -195,17 +187,17 @@ private class OrcFilterConverter( val lhs = performFilter(left, canPartialPushDownConjuncts) val rhs = performFilter(right, canPartialPushDownConjuncts) (lhs, rhs) match { - case (Some(l), Some(r)) => Left(Some(And(l, r))) - case (Some(_), None) if canPartialPushDownConjuncts => Left(lhs) - case (None, Some(_)) if canPartialPushDownConjuncts => Left(rhs) - case _ => Left(None) + case (Some(l), Some(r)) => Some(And(l, r)) + case (Some(_), None) if canPartialPushDownConjuncts => lhs + case (None, Some(_)) if canPartialPushDownConjuncts => rhs + case _ => None } case BuildSearchArgument(builder) => builder.startAnd() updateBuilder(left, builder) updateBuilder(right, builder) builder.end() - Right(Unit) + () } case Or(left, right) => @@ -222,16 +214,16 @@ private class OrcFilterConverter( // The predicate can be converted as // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). - Left(for { + for { lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) - } yield Or(lhs, rhs)) + } yield Or(lhs, rhs) case BuildSearchArgument(builder) => builder.startOr() updateBuilder(left, builder) updateBuilder(right, builder) builder.end() - Right(Unit) + () } case Not(child) => @@ -239,12 +231,12 @@ private class OrcFilterConverter( case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => val filteredSubtree = performFilter(child, canPartialPushDownConjuncts = false) - Left(filteredSubtree.map(Not(_))) + filteredSubtree.map(Not(_)) case BuildSearchArgument(builder) => builder.startNot() updateBuilder(child, builder) builder.end() - Right(Unit) + () } // NOTE: For all case branches dealing with leaf predicates below, the additional @@ -253,88 +245,88 @@ private class OrcFilterConverter( case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().equals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end() - Right(Unit) + () } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() - Right(Unit) + () } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() - Right(Unit) + () } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Left(Some(expression)) + case TrimUnconvertibleFilters(_) => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) builder.startAnd().in(quotedName, getType(attribute), castedValues.map(_.asInstanceOf[AnyRef]): _*).end() - Right(Unit) + () } case _ => actionType match { - case TrimUnconvertibleFilters(_) => Left(None) + case TrimUnconvertibleFilters(_) => None case BuildSearchArgument(builder) => throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") } @@ -345,8 +337,6 @@ private class OrcFilterConverter( expression: Filter, canPartialPushDownConjuncts: Boolean): Option[Filter] = performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression) - .left - .get /** * Builds a SearchArgument for the given Filter. This method should only be called on Filters @@ -355,5 +345,5 @@ private class OrcFilterConverter( private def updateBuilder( expression: Filter, builder: Builder): Unit = - performAction(BuildSearchArgument(builder), expression).right.get + performAction(BuildSearchArgument(builder), expression) } From 1735c0cf77e914f1f229134b191b316f9c3465fe Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 30 May 2019 12:11:21 +0300 Subject: [PATCH 49/56] Code review comments --- .../datasources/orc/OrcFilters.scala | 33 ++++++++----------- .../datasources/orc/OrcFilters.scala | 33 ++++++++----------- 2 files changed, 28 insertions(+), 38 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 07b06f10c1c0..f2cd8ed11d9b 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -66,8 +66,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { - // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(filters.flatMap(orcFilterConverter.trimUnconvertibleFilters)) + // Combines all filters using `And` to produce a single conjunction + conjunction <- buildTree(filters) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() @@ -139,11 +139,10 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { private[sql] def buildSearchArgument( expression: Filter, builder: Builder): Option[Builder] = { - val filteredExpression: Option[Filter] = performFilter( - expression, - canPartialPushDownConjuncts = true) - filteredExpression.foreach(updateBuilder(_, builder)) - filteredExpression.map(_ => builder) + performFilter(expression, canPartialPushDownConjuncts = true).map { filter => + updateBuilder(filter, builder) + builder + } } sealed trait ActionType[ReturnType] @@ -172,7 +171,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { expression match { case And(left, right) => actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // At here, it is not safe to just keep one side and remove the other side // if we do not understand what the parent filters are. // @@ -184,8 +183,8 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported // predicate can be safely removed. - val lhs = performFilter(left, canPartialPushDownConjuncts) - val rhs = performFilter(right, canPartialPushDownConjuncts) + val lhs = performAction(t, left) + val rhs = performAction(t, right) (lhs, rhs) match { case (Some(l), Some(r)) => Some(And(l, r)) case (Some(_), None) if canPartialPushDownConjuncts => lhs @@ -202,7 +201,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Or(left, right) => actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // The Or predicate is convertible when both of its children can be pushed down. // That is to say, if one/both of the children can be partially pushed down, the Or // predicate can be partially pushed down as well. @@ -215,8 +214,8 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). for { - lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) - rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) + lhs: Filter <- performAction(t, left) + rhs: Filter <- performAction(t, right) } yield Or(lhs, rhs) case BuildSearchArgument(builder) => builder.startOr() @@ -229,9 +228,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Not(child) => actionType match { case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - val filteredSubtree = - performFilter(child, canPartialPushDownConjuncts = false) - filteredSubtree.map(Not(_)) + performFilter(child, canPartialPushDownConjuncts = false).map(Not) case BuildSearchArgument(builder) => builder.startNot() updateBuilder(child, builder) @@ -342,9 +339,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { * Builds a SearchArgument for the given Filter. This method should only be called on Filters * that have previously been trimmed to remove unsupported sub-Filters! */ - private def updateBuilder( - expression: Filter, - builder: Builder): Unit = + private def updateBuilder(expression: Filter, builder: Builder): Unit = performAction(BuildSearchArgument(builder), expression) } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index ccaf243dbe29..e54282ac4a41 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -66,8 +66,8 @@ private[sql] object OrcFilters extends OrcFiltersBase { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { - // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(filters.flatMap(orcFilterConverter.trimUnconvertibleFilters)) + // Combines all filters using `And` to produce a single conjunction + conjunction <- buildTree(filters) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() @@ -139,11 +139,10 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { private[sql] def buildSearchArgument( expression: Filter, builder: Builder): Option[Builder] = { - val filteredExpression: Option[Filter] = performFilter( - expression, - canPartialPushDownConjuncts = true) - filteredExpression.foreach(updateBuilder(_, builder)) - filteredExpression.map(_ => builder) + performFilter(expression, canPartialPushDownConjuncts = true).map { filter => + updateBuilder(filter, builder) + builder + } } sealed trait ActionType[ReturnType] @@ -172,7 +171,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { expression match { case And(left, right) => actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // At here, it is not safe to just keep one side and remove the other side // if we do not understand what the parent filters are. // @@ -184,8 +183,8 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { // Pushing one side of AND down is only safe to do at the top level or in the child // AND before hitting NOT or OR conditions, and in this case, the unsupported // predicate can be safely removed. - val lhs = performFilter(left, canPartialPushDownConjuncts) - val rhs = performFilter(right, canPartialPushDownConjuncts) + val lhs = performAction(t, left) + val rhs = performAction(t, right) (lhs, rhs) match { case (Some(l), Some(r)) => Some(And(l, r)) case (Some(_), None) if canPartialPushDownConjuncts => lhs @@ -202,7 +201,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Or(left, right) => actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => // The Or predicate is convertible when both of its children can be pushed down. // That is to say, if one/both of the children can be partially pushed down, the Or // predicate can be partially pushed down as well. @@ -215,8 +214,8 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) // As per the logical in And predicate, we can push down (a1 OR b1). for { - lhs: Filter <- performFilter(left, canPartialPushDownConjuncts) - rhs: Filter <- performFilter(right, canPartialPushDownConjuncts) + lhs: Filter <- performAction(t, left) + rhs: Filter <- performAction(t, right) } yield Or(lhs, rhs) case BuildSearchArgument(builder) => builder.startOr() @@ -229,9 +228,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Not(child) => actionType match { case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - val filteredSubtree = - performFilter(child, canPartialPushDownConjuncts = false) - filteredSubtree.map(Not(_)) + performFilter(child, canPartialPushDownConjuncts = false).map(Not) case BuildSearchArgument(builder) => builder.startNot() updateBuilder(child, builder) @@ -342,8 +339,6 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { * Builds a SearchArgument for the given Filter. This method should only be called on Filters * that have previously been trimmed to remove unsupported sub-Filters! */ - private def updateBuilder( - expression: Filter, - builder: Builder): Unit = + private def updateBuilder(expression: Filter, builder: Builder): Unit = performAction(BuildSearchArgument(builder), expression) } From 641b66228b5be82c6f6436e80801378f42a1dab5 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 30 May 2019 13:59:14 +0300 Subject: [PATCH 50/56] Style improvements to case-match statements --- .../datasources/orc/OrcFilters.scala | 28 +++++++++---------- .../datasources/orc/OrcFilters.scala | 28 +++++++++---------- 2 files changed, 28 insertions(+), 28 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index f2cd8ed11d9b..7340d0ddd30b 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -139,7 +139,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { private[sql] def buildSearchArgument( expression: Filter, builder: Builder): Option[Builder] = { - performFilter(expression, canPartialPushDownConjuncts = true).map { filter => + trimUnconvertibleFilters(expression).map { filter => updateBuilder(filter, builder) builder } @@ -201,7 +201,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Or(left, right) => actionType match { - case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + case t: TrimUnconvertibleFilters => // The Or predicate is convertible when both of its children can be pushed down. // That is to say, if one/both of the children can be partially pushed down, the Or // predicate can be partially pushed down as well. @@ -227,8 +227,8 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Not(child) => actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - performFilter(child, canPartialPushDownConjuncts = false).map(Not) + case t: TrimUnconvertibleFilters => + performAction(t.copy(canPartialPushDownConjuncts = false), child).map(Not) case BuildSearchArgument(builder) => builder.startNot() updateBuilder(child, builder) @@ -242,7 +242,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -251,7 +251,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -260,7 +260,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -269,7 +269,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -278,7 +278,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -287,7 +287,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -296,7 +296,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() @@ -304,7 +304,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() @@ -312,7 +312,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) @@ -323,7 +323,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case _ => actionType match { - case TrimUnconvertibleFilters(_) => None + case _: TrimUnconvertibleFilters => None case BuildSearchArgument(builder) => throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index e54282ac4a41..bf6f5f724ca3 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -139,7 +139,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { private[sql] def buildSearchArgument( expression: Filter, builder: Builder): Option[Builder] = { - performFilter(expression, canPartialPushDownConjuncts = true).map { filter => + trimUnconvertibleFilters(expression).map { filter => updateBuilder(filter, builder) builder } @@ -201,7 +201,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Or(left, right) => actionType match { - case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + case t: TrimUnconvertibleFilters => // The Or predicate is convertible when both of its children can be pushed down. // That is to say, if one/both of the children can be partially pushed down, the Or // predicate can be partially pushed down as well. @@ -227,8 +227,8 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case Not(child) => actionType match { - case TrimUnconvertibleFilters(canPartialPushDownConjuncts) => - performFilter(child, canPartialPushDownConjuncts = false).map(Not) + case t: TrimUnconvertibleFilters => + performAction(t.copy(canPartialPushDownConjuncts = false), child).map(Not) case BuildSearchArgument(builder) => builder.startNot() updateBuilder(child, builder) @@ -242,7 +242,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -251,7 +251,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -260,7 +260,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -269,7 +269,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -278,7 +278,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -287,7 +287,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) @@ -296,7 +296,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startAnd().isNull(quotedName, getType(attribute)).end() @@ -304,7 +304,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) builder.startNot().isNull(quotedName, getType(attribute)).end() @@ -312,7 +312,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => actionType match { - case TrimUnconvertibleFilters(_) => Some(expression) + case _: TrimUnconvertibleFilters => Some(expression) case BuildSearchArgument(builder) => val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) @@ -323,7 +323,7 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case _ => actionType match { - case TrimUnconvertibleFilters(_) => None + case _: TrimUnconvertibleFilters => None case BuildSearchArgument(builder) => throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") } From 524a1e15cdd5b38e33a1c59699659b569c001fcf Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Fri, 31 May 2019 15:07:30 +0300 Subject: [PATCH 51/56] Remove performFilter method --- .../datasources/orc/OrcFilters.scala | 43 ++++++++----------- .../datasources/orc/OrcFilters.scala | 43 ++++++++----------- 2 files changed, 38 insertions(+), 48 deletions(-) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 7340d0ddd30b..db898aa26577 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -118,10 +118,6 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { import org.apache.spark.sql.sources._ import OrcFilters._ - private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { - performFilter(expression, canPartialPushDownConjuncts = true) - } - /** * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then * only building the remaining convertible nodes. @@ -145,6 +141,17 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } } + private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts = true), expression) + } + + /** + * Builds a SearchArgument for the given Filter. This method should only be called on Filters + * that have previously been trimmed to remove unsupported sub-Filters! + */ + private def updateBuilder(expression: Filter, builder: Builder): Unit = + performAction(BuildSearchArgument(builder), expression) + sealed trait ActionType[ReturnType] case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType[Option[Filter]] @@ -191,10 +198,10 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case (None, Some(_)) if canPartialPushDownConjuncts => rhs case _ => None } - case BuildSearchArgument(builder) => + case b @ BuildSearchArgument(builder) => builder.startAnd() - updateBuilder(left, builder) - updateBuilder(right, builder) + performAction(b, left) + performAction(b, right) builder.end() () } @@ -217,10 +224,10 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { lhs: Filter <- performAction(t, left) rhs: Filter <- performAction(t, right) } yield Or(lhs, rhs) - case BuildSearchArgument(builder) => + case b @ BuildSearchArgument(builder) => builder.startOr() - updateBuilder(left, builder) - updateBuilder(right, builder) + performAction(b, left) + performAction(b, right) builder.end() () } @@ -229,9 +236,9 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { actionType match { case t: TrimUnconvertibleFilters => performAction(t.copy(canPartialPushDownConjuncts = false), child).map(Not) - case BuildSearchArgument(builder) => + case b @ BuildSearchArgument(builder) => builder.startNot() - updateBuilder(child, builder) + performAction(b, child) builder.end() () } @@ -329,17 +336,5 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } } } - - private def performFilter( - expression: Filter, - canPartialPushDownConjuncts: Boolean): Option[Filter] = - performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression) - - /** - * Builds a SearchArgument for the given Filter. This method should only be called on Filters - * that have previously been trimmed to remove unsupported sub-Filters! - */ - private def updateBuilder(expression: Filter, builder: Builder): Unit = - performAction(BuildSearchArgument(builder), expression) } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index bf6f5f724ca3..343fea9fd63c 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -118,10 +118,6 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { import org.apache.spark.sql.sources._ import OrcFilters._ - private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { - performFilter(expression, canPartialPushDownConjuncts = true) - } - /** * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then * only building the remaining convertible nodes. @@ -145,6 +141,17 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } } + private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts = true), expression) + } + + /** + * Builds a SearchArgument for the given Filter. This method should only be called on Filters + * that have previously been trimmed to remove unsupported sub-Filters! + */ + private def updateBuilder(expression: Filter, builder: Builder): Unit = + performAction(BuildSearchArgument(builder), expression) + sealed trait ActionType[ReturnType] case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) extends ActionType[Option[Filter]] @@ -191,10 +198,10 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { case (None, Some(_)) if canPartialPushDownConjuncts => rhs case _ => None } - case BuildSearchArgument(builder) => + case b @ BuildSearchArgument(builder) => builder.startAnd() - updateBuilder(left, builder) - updateBuilder(right, builder) + performAction(b, left) + performAction(b, right) builder.end() () } @@ -217,10 +224,10 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { lhs: Filter <- performAction(t, left) rhs: Filter <- performAction(t, right) } yield Or(lhs, rhs) - case BuildSearchArgument(builder) => + case b @ BuildSearchArgument(builder) => builder.startOr() - updateBuilder(left, builder) - updateBuilder(right, builder) + performAction(b, left) + performAction(b, right) builder.end() () } @@ -229,9 +236,9 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { actionType match { case t: TrimUnconvertibleFilters => performAction(t.copy(canPartialPushDownConjuncts = false), child).map(Not) - case BuildSearchArgument(builder) => + case b @ BuildSearchArgument(builder) => builder.startNot() - updateBuilder(child, builder) + performAction(b, child) builder.end() () } @@ -329,16 +336,4 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } } } - - private def performFilter( - expression: Filter, - canPartialPushDownConjuncts: Boolean): Option[Filter] = - performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts), expression) - - /** - * Builds a SearchArgument for the given Filter. This method should only be called on Filters - * that have previously been trimmed to remove unsupported sub-Filters! - */ - private def updateBuilder(expression: Filter, builder: Builder): Unit = - performAction(BuildSearchArgument(builder), expression) } From 1d1de5be2370cf8913604a37be9927417bf5bd81 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 5 Jun 2019 23:03:30 +0300 Subject: [PATCH 52/56] Add a doc for trimUnconvertibleFilters --- .../spark/sql/execution/datasources/orc/OrcFilters.scala | 3 +++ .../spark/sql/execution/datasources/orc/OrcFilters.scala | 3 +++ 2 files changed, 6 insertions(+) diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index db898aa26577..e87f7d8caf4f 100644 --- a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -141,6 +141,9 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } } + /** + * Removes all sub-Filters from a given Filter that are not convertible to an ORC SearchArgument. + */ private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts = true), expression) } diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index 343fea9fd63c..ebcd3567d34c 100644 --- a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -141,6 +141,9 @@ private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { } } + /** + * Removes all sub-Filters from a given Filter that are not convertible to an ORC SearchArgument. + */ private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts = true), expression) } From 3d5620d1d892e98d245fb86cdb9f4e099b2c1132 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 6 Jun 2019 11:22:39 +0300 Subject: [PATCH 53/56] Apply changes to hive/ subtree as well --- .../spark/sql/hive/orc/OrcFilters.scala | 348 +++++++++++------- 1 file changed, 216 insertions(+), 132 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala index 3bfe157f5fe1..cee8fdbe43a1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFilters.scala @@ -62,176 +62,260 @@ import org.apache.spark.sql.types._ */ private[orc] object OrcFilters extends Logging { - private def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { - val method = klass.getMethod(name, args: _*) - method.setAccessible(true) - method - } - def createFilter(schema: StructType, filters: Array[Filter]): Option[SearchArgument] = { if (HiveUtils.isHive23) { DatasourceOrcFilters.createFilter(schema, filters).asInstanceOf[Option[SearchArgument]] } else { val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap - - // First, tries to convert each filter individually to see whether it's convertible, and then - // collect all convertible ones to build the final `SearchArgument`. - val convertibleFilters = for { - filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, newBuilder) - } yield filter - + val orcFilterConverter = new OrcFilterConverter(dataTypeMap) for { - // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters) + // Combines all filters using `And` to produce a single conjunction + conjunction <- buildTree(filters) // Then tries to build a single ORC `SearchArgument` for the conjunction predicate - builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) + builder <- orcFilterConverter.buildSearchArgument(conjunction, newBuilder) } yield builder.build() } } +} - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder): Option[Builder] = { - createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) +private class OrcFilterConverter(val dataTypeMap: Map[String, DataType]) { + + def isSearchableType(dataType: DataType): Boolean = dataType match { + // Only the values in the Spark types below can be recognized by + // the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. + case ByteType | ShortType | FloatType | DoubleType => true + case IntegerType | LongType | StringType | BooleanType => true + case TimestampType | _: DecimalType => true + case _ => false } + private def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { + val method = klass.getMethod(name, args: _*) + method.setAccessible(true) + method + } + + import org.apache.spark.sql.sources._ + /** - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the builder so far. + * Builds a SearchArgument for a Filter by first trimming the non-convertible nodes, and then + * only building the remaining convertible nodes. + * + * Doing the conversion in this way avoids the computational complexity problems introduced by + * checking whether a node is convertible while building it. The approach implemented here has + * complexity that's linear in the size of the Filter tree - O(number of Filter nodes) - we run + * a single pass over the tree to trim it, and then another pass on the trimmed tree to convert + * the remaining nodes. + * + * The alternative approach of checking-while-building can (and did) result + * in exponential complexity in the height of the tree, causing perf problems with Filters with + * as few as ~35 nodes if they were skewed. */ - private def createBuilder( - dataTypeMap: Map[String, DataType], + private[sql] def buildSearchArgument( expression: Filter, - builder: Builder, - canPartialPushDownConjuncts: Boolean): Option[Builder] = { - def isSearchableType(dataType: DataType): Boolean = dataType match { - // Only the values in the Spark types below can be recognized by - // the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. - case ByteType | ShortType | FloatType | DoubleType => true - case IntegerType | LongType | StringType | BooleanType => true - case TimestampType | _: DecimalType => true - case _ => false + builder: Builder): Option[Builder] = { + trimUnconvertibleFilters(expression).map { filter => + updateBuilder(filter, builder) + builder } + } - expression match { - case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftBuilderOption = - createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - val rightBuilderOption = - createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - (leftBuilderOption, rightBuilderOption) match { - case (Some(_), Some(_)) => - for { - lhs <- createBuilder(dataTypeMap, left, - builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() + /** + * Removes all sub-Filters from a given Filter that are not convertible to an ORC SearchArgument. + */ + private[sql] def trimUnconvertibleFilters(expression: Filter): Option[Filter] = { + performAction(TrimUnconvertibleFilters(canPartialPushDownConjuncts = true), expression) + } - case (Some(_), None) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) + /** + * Builds a SearchArgument for the given Filter. This method should only be called on Filters + * that have previously been trimmed to remove unsupported sub-Filters! + */ + private def updateBuilder(expression: Filter, builder: Builder): Unit = + performAction(BuildSearchArgument(builder), expression) - case (None, Some(_)) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) + sealed trait ActionType[ReturnType] + case class TrimUnconvertibleFilters(canPartialPushDownConjuncts: Boolean) + extends ActionType[Option[Filter]] + case class BuildSearchArgument(builder: Builder) extends ActionType[Unit] - case _ => None + // The performAction method can run both the filtering and building operations for a given + // node - we signify which one we want with the `actionType` parameter. + // + // There are a couple of benefits to coupling the two operations like this: + // 1. All the logic for a given predicate is grouped logically in the same place. You don't + // have to scroll across the whole file to see what the filter action for an And is while + // you're looking at the build action. + // 2. It's much easier to keep the implementations of the two operations up-to-date with + // each other. If the `filter` and `build` operations are implemented as separate case-matches + // in different methods, it's very easy to change one without appropriately updating the + // other. For example, if we add a new supported node type to `filter`, it would be very + // easy to forget to update `build` to support it too, thus leading to conversion errors. + private def performAction[ReturnType]( + actionType: ActionType[ReturnType], + expression: Filter): ReturnType = { + + expression match { + case And(left, right) => + actionType match { + case t @ TrimUnconvertibleFilters(canPartialPushDownConjuncts) => + // At here, it is not safe to just keep one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported + // predicate can be safely removed. + val lhs = performAction(t, left) + val rhs = performAction(t, right) + (lhs, rhs) match { + case (Some(l), Some(r)) => Some(And(l, r)) + case (Some(_), None) if canPartialPushDownConjuncts => lhs + case (None, Some(_)) if canPartialPushDownConjuncts => rhs + case _ => None + } + case b @ BuildSearchArgument(builder) => + builder.startAnd() + performAction(b, left) + performAction(b, right) + builder.end() + () } case Or(left, right) => - // The Or predicate is convertible when both of its children can be pushed down. - // That is to say, if one/both of the children can be partially pushed down, the Or - // predicate can be partially pushed down as well. - // - // Here is an example used to explain the reason. - // Let's say we have - // (a1 AND a2) OR (b1 AND b2), - // a1 and b1 is convertible, while a2 and b2 is not. - // The predicate can be converted as - // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) - // As per the logical in And predicate, we can push down (a1 OR b1). - for { - _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - lhs <- createBuilder(dataTypeMap, left, - builder.startOr(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() + actionType match { + case t: TrimUnconvertibleFilters => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + for { + lhs: Filter <- performAction(t, left) + rhs: Filter <- performAction(t, right) + } yield Or(lhs, rhs) + case b @ BuildSearchArgument(builder) => + builder.startOr() + performAction(b, left) + performAction(b, right) + builder.end() + () + } case Not(child) => - for { - _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) - negate <- createBuilder(dataTypeMap, - child, builder.startNot(), canPartialPushDownConjuncts = false) - } yield negate.end() + actionType match { + case t: TrimUnconvertibleFilters => + performAction(t.copy(canPartialPushDownConjuncts = false), child).map(Not) + case b @ BuildSearchArgument(builder) => + builder.startNot() + performAction(b, child) + builder.end() + () + } - // 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`). + // 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`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "equals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "equals", classOf[String], classOf[Object]) + method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end() + () + } case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "nullSafeEquals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "nullSafeEquals", classOf[String], classOf[Object]) + method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end() + () + } case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) + method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end() + () + } case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) + method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end() + () + } case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startNot() - val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startNot() + val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) + method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end() + () + } case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startNot() - val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) - Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startNot() + val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) + method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end() + () + } case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "isNull", classOf[String]) - Some(method.invoke(bd, attribute).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "isNull", classOf[String]) + method.invoke(bd, attribute).asInstanceOf[Builder].end() + () + } case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startNot() - val method = findMethod(bd.getClass, "isNull", classOf[String]) - Some(method.invoke(bd, attribute).asInstanceOf[Builder].end()) - + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startNot() + val method = findMethod(bd.getClass, "isNull", classOf[String]) + method.invoke(bd, attribute).asInstanceOf[Builder].end() + () + } case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - val bd = builder.startAnd() - val method = findMethod(bd.getClass, "in", classOf[String], classOf[Array[Object]]) - Some(method.invoke(bd, attribute, values.map(_.asInstanceOf[AnyRef])) - .asInstanceOf[Builder].end()) + actionType match { + case _: TrimUnconvertibleFilters => Some(expression) + case BuildSearchArgument(builder) => + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "in", classOf[String], classOf[Array[Object]]) + method.invoke(bd, attribute, values.map(_.asInstanceOf[AnyRef])) + .asInstanceOf[Builder].end() + () + } - case _ => None + case _ => + actionType match { + case _: TrimUnconvertibleFilters => None + case BuildSearchArgument(builder) => + throw new IllegalArgumentException(s"Can't build unsupported filter ${expression}") + } } } } From 67b7a86b315e68ff4c43a991e4ec33b207a3e2ac Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 6 Jun 2019 12:50:08 +0300 Subject: [PATCH 54/56] Remove filter conversion benchmark --- .../FilterPushdownBenchmark-results.txt | 23 --------------- .../benchmark/FilterPushdownBenchmark.scala | 29 ------------------- 2 files changed, 52 deletions(-) diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt index fba9b4e7a495..7dbabb403643 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt @@ -668,29 +668,6 @@ Native ORC Vectorized 367 374 Native ORC Vectorized (Pushdown) 378 382 4 0.0 377890425.0 1.0X -================================================================================================ -Predicate conversion benchmark with unbalanced Expression -================================================================================================ - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 25 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Native ORC Vectorized (Pushdown) 0 0 0 0.0 341119.0 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 5000 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Native ORC Vectorized (Pushdown) 5 5 1 0.0 4594703.0 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 15000 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Native ORC Vectorized (Pushdown) 16 18 2 0.0 16423418.0 1.0X - - ================================================================================================ Pushdown benchmark with unbalanced Column ================================================================================================ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 236253704823..ec044f082dd2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -411,35 +411,6 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } } - runBenchmark(s"Predicate conversion benchmark with unbalanced Column") { - // This benchmark tests a very isolated part of the predicate pushdown process - specifically, - // the individual action of converting a Spark `Expression` to an ORC `SearchArgument`. - // This results in more granular numbers that can help highlight small performance - // differences in this part of the code that would be hidden by slower components that - // get run when a full Spark job is executed. - // The benchmark below runs a more complete, end-to-end test which covers the whole pipeline - // and can uncover high-level performance problems, but is bad at discriminating details. - val numRows = 1 - val width = 2000 - - val columns = (1 to width).map(i => s"id c$i") - val df = spark.range(1).selectExpr(columns: _*) - Seq(25, 5000, 15000).foreach { numFilter => - val whereColumn = (1 to numFilter) - .map(i => col("c1") === lit(i)) - .foldLeft(lit(false))(_ || _) - val benchmark = new Benchmark( - s"Convert a filter with $numFilter columns to ORC filter", - numRows, minNumIters = 5, output = output) - val name = s"Native ORC Vectorized (Pushdown)" - benchmark.addCase(name) { _ => - OrcFilters.createFilter(df.schema, - DataSourceStrategy.translateFilter(whereColumn.expr).toSeq) - } - benchmark.run() - } - } - runBenchmark(s"Pushdown benchmark with unbalanced Column") { val numRows = 1 val width = 200 From 27ed4f9120ca00bbb124be6ecc94948614106950 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Thu, 6 Jun 2019 15:12:37 +0300 Subject: [PATCH 55/56] Add a comment explaining what each large-filter benchmark does --- .../sql/execution/benchmark/FilterPushdownBenchmark.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index ec044f082dd2..eba83dc570c6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -394,6 +394,13 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } runBenchmark(s"Pushdown benchmark with many filters") { + // This benchmark and the next one are similar in that they both test predicate pushdown + // where the filter itself is very large. There have been cases where the filter conversion + // would take minutes to hours for large filters due to it being implemented with exponential + // complexity in the height of the filter tree. + // The difference between these two benchmarks is that this one benchmarks pushdown with a + // large string filter (`a AND b AND c ...`), whereas the next one benchmarks pushdown with + // a large Column-based filter (`col(a) || (col(b) || (col(c)...))`). val numRows = 1 val width = 500 From 24dcc24cad757fd0cb052ae1f27a2a2937afaff7 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Tue, 18 Jun 2019 12:06:06 +0300 Subject: [PATCH 56/56] Revert benchmark changes --- .../FilterPushdownBenchmark-results.txt | 1270 ++++++++--------- .../benchmark/FilterPushdownBenchmark.scala | 67 +- 2 files changed, 625 insertions(+), 712 deletions(-) diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt index 7dbabb403643..e680ddff53dd 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt @@ -2,695 +2,669 @@ Pushdown for many distinct value case ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 0 string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6509 6563 64 2.4 413.8 1.0X -Parquet Vectorized (Pushdown) 451 455 5 34.9 28.7 14.4X -Native ORC Vectorized 4697 4880 311 3.3 298.6 1.4X -Native ORC Vectorized (Pushdown) 572 585 12 27.5 36.3 11.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 0 string row ('7864320' < value < '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6597 6624 21 2.4 419.4 1.0X -Parquet Vectorized (Pushdown) 453 456 2 34.7 28.8 14.6X -Native ORC Vectorized 4853 4887 29 3.2 308.5 1.4X -Native ORC Vectorized (Pushdown) 572 582 13 27.5 36.3 11.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 string row (value = '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6597 6648 30 2.4 419.4 1.0X -Parquet Vectorized (Pushdown) 445 448 3 35.4 28.3 14.8X -Native ORC Vectorized 4915 4954 34 3.2 312.5 1.3X -Native ORC Vectorized (Pushdown) 560 574 14 28.1 35.6 11.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 string row (value <=> '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6598 6664 80 2.4 419.5 1.0X -Parquet Vectorized (Pushdown) 439 442 3 35.8 27.9 15.0X -Native ORC Vectorized 4894 4926 30 3.2 311.1 1.3X -Native ORC Vectorized (Pushdown) 561 572 13 28.0 35.7 11.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 string row ('7864320' <= value <= '7864320'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6607 6634 27 2.4 420.1 1.0X -Parquet Vectorized (Pushdown) 440 444 3 35.8 28.0 15.0X -Native ORC Vectorized 4910 4961 48 3.2 312.2 1.3X -Native ORC Vectorized (Pushdown) 564 575 13 27.9 35.9 11.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select all string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 12348 12442 63 1.3 785.1 1.0X -Parquet Vectorized (Pushdown) 12110 12211 96 1.3 769.9 1.0X -Native ORC Vectorized 10689 10772 59 1.5 679.6 1.2X -Native ORC Vectorized (Pushdown) 10926 10971 40 1.4 694.7 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 0 int row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5789 5870 64 2.7 368.1 1.0X -Parquet Vectorized (Pushdown) 356 361 3 44.2 22.6 16.3X -Native ORC Vectorized 4326 4515 303 3.6 275.1 1.3X -Native ORC Vectorized (Pushdown) 547 565 15 28.8 34.8 10.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 0 int row (7864320 < value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5791 5837 60 2.7 368.2 1.0X -Parquet Vectorized (Pushdown) 364 373 6 43.2 23.2 15.9X -Native ORC Vectorized 4359 4398 28 3.6 277.1 1.3X -Native ORC Vectorized (Pushdown) 555 569 16 28.3 35.3 10.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 int row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5876 5916 25 2.7 373.6 1.0X -Parquet Vectorized (Pushdown) 362 367 4 43.4 23.0 16.2X -Native ORC Vectorized 4393 4453 44 3.6 279.3 1.3X -Native ORC Vectorized (Pushdown) 552 567 16 28.5 35.1 10.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 int row (value <=> 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5879 5898 16 2.7 373.8 1.0X -Parquet Vectorized (Pushdown) 359 369 9 43.8 22.8 16.4X -Native ORC Vectorized 4405 4441 30 3.6 280.0 1.3X -Native ORC Vectorized (Pushdown) 548 564 19 28.7 34.8 10.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 int row (7864320 <= value <= 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5846 5930 75 2.7 371.7 1.0X -Parquet Vectorized (Pushdown) 363 372 6 43.4 23.1 16.1X -Native ORC Vectorized 4425 4456 23 3.6 281.3 1.3X -Native ORC Vectorized (Pushdown) 551 572 24 28.6 35.0 10.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 int row (7864319 < value < 7864321): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5835 5888 43 2.7 370.9 1.0X -Parquet Vectorized (Pushdown) 363 368 3 43.3 23.1 16.1X -Native ORC Vectorized 4426 4445 24 3.6 281.4 1.3X -Native ORC Vectorized (Pushdown) 547 563 16 28.7 34.8 10.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% int rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6470 6540 48 2.4 411.4 1.0X -Parquet Vectorized (Pushdown) 1548 1570 16 10.2 98.4 4.2X -Native ORC Vectorized 5078 5106 22 3.1 322.9 1.3X -Native ORC Vectorized (Pushdown) 1625 1641 11 9.7 103.3 4.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% int rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8568 8628 45 1.8 544.7 1.0X -Parquet Vectorized (Pushdown) 5826 5891 54 2.7 370.4 1.5X -Native ORC Vectorized 7233 7254 18 2.2 459.8 1.2X -Native ORC Vectorized (Pushdown) 5447 5481 31 2.9 346.3 1.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% int rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 10653 10701 48 1.5 677.3 1.0X -Parquet Vectorized (Pushdown) 10210 10244 40 1.5 649.1 1.0X -Native ORC Vectorized 9398 9441 32 1.7 597.5 1.1X -Native ORC Vectorized (Pushdown) 9271 9331 56 1.7 589.4 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select all int rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 11098 11146 39 1.4 705.6 1.0X -Parquet Vectorized (Pushdown) 11187 11254 45 1.4 711.2 1.0X -Native ORC Vectorized 9847 9895 43 1.6 626.0 1.1X -Native ORC Vectorized (Pushdown) 10227 12071 623 1.5 650.2 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select all int rows (value > -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 11873 14239 NaN 1.3 754.8 1.0X -Parquet Vectorized (Pushdown) 11854 11911 36 1.3 753.7 1.0X -Native ORC Vectorized 10197 10482 397 1.5 648.3 1.2X -Native ORC Vectorized (Pushdown) 10450 10471 16 1.5 664.4 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select all int rows (value != -1): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 11715 11776 53 1.3 744.8 1.0X -Parquet Vectorized (Pushdown) 12178 15502 NaN 1.3 774.2 1.0X -Native ORC Vectorized 10196 10256 62 1.5 648.2 1.1X -Native ORC Vectorized (Pushdown) 10448 10479 21 1.5 664.3 1.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11405 / 11485 1.4 725.1 1.0X +Parquet Vectorized (Pushdown) 675 / 690 23.3 42.9 16.9X +Native ORC Vectorized 7127 / 7170 2.2 453.1 1.6X +Native ORC Vectorized (Pushdown) 519 / 541 30.3 33.0 22.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 string row ('7864320' < value < '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11457 / 11473 1.4 728.4 1.0X +Parquet Vectorized (Pushdown) 656 / 686 24.0 41.7 17.5X +Native ORC Vectorized 7328 / 7342 2.1 465.9 1.6X +Native ORC Vectorized (Pushdown) 539 / 565 29.2 34.2 21.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row (value = '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11878 / 11888 1.3 755.2 1.0X +Parquet Vectorized (Pushdown) 630 / 654 25.0 40.1 18.9X +Native ORC Vectorized 7342 / 7362 2.1 466.8 1.6X +Native ORC Vectorized (Pushdown) 519 / 537 30.3 33.0 22.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row (value <=> '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11423 / 11440 1.4 726.2 1.0X +Parquet Vectorized (Pushdown) 625 / 643 25.2 39.7 18.3X +Native ORC Vectorized 7315 / 7335 2.2 465.1 1.6X +Native ORC Vectorized (Pushdown) 507 / 520 31.0 32.2 22.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 string row ('7864320' <= value <= '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11440 / 11478 1.4 727.3 1.0X +Parquet Vectorized (Pushdown) 634 / 652 24.8 40.3 18.0X +Native ORC Vectorized 7311 / 7324 2.2 464.8 1.6X +Native ORC Vectorized (Pushdown) 517 / 548 30.4 32.8 22.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 20750 / 20872 0.8 1319.3 1.0X +Parquet Vectorized (Pushdown) 21002 / 21032 0.7 1335.3 1.0X +Native ORC Vectorized 16714 / 16742 0.9 1062.6 1.2X +Native ORC Vectorized (Pushdown) 16926 / 16965 0.9 1076.1 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 int row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10510 / 10532 1.5 668.2 1.0X +Parquet Vectorized (Pushdown) 642 / 665 24.5 40.8 16.4X +Native ORC Vectorized 6609 / 6618 2.4 420.2 1.6X +Native ORC Vectorized (Pushdown) 502 / 512 31.4 31.9 21.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 int row (7864320 < value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10505 / 10514 1.5 667.9 1.0X +Parquet Vectorized (Pushdown) 659 / 673 23.9 41.9 15.9X +Native ORC Vectorized 6634 / 6641 2.4 421.8 1.6X +Native ORC Vectorized (Pushdown) 513 / 526 30.7 32.6 20.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10555 / 10570 1.5 671.1 1.0X +Parquet Vectorized (Pushdown) 651 / 668 24.2 41.4 16.2X +Native ORC Vectorized 6721 / 6728 2.3 427.3 1.6X +Native ORC Vectorized (Pushdown) 508 / 519 31.0 32.3 20.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (value <=> 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10556 / 10566 1.5 671.1 1.0X +Parquet Vectorized (Pushdown) 647 / 654 24.3 41.1 16.3X +Native ORC Vectorized 6716 / 6728 2.3 427.0 1.6X +Native ORC Vectorized (Pushdown) 510 / 521 30.9 32.4 20.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (7864320 <= value <= 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10556 / 10565 1.5 671.1 1.0X +Parquet Vectorized (Pushdown) 649 / 654 24.2 41.3 16.3X +Native ORC Vectorized 6700 / 6712 2.3 426.0 1.6X +Native ORC Vectorized (Pushdown) 509 / 520 30.9 32.3 20.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 int row (7864319 < value < 7864321): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10547 / 10566 1.5 670.5 1.0X +Parquet Vectorized (Pushdown) 649 / 653 24.2 41.3 16.3X +Native ORC Vectorized 6703 / 6713 2.3 426.2 1.6X +Native ORC Vectorized (Pushdown) 510 / 520 30.8 32.5 20.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% int rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11478 / 11525 1.4 729.7 1.0X +Parquet Vectorized (Pushdown) 2576 / 2587 6.1 163.8 4.5X +Native ORC Vectorized 7633 / 7657 2.1 485.3 1.5X +Native ORC Vectorized (Pushdown) 2076 / 2096 7.6 132.0 5.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% int rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 14785 / 14802 1.1 940.0 1.0X +Parquet Vectorized (Pushdown) 9971 / 9977 1.6 633.9 1.5X +Native ORC Vectorized 11082 / 11107 1.4 704.6 1.3X +Native ORC Vectorized (Pushdown) 8061 / 8073 2.0 512.5 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% int rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 18174 / 18214 0.9 1155.5 1.0X +Parquet Vectorized (Pushdown) 17387 / 17403 0.9 1105.5 1.0X +Native ORC Vectorized 14465 / 14492 1.1 919.7 1.3X +Native ORC Vectorized (Pushdown) 14024 / 14041 1.1 891.6 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 19004 / 19014 0.8 1208.2 1.0X +Parquet Vectorized (Pushdown) 19219 / 19232 0.8 1221.9 1.0X +Native ORC Vectorized 15266 / 15290 1.0 970.6 1.2X +Native ORC Vectorized (Pushdown) 15469 / 15482 1.0 983.5 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value > -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 19036 / 19052 0.8 1210.3 1.0X +Parquet Vectorized (Pushdown) 19287 / 19306 0.8 1226.2 1.0X +Native ORC Vectorized 15311 / 15371 1.0 973.5 1.2X +Native ORC Vectorized (Pushdown) 15517 / 15590 1.0 986.5 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all int rows (value != -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 19072 / 19102 0.8 1212.6 1.0X +Parquet Vectorized (Pushdown) 19288 / 19318 0.8 1226.3 1.0X +Native ORC Vectorized 15277 / 15293 1.0 971.3 1.2X +Native ORC Vectorized (Pushdown) 15479 / 15499 1.0 984.1 1.2X ================================================================================================ Pushdown for few distinct value case (use dictionary encoding) ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 0 distinct string row (value IS NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5997 6029 23 2.6 381.3 1.0X -Parquet Vectorized (Pushdown) 328 336 7 47.9 20.9 18.3X -Native ORC Vectorized 5886 6011 109 2.7 374.2 1.0X -Native ORC Vectorized (Pushdown) 1086 1111 22 14.5 69.1 5.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 0 distinct string row ('100' < value < '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6097 6139 45 2.6 387.6 1.0X -Parquet Vectorized (Pushdown) 331 342 6 47.5 21.1 18.4X -Native ORC Vectorized 6018 6070 33 2.6 382.6 1.0X -Native ORC Vectorized (Pushdown) 1084 1099 14 14.5 68.9 5.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 distinct string row (value = '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6175 6202 26 2.5 392.6 1.0X -Parquet Vectorized (Pushdown) 474 488 10 33.2 30.1 13.0X -Native ORC Vectorized 6236 6270 41 2.5 396.5 1.0X -Native ORC Vectorized (Pushdown) 1203 1226 18 13.1 76.5 5.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 distinct string row (value <=> '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6192 7882 704 2.5 393.7 1.0X -Parquet Vectorized (Pushdown) 511 769 265 30.8 32.5 12.1X -Native ORC Vectorized 6592 7214 441 2.4 419.1 0.9X -Native ORC Vectorized (Pushdown) 1306 1446 124 12.0 83.0 4.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 distinct string row ('100' <= value <= '100'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6708 7325 686 2.3 426.5 1.0X -Parquet Vectorized (Pushdown) 482 630 164 32.7 30.6 13.9X -Native ORC Vectorized 6645 6828 109 2.4 422.5 1.0X -Native ORC Vectorized (Pushdown) 1268 1432 208 12.4 80.6 5.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select all distinct string rows (value IS NOT NULL): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 14204 15124 863 1.1 903.1 1.0X -Parquet Vectorized (Pushdown) 13976 15133 1013 1.1 888.6 1.0X -Native ORC Vectorized 14969 15811 1277 1.1 951.7 0.9X -Native ORC Vectorized (Pushdown) 15815 17290 1437 1.0 1005.5 0.9X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 distinct string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10250 / 10274 1.5 651.7 1.0X +Parquet Vectorized (Pushdown) 571 / 576 27.5 36.3 17.9X +Native ORC Vectorized 8651 / 8660 1.8 550.0 1.2X +Native ORC Vectorized (Pushdown) 909 / 933 17.3 57.8 11.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 0 distinct string row ('100' < value < '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10420 / 10426 1.5 662.5 1.0X +Parquet Vectorized (Pushdown) 574 / 579 27.4 36.5 18.2X +Native ORC Vectorized 8973 / 8982 1.8 570.5 1.2X +Native ORC Vectorized (Pushdown) 916 / 955 17.2 58.2 11.4X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 distinct string row (value = '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10428 / 10441 1.5 663.0 1.0X +Parquet Vectorized (Pushdown) 789 / 809 19.9 50.2 13.2X +Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X +Native ORC Vectorized (Pushdown) 1130 / 1145 13.9 71.8 9.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 distinct string row (value <=> '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10402 / 10416 1.5 661.3 1.0X +Parquet Vectorized (Pushdown) 791 / 806 19.9 50.3 13.2X +Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X +Native ORC Vectorized (Pushdown) 1112 / 1145 14.1 70.7 9.4X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 distinct string row ('100' <= value <= '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10548 / 10563 1.5 670.6 1.0X +Parquet Vectorized (Pushdown) 790 / 796 19.9 50.2 13.4X +Native ORC Vectorized 9144 / 9153 1.7 581.3 1.2X +Native ORC Vectorized (Pushdown) 1117 / 1148 14.1 71.0 9.4X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select all distinct string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 20445 / 20469 0.8 1299.8 1.0X +Parquet Vectorized (Pushdown) 20686 / 20699 0.8 1315.2 1.0X +Native ORC Vectorized 18851 / 18953 0.8 1198.5 1.1X +Native ORC Vectorized (Pushdown) 19255 / 19268 0.8 1224.2 1.1X ================================================================================================ Pushdown benchmark for StringStartsWith ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -StringStartsWith filter: (value like '10%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 10327 10682 540 1.5 656.5 1.0X -Parquet Vectorized (Pushdown) 2991 3328 491 5.3 190.1 3.5X -Native ORC Vectorized 7941 8369 323 2.0 504.9 1.3X -Native ORC Vectorized (Pushdown) 8354 8651 450 1.9 531.1 1.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -StringStartsWith filter: (value like '1000%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8599 8687 56 1.8 546.7 1.0X -Parquet Vectorized (Pushdown) 481 492 11 32.7 30.6 17.9X -Native ORC Vectorized 6168 6224 43 2.5 392.2 1.4X -Native ORC Vectorized (Pushdown) 6386 6469 73 2.5 406.0 1.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -StringStartsWith filter: (value like '786432%'): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8628 8670 58 1.8 548.6 1.0X -Parquet Vectorized (Pushdown) 463 477 12 34.0 29.4 18.6X -Native ORC Vectorized 6151 6195 38 2.6 391.1 1.4X -Native ORC Vectorized (Pushdown) 6401 6432 30 2.5 407.0 1.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +StringStartsWith filter: (value like '10%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 14265 / 15213 1.1 907.0 1.0X +Parquet Vectorized (Pushdown) 4228 / 4870 3.7 268.8 3.4X +Native ORC Vectorized 10116 / 10977 1.6 643.2 1.4X +Native ORC Vectorized (Pushdown) 10653 / 11376 1.5 677.3 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +StringStartsWith filter: (value like '1000%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11499 / 11539 1.4 731.1 1.0X +Parquet Vectorized (Pushdown) 669 / 672 23.5 42.5 17.2X +Native ORC Vectorized 7343 / 7363 2.1 466.8 1.6X +Native ORC Vectorized (Pushdown) 7559 / 7568 2.1 480.6 1.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +StringStartsWith filter: (value like '786432%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11463 / 11468 1.4 728.8 1.0X +Parquet Vectorized (Pushdown) 647 / 651 24.3 41.1 17.7X +Native ORC Vectorized 7322 / 7338 2.1 465.5 1.6X +Native ORC Vectorized (Pushdown) 7533 / 7544 2.1 478.9 1.5X ================================================================================================ Pushdown benchmark for decimal ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 decimal(9, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3178 3238 56 4.9 202.0 1.0X -Parquet Vectorized (Pushdown) 102 105 3 154.8 6.5 31.3X -Native ORC Vectorized 3224 3270 33 4.9 205.0 1.0X -Native ORC Vectorized (Pushdown) 603 614 9 26.1 38.3 5.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% decimal(9, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 4309 4388 54 3.6 274.0 1.0X -Parquet Vectorized (Pushdown) 1894 1904 11 8.3 120.4 2.3X -Native ORC Vectorized 4400 4446 39 3.6 279.7 1.0X -Native ORC Vectorized (Pushdown) 4503 4515 7 3.5 286.3 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% decimal(9, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8049 8105 54 2.0 511.7 1.0X -Parquet Vectorized (Pushdown) 7630 7665 25 2.1 485.1 1.1X -Native ORC Vectorized 8316 8392 45 1.9 528.7 1.0X -Native ORC Vectorized (Pushdown) 8501 8529 28 1.9 540.5 0.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% decimal(9, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8846 8921 79 1.8 562.4 1.0X -Parquet Vectorized (Pushdown) 8859 8918 45 1.8 563.2 1.0X -Native ORC Vectorized 9275 9314 28 1.7 589.7 1.0X -Native ORC Vectorized (Pushdown) 9407 9427 13 1.7 598.1 0.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 decimal(18, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3198 3226 35 4.9 203.3 1.0X -Parquet Vectorized (Pushdown) 101 106 4 155.3 6.4 31.6X -Native ORC Vectorized 3242 3272 30 4.9 206.2 1.0X -Native ORC Vectorized (Pushdown) 1729 1760 19 9.1 109.9 1.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% decimal(18, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3829 3855 23 4.1 243.4 1.0X -Parquet Vectorized (Pushdown) 1013 1027 18 15.5 64.4 3.8X -Native ORC Vectorized 3833 3852 21 4.1 243.7 1.0X -Native ORC Vectorized (Pushdown) 3925 3964 38 4.0 249.5 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% decimal(18, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6150 6195 48 2.6 391.0 1.0X -Parquet Vectorized (Pushdown) 4568 4609 24 3.4 290.4 1.3X -Native ORC Vectorized 6248 6283 29 2.5 397.2 1.0X -Native ORC Vectorized (Pushdown) 6358 6376 22 2.5 404.2 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% decimal(18, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8371 8452 46 1.9 532.2 1.0X -Parquet Vectorized (Pushdown) 8109 8134 25 1.9 515.6 1.0X -Native ORC Vectorized 8629 8703 72 1.8 548.6 1.0X -Native ORC Vectorized (Pushdown) 8722 8777 39 1.8 554.6 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 decimal(38, 2) row (value = 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 4512 4546 23 3.5 286.9 1.0X -Parquet Vectorized (Pushdown) 109 113 3 143.7 7.0 41.2X -Native ORC Vectorized 3297 3319 16 4.8 209.6 1.4X -Native ORC Vectorized (Pushdown) 175 182 6 89.9 11.1 25.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% decimal(38, 2) rows (value < 1572864): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5363 5389 25 2.9 340.9 1.0X -Parquet Vectorized (Pushdown) 1320 1342 18 11.9 83.9 4.1X -Native ORC Vectorized 4034 4057 18 3.9 256.5 1.3X -Native ORC Vectorized (Pushdown) 1225 1235 10 12.8 77.9 4.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% decimal(38, 2) rows (value < 7864320): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8240 8267 29 1.9 523.9 1.0X -Parquet Vectorized (Pushdown) 7615 8290 727 2.1 484.1 1.1X -Native ORC Vectorized 8604 8654 53 1.8 547.0 1.0X -Native ORC Vectorized (Pushdown) 6619 6664 27 2.4 420.8 1.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% decimal(38, 2) rows (value < 14155776): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 13919 14010 84 1.1 884.9 1.0X -Parquet Vectorized (Pushdown) 13455 13501 54 1.2 855.4 1.0X -Native ORC Vectorized 12117 12164 72 1.3 770.4 1.1X -Native ORC Vectorized (Pushdown) 11737 11801 37 1.3 746.2 1.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(9, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 5543 / 5564 2.8 352.4 1.0X +Parquet Vectorized (Pushdown) 168 / 174 93.7 10.7 33.0X +Native ORC Vectorized 4992 / 5052 3.2 317.4 1.1X +Native ORC Vectorized (Pushdown) 840 / 850 18.7 53.4 6.6X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(9, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 7312 / 7358 2.2 464.9 1.0X +Parquet Vectorized (Pushdown) 3008 / 3078 5.2 191.2 2.4X +Native ORC Vectorized 6775 / 6798 2.3 430.7 1.1X +Native ORC Vectorized (Pushdown) 6819 / 6832 2.3 433.5 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(9, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 13232 / 13241 1.2 841.3 1.0X +Parquet Vectorized (Pushdown) 12555 / 12569 1.3 798.2 1.1X +Native ORC Vectorized 12597 / 12627 1.2 800.9 1.1X +Native ORC Vectorized (Pushdown) 12677 / 12711 1.2 806.0 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(9, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 14725 / 14729 1.1 936.2 1.0X +Parquet Vectorized (Pushdown) 14781 / 14800 1.1 939.7 1.0X +Native ORC Vectorized 15360 / 15453 1.0 976.5 1.0X +Native ORC Vectorized (Pushdown) 15444 / 15466 1.0 981.9 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(18, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 5746 / 5763 2.7 365.3 1.0X +Parquet Vectorized (Pushdown) 166 / 169 94.8 10.6 34.6X +Native ORC Vectorized 5007 / 5023 3.1 318.3 1.1X +Native ORC Vectorized (Pushdown) 2629 / 2640 6.0 167.1 2.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(18, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6827 / 6864 2.3 434.0 1.0X +Parquet Vectorized (Pushdown) 1809 / 1827 8.7 115.0 3.8X +Native ORC Vectorized 6287 / 6296 2.5 399.7 1.1X +Native ORC Vectorized (Pushdown) 6364 / 6377 2.5 404.6 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(18, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 11315 / 11342 1.4 719.4 1.0X +Parquet Vectorized (Pushdown) 8431 / 8450 1.9 536.0 1.3X +Native ORC Vectorized 11591 / 11611 1.4 736.9 1.0X +Native ORC Vectorized (Pushdown) 11424 / 11475 1.4 726.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(18, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 15703 / 15712 1.0 998.4 1.0X +Parquet Vectorized (Pushdown) 14982 / 15009 1.0 952.5 1.0X +Native ORC Vectorized 16887 / 16955 0.9 1073.7 0.9X +Native ORC Vectorized (Pushdown) 16518 / 16530 1.0 1050.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 decimal(38, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 8101 / 8130 1.9 515.1 1.0X +Parquet Vectorized (Pushdown) 184 / 187 85.6 11.7 44.1X +Native ORC Vectorized 4998 / 5027 3.1 317.8 1.6X +Native ORC Vectorized (Pushdown) 165 / 168 95.6 10.5 49.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% decimal(38, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 9405 / 9447 1.7 597.9 1.0X +Parquet Vectorized (Pushdown) 2269 / 2275 6.9 144.2 4.1X +Native ORC Vectorized 6167 / 6203 2.6 392.1 1.5X +Native ORC Vectorized (Pushdown) 1783 / 1787 8.8 113.3 5.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% decimal(38, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 14700 / 14707 1.1 934.6 1.0X +Parquet Vectorized (Pushdown) 10699 / 10712 1.5 680.2 1.4X +Native ORC Vectorized 10687 / 10703 1.5 679.5 1.4X +Native ORC Vectorized (Pushdown) 8364 / 8415 1.9 531.8 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% decimal(38, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 19780 / 19894 0.8 1257.6 1.0X +Parquet Vectorized (Pushdown) 19003 / 19025 0.8 1208.1 1.0X +Native ORC Vectorized 15385 / 15404 1.0 978.2 1.3X +Native ORC Vectorized (Pushdown) 15032 / 15060 1.0 955.7 1.3X ================================================================================================ Pushdown benchmark for InSet -> InFilters ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 5, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 7961 8018 63 2.0 506.1 1.0X -Parquet Vectorized (Pushdown) 475 496 18 33.1 30.2 16.7X -Native ORC Vectorized 4700 5828 887 3.3 298.8 1.7X -Native ORC Vectorized (Pushdown) 576 593 19 27.3 36.6 13.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 5, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5976 6101 93 2.6 380.0 1.0X -Parquet Vectorized (Pushdown) 377 384 5 41.7 24.0 15.9X -Native ORC Vectorized 4535 4573 28 3.5 288.3 1.3X -Native ORC Vectorized (Pushdown) 557 575 19 28.2 35.4 10.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 5, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5888 5962 49 2.7 374.3 1.0X -Parquet Vectorized (Pushdown) 373 383 12 42.2 23.7 15.8X -Native ORC Vectorized 4525 4561 31 3.5 287.7 1.3X -Native ORC Vectorized (Pushdown) 563 575 15 27.9 35.8 10.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 10, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 5883 5916 28 2.7 374.0 1.0X -Parquet Vectorized (Pushdown) 394 402 7 39.9 25.1 14.9X -Native ORC Vectorized 4561 6094 1481 3.4 290.0 1.3X -Native ORC Vectorized (Pushdown) 579 618 34 27.1 36.8 10.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 10, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6225 6538 651 2.5 395.8 1.0X -Parquet Vectorized (Pushdown) 415 422 7 37.9 26.4 15.0X -Native ORC Vectorized 4652 4698 41 3.4 295.8 1.3X -Native ORC Vectorized (Pushdown) 581 592 14 27.1 36.9 10.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 10, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6226 6249 19 2.5 395.9 1.0X -Parquet Vectorized (Pushdown) 412 418 4 38.1 26.2 15.1X -Native ORC Vectorized 4657 4690 24 3.4 296.1 1.3X -Native ORC Vectorized (Pushdown) 576 588 15 27.3 36.6 10.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 50, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6459 6476 18 2.4 410.6 1.0X -Parquet Vectorized (Pushdown) 6574 6600 23 2.4 418.0 1.0X -Native ORC Vectorized 4884 4933 37 3.2 310.5 1.3X -Native ORC Vectorized (Pushdown) 688 700 8 22.9 43.7 9.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 50, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6432 6504 62 2.4 408.9 1.0X -Parquet Vectorized (Pushdown) 6624 6657 32 2.4 421.1 1.0X -Native ORC Vectorized 4910 4950 28 3.2 312.2 1.3X -Native ORC Vectorized (Pushdown) 696 709 9 22.6 44.3 9.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 50, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6453 6496 30 2.4 410.2 1.0X -Parquet Vectorized (Pushdown) 6581 6614 26 2.4 418.4 1.0X -Native ORC Vectorized 4900 4965 53 3.2 311.5 1.3X -Native ORC Vectorized (Pushdown) 706 716 7 22.3 44.9 9.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 100, distribution: 10): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6421 6450 26 2.4 408.2 1.0X -Parquet Vectorized (Pushdown) 6570 6586 17 2.4 417.7 1.0X -Native ORC Vectorized 4885 4911 29 3.2 310.6 1.3X -Native ORC Vectorized (Pushdown) 765 785 14 20.6 48.6 8.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 100, distribution: 50): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6171 6261 107 2.5 392.3 1.0X -Parquet Vectorized (Pushdown) 6269 6324 36 2.5 398.6 1.0X -Native ORC Vectorized 4782 4806 18 3.3 304.1 1.3X -Native ORC Vectorized (Pushdown) 829 838 10 19.0 52.7 7.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -InSet -> InFilters (values count: 100, distribution: 90): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6142 6190 56 2.6 390.5 1.0X -Parquet Vectorized (Pushdown) 6253 6287 29 2.5 397.6 1.0X -Native ORC Vectorized 4787 4834 43 3.3 304.4 1.3X -Native ORC Vectorized (Pushdown) 813 836 17 19.4 51.7 7.6X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10521 / 10534 1.5 668.9 1.0X +Parquet Vectorized (Pushdown) 677 / 691 23.2 43.1 15.5X +Native ORC Vectorized 6768 / 6776 2.3 430.3 1.6X +Native ORC Vectorized (Pushdown) 501 / 512 31.4 31.8 21.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10531 / 10538 1.5 669.5 1.0X +Parquet Vectorized (Pushdown) 677 / 718 23.2 43.0 15.6X +Native ORC Vectorized 6765 / 6773 2.3 430.1 1.6X +Native ORC Vectorized (Pushdown) 499 / 507 31.5 31.7 21.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 5, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10540 / 10553 1.5 670.1 1.0X +Parquet Vectorized (Pushdown) 678 / 710 23.2 43.1 15.5X +Native ORC Vectorized 6787 / 6794 2.3 431.5 1.6X +Native ORC Vectorized (Pushdown) 501 / 509 31.4 31.9 21.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10551 / 10559 1.5 670.8 1.0X +Parquet Vectorized (Pushdown) 703 / 708 22.4 44.7 15.0X +Native ORC Vectorized 6791 / 6802 2.3 431.7 1.6X +Native ORC Vectorized (Pushdown) 519 / 526 30.3 33.0 20.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10561 / 10565 1.5 671.4 1.0X +Parquet Vectorized (Pushdown) 711 / 716 22.1 45.2 14.9X +Native ORC Vectorized 6791 / 6806 2.3 431.8 1.6X +Native ORC Vectorized (Pushdown) 529 / 537 29.8 33.6 20.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 10, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10572 / 10590 1.5 672.1 1.0X +Parquet Vectorized (Pushdown) 713 / 716 22.1 45.3 14.8X +Native ORC Vectorized 6808 / 6815 2.3 432.9 1.6X +Native ORC Vectorized (Pushdown) 530 / 541 29.7 33.7 19.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10871 / 10882 1.4 691.2 1.0X +Parquet Vectorized (Pushdown) 11104 / 11110 1.4 706.0 1.0X +Native ORC Vectorized 7088 / 7104 2.2 450.7 1.5X +Native ORC Vectorized (Pushdown) 665 / 677 23.6 42.3 16.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10861 / 10867 1.4 690.5 1.0X +Parquet Vectorized (Pushdown) 11094 / 11099 1.4 705.3 1.0X +Native ORC Vectorized 7075 / 7092 2.2 449.8 1.5X +Native ORC Vectorized (Pushdown) 718 / 733 21.9 45.6 15.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 50, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10868 / 10887 1.4 691.0 1.0X +Parquet Vectorized (Pushdown) 11100 / 11106 1.4 705.7 1.0X +Native ORC Vectorized 7087 / 7093 2.2 450.6 1.5X +Native ORC Vectorized (Pushdown) 712 / 731 22.1 45.3 15.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10850 / 10888 1.4 689.8 1.0X +Parquet Vectorized (Pushdown) 11086 / 11105 1.4 704.9 1.0X +Native ORC Vectorized 7090 / 7101 2.2 450.8 1.5X +Native ORC Vectorized (Pushdown) 867 / 882 18.1 55.1 12.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10816 / 10819 1.5 687.7 1.0X +Parquet Vectorized (Pushdown) 11052 / 11059 1.4 702.7 1.0X +Native ORC Vectorized 7037 / 7044 2.2 447.4 1.5X +Native ORC Vectorized (Pushdown) 919 / 931 17.1 58.4 11.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +InSet -> InFilters (values count: 100, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10807 / 10815 1.5 687.1 1.0X +Parquet Vectorized (Pushdown) 11047 / 11054 1.4 702.4 1.0X +Native ORC Vectorized 7042 / 7047 2.2 447.7 1.5X +Native ORC Vectorized (Pushdown) 950 / 961 16.6 60.4 11.4X ================================================================================================ Pushdown benchmark for tinyint ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 tinyint row (value = CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3402 3419 14 4.6 216.3 1.0X -Parquet Vectorized (Pushdown) 193 202 12 81.5 12.3 17.6X -Native ORC Vectorized 2106 2131 21 7.5 133.9 1.6X -Native ORC Vectorized (Pushdown) 280 289 8 56.2 17.8 12.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3893 3921 26 4.0 247.5 1.0X -Parquet Vectorized (Pushdown) 966 972 7 16.3 61.4 4.0X -Native ORC Vectorized 2604 2624 22 6.0 165.5 1.5X -Native ORC Vectorized (Pushdown) 927 942 13 17.0 59.0 4.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6048 6081 25 2.6 384.5 1.0X -Parquet Vectorized (Pushdown) 4388 4411 20 3.6 279.0 1.4X -Native ORC Vectorized 4714 4725 16 3.3 299.7 1.3X -Native ORC Vectorized (Pushdown) 4123 4386 338 3.8 262.1 1.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8462 9675 925 1.9 538.0 1.0X -Parquet Vectorized (Pushdown) 8124 8156 28 1.9 516.5 1.0X -Native ORC Vectorized 6884 6934 31 2.3 437.7 1.2X -Native ORC Vectorized (Pushdown) 6764 6793 26 2.3 430.1 1.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 tinyint row (value = CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6034 / 6048 2.6 383.6 1.0X +Parquet Vectorized (Pushdown) 333 / 344 47.2 21.2 18.1X +Native ORC Vectorized 3240 / 3307 4.9 206.0 1.9X +Native ORC Vectorized (Pushdown) 330 / 341 47.6 21.0 18.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6759 / 6800 2.3 429.7 1.0X +Parquet Vectorized (Pushdown) 1533 / 1537 10.3 97.5 4.4X +Native ORC Vectorized 3863 / 3874 4.1 245.6 1.7X +Native ORC Vectorized (Pushdown) 1235 / 1248 12.7 78.5 5.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10247 / 10289 1.5 651.5 1.0X +Parquet Vectorized (Pushdown) 7430 / 7453 2.1 472.4 1.4X +Native ORC Vectorized 6995 / 7009 2.2 444.7 1.5X +Native ORC Vectorized (Pushdown) 5561 / 5571 2.8 353.6 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 13949 / 13991 1.1 886.9 1.0X +Parquet Vectorized (Pushdown) 13486 / 13511 1.2 857.4 1.0X +Native ORC Vectorized 10149 / 10186 1.5 645.3 1.4X +Native ORC Vectorized (Pushdown) 9889 / 9905 1.6 628.7 1.4X ================================================================================================ Pushdown benchmark for Timestamp ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3712 3750 30 4.2 236.0 1.0X -Parquet Vectorized (Pushdown) 3739 3764 30 4.2 237.7 1.0X -Native ORC Vectorized 2064 2087 16 7.6 131.3 1.8X -Native ORC Vectorized (Pushdown) 147 153 7 106.6 9.4 25.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 4337 4376 42 3.6 275.8 1.0X -Parquet Vectorized (Pushdown) 4352 4376 19 3.6 276.7 1.0X -Native ORC Vectorized 2648 2668 16 5.9 168.4 1.6X -Native ORC Vectorized (Pushdown) 919 937 13 17.1 58.4 4.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6703 7302 1119 2.3 426.2 1.0X -Parquet Vectorized (Pushdown) 6687 6709 28 2.4 425.1 1.0X -Native ORC Vectorized 4842 4876 28 3.2 307.8 1.4X -Native ORC Vectorized (Pushdown) 3925 3945 11 4.0 249.6 1.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8894 8914 19 1.8 565.5 1.0X -Parquet Vectorized (Pushdown) 8930 8962 29 1.8 567.8 1.0X -Native ORC Vectorized 7086 7115 34 2.2 450.5 1.3X -Native ORC Vectorized (Pushdown) 6973 7018 44 2.3 443.3 1.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3282 3321 23 4.8 208.6 1.0X -Parquet Vectorized (Pushdown) 107 112 5 147.4 6.8 30.8X -Native ORC Vectorized 2046 2057 11 7.7 130.1 1.6X -Native ORC Vectorized (Pushdown) 146 152 9 107.5 9.3 22.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3922 3944 21 4.0 249.3 1.0X -Parquet Vectorized (Pushdown) 1033 1040 6 15.2 65.7 3.8X -Native ORC Vectorized 2659 2674 11 5.9 169.0 1.5X -Native ORC Vectorized (Pushdown) 921 938 10 17.1 58.6 4.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6254 6287 33 2.5 397.6 1.0X -Parquet Vectorized (Pushdown) 4673 4697 21 3.4 297.1 1.3X -Native ORC Vectorized 4884 4914 27 3.2 310.5 1.3X -Native ORC Vectorized (Pushdown) 3976 3996 14 4.0 252.8 1.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8269 8458 110 1.9 525.8 1.0X -Parquet Vectorized (Pushdown) 7961 7980 21 2.0 506.1 1.0X -Native ORC Vectorized 6993 7026 22 2.2 444.6 1.2X -Native ORC Vectorized (Pushdown) 6883 6928 73 2.3 437.6 1.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3357 3375 17 4.7 213.4 1.0X -Parquet Vectorized (Pushdown) 101 105 5 156.1 6.4 33.3X -Native ORC Vectorized 2003 2031 17 7.9 127.4 1.7X -Native ORC Vectorized (Pushdown) 146 151 8 108.1 9.3 23.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 3983 3997 11 3.9 253.2 1.0X -Parquet Vectorized (Pushdown) 1018 1034 16 15.4 64.7 3.9X -Native ORC Vectorized 2591 2611 15 6.1 164.8 1.5X -Native ORC Vectorized (Pushdown) 911 917 4 17.3 57.9 4.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 6218 6235 17 2.5 395.3 1.0X -Parquet Vectorized (Pushdown) 4571 4620 38 3.4 290.6 1.4X -Native ORC Vectorized 4799 4831 29 3.3 305.1 1.3X -Native ORC Vectorized (Pushdown) 3895 3915 14 4.0 247.7 1.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 8381 8407 29 1.9 532.9 1.0X -Parquet Vectorized (Pushdown) 8058 8132 84 2.0 512.3 1.0X -Native ORC Vectorized 6990 7043 35 2.3 444.4 1.2X -Native ORC Vectorized (Pushdown) 6859 6897 24 2.3 436.1 1.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6307 / 6310 2.5 401.0 1.0X +Parquet Vectorized (Pushdown) 6360 / 6397 2.5 404.3 1.0X +Native ORC Vectorized 2912 / 2917 5.4 185.1 2.2X +Native ORC Vectorized (Pushdown) 138 / 141 114.4 8.7 45.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 7225 / 7233 2.2 459.4 1.0X +Parquet Vectorized (Pushdown) 7250 / 7255 2.2 461.0 1.0X +Native ORC Vectorized 3772 / 3783 4.2 239.8 1.9X +Native ORC Vectorized (Pushdown) 1277 / 1282 12.3 81.2 5.7X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10952 / 10965 1.4 696.3 1.0X +Parquet Vectorized (Pushdown) 10985 / 10998 1.4 698.4 1.0X +Native ORC Vectorized 7178 / 7227 2.2 456.3 1.5X +Native ORC Vectorized (Pushdown) 5825 / 5830 2.7 370.3 1.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 14560 / 14583 1.1 925.7 1.0X +Parquet Vectorized (Pushdown) 14608 / 14620 1.1 928.7 1.0X +Native ORC Vectorized 10601 / 10640 1.5 674.0 1.4X +Native ORC Vectorized (Pushdown) 10392 / 10406 1.5 660.7 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 5653 / 5658 2.8 359.4 1.0X +Parquet Vectorized (Pushdown) 165 / 169 95.1 10.5 34.2X +Native ORC Vectorized 2918 / 2921 5.4 185.5 1.9X +Native ORC Vectorized (Pushdown) 137 / 145 114.9 8.7 41.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6540 / 6552 2.4 415.8 1.0X +Parquet Vectorized (Pushdown) 1610 / 1614 9.8 102.3 4.1X +Native ORC Vectorized 3775 / 3788 4.2 240.0 1.7X +Native ORC Vectorized (Pushdown) 1274 / 1277 12.3 81.0 5.1X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10259 / 10278 1.5 652.3 1.0X +Parquet Vectorized (Pushdown) 7591 / 7601 2.1 482.6 1.4X +Native ORC Vectorized 7185 / 7194 2.2 456.8 1.4X +Native ORC Vectorized (Pushdown) 5828 / 5843 2.7 370.6 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 13850 / 13868 1.1 880.5 1.0X +Parquet Vectorized (Pushdown) 13433 / 13450 1.2 854.0 1.0X +Native ORC Vectorized 10635 / 10669 1.5 676.1 1.3X +Native ORC Vectorized (Pushdown) 10437 / 10448 1.5 663.6 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 5884 / 5888 2.7 374.1 1.0X +Parquet Vectorized (Pushdown) 166 / 170 94.7 10.6 35.4X +Native ORC Vectorized 2913 / 2916 5.4 185.2 2.0X +Native ORC Vectorized (Pushdown) 136 / 144 115.4 8.7 43.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6763 / 6776 2.3 430.0 1.0X +Parquet Vectorized (Pushdown) 1634 / 1638 9.6 103.9 4.1X +Native ORC Vectorized 3777 / 3785 4.2 240.1 1.8X +Native ORC Vectorized (Pushdown) 1276 / 1279 12.3 81.2 5.3X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 10460 / 10469 1.5 665.0 1.0X +Parquet Vectorized (Pushdown) 7689 / 7698 2.0 488.9 1.4X +Native ORC Vectorized 7190 / 7197 2.2 457.1 1.5X +Native ORC Vectorized (Pushdown) 5820 / 5834 2.7 370.0 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 14033 / 14039 1.1 892.2 1.0X +Parquet Vectorized (Pushdown) 13608 / 13636 1.2 865.2 1.0X +Native ORC Vectorized 10635 / 10686 1.5 676.2 1.3X +Native ORC Vectorized (Pushdown) 10420 / 10442 1.5 662.5 1.3X ================================================================================================ Pushdown benchmark with many filters ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 1 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 152 157 4 0.0 151562084.0 1.0X -Parquet Vectorized (Pushdown) 156 160 5 0.0 155523780.0 1.0X -Native ORC Vectorized 145 150 4 0.0 144571223.0 1.0X -Native ORC Vectorized (Pushdown) 145 150 4 0.0 144905986.0 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 250 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 250 257 6 0.0 249773194.0 1.0X -Parquet Vectorized (Pushdown) 263 268 5 0.0 262666962.0 1.0X -Native ORC Vectorized 251 263 9 0.0 251438048.0 1.0X -Native ORC Vectorized (Pushdown) 260 263 3 0.0 259811377.0 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Parquet Vectorized 366 377 6 0.0 366122909.0 1.0X -Parquet Vectorized (Pushdown) 395 399 3 0.0 394562758.0 0.9X -Native ORC Vectorized 367 374 4 0.0 366931788.0 1.0X -Native ORC Vectorized (Pushdown) 378 382 4 0.0 377890425.0 1.0X - - -================================================================================================ -Pushdown benchmark with unbalanced Column -================================================================================================ - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 25 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Native ORC Vectorized (Pushdown) 94 97 3 0.0 94040154.0 1.0X -Native Parquet Vectorized (Pushdown) 88 92 3 0.0 88400127.0 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 500 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Native ORC Vectorized (Pushdown) 958 964 5 0.0 957697803.0 1.0X -Native Parquet Vectorized (Pushdown) 1021 1034 13 0.0 1021483274.0 0.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.14.2 -Intel(R) Core(TM) i7-8850H CPU @ 2.60GHz -Select 1 row with 1000 filters: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------- -Native ORC Vectorized (Pushdown) 5465 5469 6 0.0 5465108758.0 1.0X -Native Parquet Vectorized (Pushdown) 5741 6229 721 0.0 5741040597.0 1.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 row with 1 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 319 / 323 0.0 318789986.0 1.0X +Parquet Vectorized (Pushdown) 323 / 347 0.0 322755287.0 1.0X +Native ORC Vectorized 316 / 336 0.0 315670745.0 1.0X +Native ORC Vectorized (Pushdown) 317 / 320 0.0 317392594.0 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 row with 250 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 2192 / 2218 0.0 2191883823.0 1.0X +Parquet Vectorized (Pushdown) 2675 / 2687 0.0 2675439029.0 0.8X +Native ORC Vectorized 2158 / 2162 0.0 2157646071.0 1.0X +Native ORC Vectorized (Pushdown) 2309 / 2326 0.0 2309096612.0 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Select 1 row with 500 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Parquet Vectorized 6219 / 6248 0.0 6218727737.0 1.0X +Parquet Vectorized (Pushdown) 7376 / 7436 0.0 7375977710.0 0.8X +Native ORC Vectorized 6252 / 6279 0.0 6252473320.0 1.0X +Native ORC Vectorized (Pushdown) 6858 / 6876 0.0 6857854486.0 0.9X diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index eba83dc570c6..b04024371713 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -24,16 +24,12 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.internal.config.UI._ -import org.apache.spark.sql.{Column, DataFrame, SparkSession} -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Expression, Literal, Or} +import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.execution.datasources.DataSourceStrategy -import org.apache.spark.sql.execution.datasources.orc.OrcFilters -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.functions.monotonically_increasing_id import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType -import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.{ByteType, Decimal, DecimalType, TimestampType} /** * Benchmark to measure read performance with Filter pushdown. @@ -139,34 +135,6 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { benchmark.run() } - def filterPushDownBenchmarkWithColumn( - values: Int, - title: String, - whereColumn: Column, - selectExpr: String = "*" - ): Unit = { - val benchmark = new Benchmark(title, values, minNumIters = 5, output = output) - benchmark.addCase("Native ORC Vectorized (Pushdown)") { _ => - withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { - spark - .table("orcTable") - .select(selectExpr) - .filter(whereColumn) - .collect() - } - } - benchmark.addCase("Native Parquet Vectorized (Pushdown)") { _ => - withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true") { - spark - .table("parquetTable") - .select(selectExpr) - .filter(whereColumn) - .collect() - } - } - benchmark.run() - } - private def runIntBenchmark(numRows: Int, width: Int, mid: Int): Unit = { Seq("value IS NULL", s"$mid < value AND value < $mid").foreach { whereExpr => val title = s"Select 0 int row ($whereExpr)".replace("value AND value", "value") @@ -394,13 +362,6 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } runBenchmark(s"Pushdown benchmark with many filters") { - // This benchmark and the next one are similar in that they both test predicate pushdown - // where the filter itself is very large. There have been cases where the filter conversion - // would take minutes to hours for large filters due to it being implemented with exponential - // complexity in the height of the filter tree. - // The difference between these two benchmarks is that this one benchmarks pushdown with a - // large string filter (`a AND b AND c ...`), whereas the next one benchmarks pushdown with - // a large Column-based filter (`col(a) || (col(b) || (col(c)...))`). val numRows = 1 val width = 500 @@ -417,27 +378,5 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } } } - - runBenchmark(s"Pushdown benchmark with unbalanced Column") { - val numRows = 1 - val width = 200 - - withTempPath { dir => - val columns = (1 to width).map(i => s"id c$i") - val df = spark.range(1).selectExpr(columns: _*) - withTempTable("orcTable", "parquetTable") { - saveAsTable(df, dir) - Seq(25, 500, 1000).foreach { numFilter => - val whereColumn = (1 to numFilter) - .map(i => col("c1") === lit(i)) - .foldLeft(lit(false))(_ || _) - filterPushDownBenchmarkWithColumn( - numRows, - s"Select 1 row with $numFilter filters", - whereColumn) - } - } - } - } } }