From 895638f47fae356b80f75cad42fc635446be108b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 16 Dec 2019 20:31:48 +0300 Subject: [PATCH 01/43] Return Seq[InternalRow] from convert() --- .../sql/catalyst/csv/UnivocityParser.scala | 52 ++++++++++++------- .../catalyst/expressions/csvExpressions.scala | 2 +- .../apache/spark/sql/DataFrameReader.scala | 2 +- 3 files changed, 36 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 661525a65294d..a5c1bf0c4db57 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -72,7 +72,8 @@ class UnivocityParser( new CsvParser(parserSetting) } - private val row = new GenericInternalRow(requiredSchema.length) + private val singleRow = Seq(new GenericInternalRow(requiredSchema.length)) + private val noRows = Seq.empty[InternalRow] private val timestampFormatter = TimestampFormatter( options.timestampFormat, @@ -194,7 +195,7 @@ class UnivocityParser( private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. - (_: String) => InternalRow.empty + (_: String) => Seq(InternalRow.empty) } else { // parse if the columnPruning is disabled or requiredSchema is nonEmpty (input: String) => convert(tokenizer.parseLine(input)) @@ -204,7 +205,7 @@ class UnivocityParser( * Parses a single CSV string and turns it into either one resulting row or no row (if the * the record is malformed). */ - def parse(input: String): InternalRow = doParse(input) + def parse(input: String): Seq[InternalRow] = doParse(input) private val getToken = if (options.columnPruning) { (tokens: Array[String], index: Int) => tokens(index) @@ -212,7 +213,7 @@ class UnivocityParser( (tokens: Array[String], index: Int) => tokens(tokenIndexArr(index)) } - private def convert(tokens: Array[String]): InternalRow = { + private def convert(tokens: Array[String]): Seq[InternalRow] = { if (tokens == null) { throw BadRecordException( () => getCurrentInput, @@ -229,7 +230,7 @@ class UnivocityParser( } def getPartialResult(): Option[InternalRow] = { try { - Some(convert(checkedTokens)) + convert(checkedTokens).headOption } catch { case _: BadRecordException => None } @@ -243,23 +244,38 @@ class UnivocityParser( } else { // When the length of the returned tokens is identical to the length of the parsed schema, // we just need to convert the tokens that correspond to the required columns. - var badRecordException: Option[Throwable] = None var i = 0 + val r = singleRow.head + var skipValueConversion = false + var badRecordException: Option[Throwable] = None + while (i < requiredSchema.length) { - try { - row(i) = valueConverters(i).apply(getToken(tokens, i)) - } catch { - case NonFatal(e) => - badRecordException = badRecordException.orElse(Some(e)) - row.setNullAt(i) + if (skipValueConversion) { + r.setNullAt(i) + } else { + try { + r(i) = valueConverters(i).apply(getToken(tokens, i)) + if (false) { + skipValueConversion = true + } + } catch { + case NonFatal(e) => + badRecordException = Some(e) + skipValueConversion = true + r.setNullAt(i) + } } i += 1 } - - if (badRecordException.isEmpty) { - row + if (skipValueConversion) { + if (badRecordException.isDefined) { + throw BadRecordException( + () => getCurrentInput, () => singleRow.headOption, badRecordException.get) + } else { + noRows + } } else { - throw BadRecordException(() => getCurrentInput, () => Some(row), badRecordException.get) + singleRow } } } @@ -291,7 +307,7 @@ private[sql] object UnivocityParser { schema: StructType): Iterator[InternalRow] = { val tokenizer = parser.tokenizer val safeParser = new FailureSafeParser[Array[String]]( - input => Seq(parser.convert(input)), + input => parser.convert(input), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) @@ -344,7 +360,7 @@ private[sql] object UnivocityParser { val filteredLines: Iterator[String] = CSVExprUtils.filterCommentAndEmpty(lines, options) val safeParser = new FailureSafeParser[String]( - input => Seq(parser.parse(input)), + input => parser.parse(input), parser.options.parseMode, schema, parser.options.columnNameOfCorruptRecord) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala index 67c24f687af08..aa45a79ebd225 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -114,7 +114,7 @@ case class CsvToStructs( StructType(nullableSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val rawParser = new UnivocityParser(actualSchema, actualSchema, parsedOptions) new FailureSafeParser[String]( - input => Seq(rawParser.parse(input)), + input => rawParser.parse(input), mode, nullableSchema, parsedOptions.columnNameOfCorruptRecord) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 8570e4640feea..3793e6c0480a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -548,7 +548,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val parsed = linesWithoutHeader.mapPartitions { iter => val rawParser = new UnivocityParser(actualSchema, parsedOptions) val parser = new FailureSafeParser[String]( - input => Seq(rawParser.parse(input)), + input => rawParser.parse(input), parsedOptions.parseMode, schema, parsedOptions.columnNameOfCorruptRecord) From 4bc8d9b08ccb2d87c33dab937817ca56ed37ddde Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 16 Dec 2019 20:57:38 +0300 Subject: [PATCH 02/43] Pass filters to CSV datasource v1 --- .../apache/spark/sql/catalyst/csv/UnivocityParser.scala | 9 ++++++--- .../sql/execution/datasources/csv/CSVFileFormat.scala | 6 +++++- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index a5c1bf0c4db57..8c03efe5f7183 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -20,13 +20,12 @@ package org.apache.spark.sql.catalyst.csv import java.io.InputStream import scala.util.control.NonFatal - import com.univocity.parsers.csv.CsvParser - import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -43,11 +42,15 @@ import org.apache.spark.unsafe.types.UTF8String class UnivocityParser( dataSchema: StructType, requiredSchema: StructType, - val options: CSVOptions) extends Logging { + val options: CSVOptions, + filters: Seq[Filter]) extends Logging { require(requiredSchema.toSet.subsetOf(dataSchema.toSet), s"requiredSchema (${requiredSchema.catalogString}) should be the subset of " + s"dataSchema (${dataSchema.catalogString}).") + def this(dataSchema: StructType, requiredSchema: StructType, options: CSVOptions) = { + this(dataSchema, requiredSchema, options, Seq.empty) + } def this(schema: StructType, options: CSVOptions) = this(schema, schema, options) // A `ValueConverter` is responsible for converting the given value to a desired type. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 8abc6fcacd4c5..00d90f19ddb7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -134,7 +134,11 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val actualRequiredSchema = StructType( requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val parser = new UnivocityParser(actualDataSchema, actualRequiredSchema, parsedOptions) + val parser = new UnivocityParser( + actualDataSchema, + actualRequiredSchema, + parsedOptions, + filters) val schema = if (columnPruning) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( From 012419916f6e75171751eec672cc0e22f33c3453 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Dec 2019 00:08:05 +0300 Subject: [PATCH 03/43] Add CSVFilters --- .../spark/sql/catalyst/csv/CSVFilters.scala | 28 +++++++++++++++++++ .../sql/catalyst/csv/UnivocityParser.scala | 6 +++- 2 files changed, 33 insertions(+), 1 deletion(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala new file mode 100644 index 0000000000000..a0fb7b3b84bba --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -0,0 +1,28 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.csv + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.StructType + +class CSVFilters(filters: Seq[Filter], schema: StructType) { + def skipRow(row: InternalRow): Boolean = { + true + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 8c03efe5f7183..4b8e95e315586 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.catalyst.csv import java.io.InputStream import scala.util.control.NonFatal + import com.univocity.parsers.csv.CsvParser + import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} @@ -87,6 +89,8 @@ class UnivocityParser( options.zoneId, options.locale) + private val csvFilters = new CSVFilters(filters, requiredSchema) + // Retrieve the raw record string. private def getCurrentInput: UTF8String = { UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) @@ -258,7 +262,7 @@ class UnivocityParser( } else { try { r(i) = valueConverters(i).apply(getToken(tokens, i)) - if (false) { + if (csvFilters.skipRow(r)) { skipValueConversion = true } } catch { From fb8912e8f5f11bc319e7bce85fd2092186fd96d8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Dec 2019 00:42:27 +0300 Subject: [PATCH 04/43] Add filterToExpression --- .../spark/sql/catalyst/csv/CSVFilters.scala | 82 ++++++++++++++++++- 1 file changed, 80 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index a0fb7b3b84bba..37764a68db488 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -17,11 +17,89 @@ package org.apache.spark.sql.catalyst.csv +import scala.util.Try + import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType -class CSVFilters(filters: Seq[Filter], schema: StructType) { +class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { + private def reorderFilters(): Array[Seq[sources.Filter]] = { + val numFields = schema.fields.length + val ordered = new Array[Seq[sources.Filter]](numFields) + for (filter <- filters) { + val maxIndex = filter.references.map(schema.fieldIndex).max + if (maxIndex < numFields) { + val f = ordered(maxIndex) + ordered(maxIndex) = if (f == null) Seq(filter) else f :+ filter + } else { + throw new IllegalArgumentException( + s"The filter $filter has an attribute out of the schema $schema: $maxIndex") + } + } + ordered + } + + private def toRef(attr: String): Option[BoundReference] = { + schema.getFieldIndex(attr).map { index => + val field = schema(index) + BoundReference(schema.fieldIndex(attr), field.dataType, field.nullable) + } + } + + private def toLiteral(value: Any): Option[Literal] = { + Try(Literal(value)).toOption + } + + private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = a.zip(b).headOption + + private def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { + zip(toRef(name), toLiteral(value)) + } + + private def filterToExpression(filter: sources.Filter): Option[Expression] = { + def translate(filter: sources.Filter): Option[Expression] = filter match { + case sources.And(left, right) => + zip(translate(left), translate(right)).map(And.tupled) + case sources.Or(left, right) => + zip(translate(left), translate(right)).map(Or.tupled) + case sources.Not(child) => + translate(child).map(Not) + case sources.EqualTo(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualTo.tupled) + case sources.EqualNullSafe(attribute, value) => + zipAttributeAndValue(attribute, value).map(EqualNullSafe.tupled) + case sources.IsNull(attribute) => + toRef(attribute).map(IsNull) + case sources.IsNotNull(attribute) => + toRef(attribute).map(IsNotNull) + case sources.In(attribute, values) => + val literals = values.toSeq.flatMap(toLiteral) + if (literals.length == values.length) { + toRef(attribute).map(In(_, literals)) + } else { + None + } + case sources.GreaterThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThan.tupled) + case sources.GreaterThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(GreaterThanOrEqual.tupled) + case sources.LessThan(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThan.tupled) + case sources.LessThanOrEqual(attribute, value) => + zipAttributeAndValue(attribute, value).map(LessThanOrEqual.tupled) + case sources.StringContains(attribute, value) => + zipAttributeAndValue(attribute, value).map(Contains.tupled) + case sources.StringStartsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(StartsWith.tupled) + case sources.StringEndsWith(attribute, value) => + zipAttributeAndValue(attribute, value).map(EndsWith.tupled) + case _ => None + } + translate(filter) + } + def skipRow(row: InternalRow): Boolean = { true } From c2515b6aafa75ab62e00793cc845acc0186d4d07 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Dec 2019 11:54:17 +0300 Subject: [PATCH 05/43] Initial impl of CSVFilters --- .../spark/sql/catalyst/csv/CSVFilters.scala | 31 +++++++++---------- .../sql/catalyst/csv/UnivocityParser.scala | 2 +- 2 files changed, 15 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 37764a68db488..122c43904e149 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -25,21 +25,6 @@ import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { - private def reorderFilters(): Array[Seq[sources.Filter]] = { - val numFields = schema.fields.length - val ordered = new Array[Seq[sources.Filter]](numFields) - for (filter <- filters) { - val maxIndex = filter.references.map(schema.fieldIndex).max - if (maxIndex < numFields) { - val f = ordered(maxIndex) - ordered(maxIndex) = if (f == null) Seq(filter) else f :+ filter - } else { - throw new IllegalArgumentException( - s"The filter $filter has an attribute out of the schema $schema: $maxIndex") - } - } - ordered - } private def toRef(attr: String): Option[BoundReference] = { schema.getFieldIndex(attr).map { index => @@ -100,7 +85,19 @@ class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { translate(filter) } - def skipRow(row: InternalRow): Boolean = { - true + private val predicates: Array[Seq[BasePredicate]] = { + val parr = Array.fill(schema.fields.length)(Seq.empty[BasePredicate]) + for (filter <- filters) { + val index = filter.references.map(schema.fieldIndex).max + for (expr <- filterToExpression(filter)) { + val predicate = Predicate.create(expr) + parr(index) :+= predicate + } + } + parr + } + + def skipRow(row: InternalRow, index: Int): Boolean = { + predicates(index).exists(!_.eval(row)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 4b8e95e315586..9f00460ef80e3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -262,7 +262,7 @@ class UnivocityParser( } else { try { r(i) = valueConverters(i).apply(getToken(tokens, i)) - if (csvFilters.skipRow(r)) { + if (csvFilters.skipRow(r, i)) { skipValueConversion = true } } catch { From 9ced6075b9ac980bb0e055312a05a6e1a6e4dab8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Dec 2019 13:05:32 +0300 Subject: [PATCH 06/43] Support filters push down in CSV v2 --- .../v2/csv/CSVPartitionReaderFactory.scala | 7 ++++-- .../datasources/v2/csv/CSVScan.scala | 6 +++-- .../datasources/v2/csv/CSVScanBuilder.scala | 23 ++++++++++++++++--- 3 files changed, 29 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index a20b0f1560a1d..31d31bd43f453 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.execution.datasources.PartitionedFile import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration @@ -43,7 +44,8 @@ case class CSVPartitionReaderFactory( dataSchema: StructType, readDataSchema: StructType, partitionSchema: StructType, - parsedOptions: CSVOptions) extends FilePartitionReaderFactory { + parsedOptions: CSVOptions, + filters: Seq[Filter]) extends FilePartitionReaderFactory { private val columnPruning = sqlConf.csvColumnPruning override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { @@ -55,7 +57,8 @@ case class CSVPartitionReaderFactory( val parser = new UnivocityParser( actualDataSchema, actualReadDataSchema, - parsedOptions) + parsedOptions, + filters) val schema = if (columnPruning) actualReadDataSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 5125de9313a4c..41d43335d55ac 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.connector.read.PartitionReaderFactory import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.csv.CSVDataSource import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration @@ -37,7 +38,8 @@ case class CSVScan( dataSchema: StructType, readDataSchema: StructType, readPartitionSchema: StructType, - options: CaseInsensitiveStringMap) + options: CaseInsensitiveStringMap, + filters: Seq[Filter]) extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { private lazy val parsedOptions: CSVOptions = new CSVOptions( @@ -85,6 +87,6 @@ case class CSVScan( // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. CSVPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + dataSchema, readDataSchema, readPartitionSchema, parsedOptions, filters) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 8b486d0344506..1839f8f473527 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.connector.read.Scan +import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter import org.apache.spark.sql.types.StructType import org.apache.spark.sql.util.CaseInsensitiveStringMap @@ -30,9 +31,25 @@ case class CSVScanBuilder( schema: StructType, dataSchema: StructType, options: CaseInsensitiveStringMap) - extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { override def build(): Scan = { - CSVScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + CSVScan( + sparkSession, + fileIndex, + dataSchema, + readDataSchema(), + readPartitionSchema(), + options, + pushedFilters()) } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + _pushedFilters = filters + Array.empty + } + + override def pushedFilters(): Array[Filter] = _pushedFilters } From 20dbef04275bbc85fe0dd29c240e4805ddf1da20 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 17 Dec 2019 13:05:45 +0300 Subject: [PATCH 07/43] Add a test to CSVSuite --- .../execution/datasources/csv/CSVSuite.scala | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f6adc7acb2772..c6af4fa0ef6a8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2204,4 +2204,23 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { checkAnswer(resultDF, Row("a", 2, "e", "c")) } } + + test("filters push down") { + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + "c1,c2", + "1,2019-11-14 20:35:30", + s"2,$t").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + val readback = spark.read + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .schema("c1 integer, c2 timestamp") + .csv(path.getAbsolutePath) + .where($"c1" === 2) + checkAnswer(readback, Row(2, Timestamp.valueOf(t))) + } + } } From becfe1e02f2ce263724ccb5c295ca5b0245905ca Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 00:30:45 +0300 Subject: [PATCH 08/43] Keep only one predicate per field --- .../spark/sql/catalyst/csv/CSVFilters.scala | 20 ++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 122c43904e149..1f019a0ed84f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -85,19 +85,25 @@ class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { translate(filter) } - private val predicates: Array[Seq[BasePredicate]] = { - val parr = Array.fill(schema.fields.length)(Seq.empty[BasePredicate]) + private val predicates: Array[BasePredicate] = { + val len = schema.fields.length + val groupedExprs = Array.fill(len)(Seq.empty[Expression]) for (filter <- filters) { val index = filter.references.map(schema.fieldIndex).max - for (expr <- filterToExpression(filter)) { - val predicate = Predicate.create(expr) - parr(index) :+= predicate + groupedExprs(index) ++= filterToExpression(filter) + } + val groupedPredicates = Array.fill[BasePredicate](len)(null) + for (i <- 0 until len) { + if (!groupedExprs(i).isEmpty) { + val reducedExpr = groupedExprs(i).reduce(And) + groupedPredicates(i) = Predicate.create(reducedExpr) } } - parr + groupedPredicates } def skipRow(row: InternalRow, index: Int): Boolean = { - predicates(index).exists(!_.eval(row)) + val predicate = predicates(index) + predicate != null && !predicate.eval(row) } } From 77e7d5468327dc1ce7bb5bd1dcb8cccad7a67784 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 13:01:57 +0300 Subject: [PATCH 09/43] Add a benchmark --- .../datasources/csv/CSVBenchmark.scala | 35 +++++++++++++++++++ 1 file changed, 35 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index a4cffedaf82d7..2e80ea22e6843 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -298,6 +298,40 @@ object CSVBenchmark extends SqlBasedBenchmark { } } + private def filtersPushdownBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark(s"Filters pushdown", rowsNum, output = output) + val colsNum = 100 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", TimestampType)) + val schema = StructType(StructField("key", IntegerType) +: fields) + def columns(): Seq[Column] = { + val ts = Seq.tabulate(colsNum) { i => + lit(Instant.ofEpochSecond(i * 12345678)).as(s"col$i") + } + ($"id" % 1000).as("key") +: ts + } + withTempPath { path => + spark.range(rowsNum).select(columns(): _*) + .write.option("header", true) + .csv(path.getAbsolutePath) + def readback = { + spark.read + .option("header", true) + .schema(schema) + .csv(path.getAbsolutePath) + } + + benchmark.addCase(s"w/o filters", numIters) { _ => + toNoop(readback) + } + + benchmark.addCase(s"w/ filters", numIters) { _ => + toNoop(readback.filter($"key" === 0)) + } + + benchmark.run() + } + } + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Benchmark to measure CSV read/write performance") { val numIters = 3 @@ -305,6 +339,7 @@ object CSVBenchmark extends SqlBasedBenchmark { multiColumnsBenchmark(rowsNum = 1000 * 1000, numIters) countBenchmark(rowsNum = 10 * 1000 * 1000, numIters) datetimeBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + filtersPushdownBenchmark(rowsNum = 100 * 1000, numIters) } } } From 415e4ce690f6429a8b8b611db37b5b67fd76d202 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 15:07:18 +0300 Subject: [PATCH 10/43] SQL config `spark.sql.csv.filterPushdown.enabled` --- .../spark/sql/catalyst/csv/CSVFilters.scala | 21 ++++++------ .../apache/spark/sql/internal/SQLConf.scala | 5 +++ .../execution/datasources/csv/CSVSuite.scala | 32 ++++++++++--------- 3 files changed, 34 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 1f019a0ed84f1..cd27d16f41718 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -21,6 +21,7 @@ import scala.util.Try import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType @@ -87,16 +88,18 @@ class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { private val predicates: Array[BasePredicate] = { val len = schema.fields.length - val groupedExprs = Array.fill(len)(Seq.empty[Expression]) - for (filter <- filters) { - val index = filter.references.map(schema.fieldIndex).max - groupedExprs(index) ++= filterToExpression(filter) - } val groupedPredicates = Array.fill[BasePredicate](len)(null) - for (i <- 0 until len) { - if (!groupedExprs(i).isEmpty) { - val reducedExpr = groupedExprs(i).reduce(And) - groupedPredicates(i) = Predicate.create(reducedExpr) + if (SQLConf.get.getConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED)) { + val groupedExprs = Array.fill(len)(Seq.empty[Expression]) + for (filter <- filters) { + val index = filter.references.map(schema.fieldIndex).max + groupedExprs(index) ++= filterToExpression(filter) + } + for (i <- 0 until len) { + if (!groupedExprs(i).isEmpty) { + val reducedExpr = groupedExprs(i).reduce(And) + groupedPredicates(i) = Predicate.create(reducedExpr) + } } } groupedPredicates diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 91347cfe80366..90a6f9bbd8cce 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2105,6 +2105,11 @@ object SQLConf { "defined by `from` and `to`.") .booleanConf .createWithDefault(false) + + val CSV_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.csv.filterPushdown.enabled") + .doc("When true, enable filter pushdown to CSV datasource.") + .booleanConf + .createWithDefault(true) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index c6af4fa0ef6a8..f3b1e6ba15418 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2206,21 +2206,23 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } test("filters push down") { - withTempPath { path => - val t = "2019-12-17 00:01:02" - Seq( - "c1,c2", - "1,2019-11-14 20:35:30", - s"2,$t").toDF("data") - .repartition(1) - .write.text(path.getAbsolutePath) - val readback = spark.read - .option("header", true) - .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") - .schema("c1 integer, c2 timestamp") - .csv(path.getAbsolutePath) - .where($"c1" === 2) - checkAnswer(readback, Row(2, Timestamp.valueOf(t))) + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> "true") { + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + "c1,c2", + "1,2019-11-14 20:35:30", + s"2,$t").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + val readback = spark.read + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .schema("c1 integer, c2 timestamp") + .csv(path.getAbsolutePath) + .where($"c1" === 2) + checkAnswer(readback, Row(2, Timestamp.valueOf(t))) + } } } } From 3db517f9c7013512dbab462970c29c11ff25ed0b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 15:15:24 +0300 Subject: [PATCH 11/43] Use SQL config in CSVBenchmark --- .../execution/datasources/csv/CSVBenchmark.scala | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index 2e80ea22e6843..42a864f115602 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.{Column, Dataset, Row} import org.apache.spark.sql.SaveMode.Overwrite import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -324,8 +325,18 @@ object CSVBenchmark extends SqlBasedBenchmark { toNoop(readback) } + def withFilter(configEnabled: Boolean): Unit = { + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { + toNoop(readback.filter($"key" === 0)) + } + } + + benchmark.addCase(s"pushdown disabled", numIters) { _ => + withFilter(configEnabled = false) + } + benchmark.addCase(s"w/ filters", numIters) { _ => - toNoop(readback.filter($"key" === 0)) + withFilter(configEnabled = true) } benchmark.run() From 98963bcc526522c1d1424424fd592ecb98dd1803 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 15:28:22 +0300 Subject: [PATCH 12/43] Refactoring --- .../spark/sql/catalyst/csv/CSVFilters.scala | 66 +++++++++---------- 1 file changed, 30 insertions(+), 36 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index cd27d16f41718..4362052b960de 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -27,24 +27,42 @@ import org.apache.spark.sql.types.StructType class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { - private def toRef(attr: String): Option[BoundReference] = { - schema.getFieldIndex(attr).map { index => - val field = schema(index) - BoundReference(schema.fieldIndex(attr), field.dataType, field.nullable) + private val predicates: Array[BasePredicate] = { + val len = schema.fields.length + val groupedPredicates = Array.fill[BasePredicate](len)(null) + if (SQLConf.get.getConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED)) { + val groupedExprs = Array.fill(len)(Seq.empty[Expression]) + for (filter <- filters) { + val index = filter.references.map(schema.fieldIndex).max + groupedExprs(index) ++= filterToExpression(filter) + } + for (i <- 0 until len) { + if (!groupedExprs(i).isEmpty) { + val reducedExpr = groupedExprs(i).reduce(And) + groupedPredicates(i) = Predicate.create(reducedExpr) + } + } } + groupedPredicates } - private def toLiteral(value: Any): Option[Literal] = { - Try(Literal(value)).toOption - } - - private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = a.zip(b).headOption - - private def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { - zip(toRef(name), toLiteral(value)) + def skipRow(row: InternalRow, index: Int): Boolean = { + val predicate = predicates(index) + predicate != null && !predicate.eval(row) } private def filterToExpression(filter: sources.Filter): Option[Expression] = { + def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = a.zip(b).headOption + def toLiteral(value: Any): Option[Literal] = Try(Literal(value)).toOption + def toRef(attr: String): Option[BoundReference] = { + schema.getFieldIndex(attr).map { index => + val field = schema(index) + BoundReference(schema.fieldIndex(attr), field.dataType, field.nullable) + } + } + def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { + zip(toRef(name), toLiteral(value)) + } def translate(filter: sources.Filter): Option[Expression] = filter match { case sources.And(left, right) => zip(translate(left), translate(right)).map(And.tupled) @@ -85,28 +103,4 @@ class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { } translate(filter) } - - private val predicates: Array[BasePredicate] = { - val len = schema.fields.length - val groupedPredicates = Array.fill[BasePredicate](len)(null) - if (SQLConf.get.getConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED)) { - val groupedExprs = Array.fill(len)(Seq.empty[Expression]) - for (filter <- filters) { - val index = filter.references.map(schema.fieldIndex).max - groupedExprs(index) ++= filterToExpression(filter) - } - for (i <- 0 until len) { - if (!groupedExprs(i).isEmpty) { - val reducedExpr = groupedExprs(i).reduce(And) - groupedPredicates(i) = Predicate.create(reducedExpr) - } - } - } - groupedPredicates - } - - def skipRow(row: InternalRow, index: Int): Boolean = { - val predicate = predicates(index) - predicate != null && !predicate.eval(row) - } } From 05111a5611c03cebfd6c28da5dd6f0ff83d7d08e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 16:18:18 +0300 Subject: [PATCH 13/43] Add comments for skipRow --- .../org/apache/spark/sql/catalyst/csv/CSVFilters.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 4362052b960de..e5f4b1701ae50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -46,6 +46,15 @@ class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { groupedPredicates } + /** + * Apply all filters that refer to row fields at the positions from 0 to index. + * @param row The internal row to check. + * @param index Maximum field index. The function assumes that all fields + * from 0 to index position are set. + * @return false iff row fields at the position from 0 to index pass filters + * or there are no applicable filters + * otherwise false if at least one of the filters returns false. + */ def skipRow(row: InternalRow, index: Int): Boolean = { val predicate = predicates(index) predicate != null && !predicate.eval(row) From 899cf176408b33f178e7b147acd6e7f217c794f7 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 21:34:17 +0300 Subject: [PATCH 14/43] Apply filters only on CSV level --- .../spark/sql/catalyst/csv/CSVFilters.scala | 66 +++++++++++++++---- .../sql/catalyst/csv/UnivocityParser.scala | 64 +++++++++++------- .../apache/spark/sql/internal/SQLConf.scala | 2 + .../datasources/csv/CSVFileFormat.scala | 5 +- .../v2/csv/CSVPartitionReaderFactory.scala | 5 +- .../datasources/v2/csv/CSVScanBuilder.scala | 3 +- .../execution/datasources/csv/CSVSuite.scala | 3 + 7 files changed, 104 insertions(+), 44 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index e5f4b1701ae50..92dcc088d81ec 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -25,16 +25,29 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType -class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { +class CSVFilters( + filters: Seq[sources.Filter], + dataSchema: StructType, + requiredSchema: StructType) { + require(checkFilters(), "All filters must be applicable to dataSchema") + + /** The schema to read from the underlying CSV parser */ + val readSchema: StructType = { + val refs = filters.flatMap(_.references).toSet + val readFields = dataSchema.filter { field => + requiredSchema.contains(field) || refs.contains(field.name) + } + StructType(readFields) + } private val predicates: Array[BasePredicate] = { - val len = schema.fields.length + val len = readSchema.fields.length val groupedPredicates = Array.fill[BasePredicate](len)(null) - if (SQLConf.get.getConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED)) { + if (SQLConf.get.csvFilterPushDown) { val groupedExprs = Array.fill(len)(Seq.empty[Expression]) for (filter <- filters) { - val index = filter.references.map(schema.fieldIndex).max - groupedExprs(index) ++= filterToExpression(filter) + val index = filter.references.map(readSchema.fieldIndex).max + groupedExprs(index) ++= CSVFilters.filterToExpression(filter, toRef) } for (i <- 0 until len) { if (!groupedExprs(i).isEmpty) { @@ -60,15 +73,42 @@ class CSVFilters(filters: Seq[sources.Filter], schema: StructType) { predicate != null && !predicate.eval(row) } - private def filterToExpression(filter: sources.Filter): Option[Expression] = { - def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = a.zip(b).headOption - def toLiteral(value: Any): Option[Literal] = Try(Literal(value)).toOption - def toRef(attr: String): Option[BoundReference] = { - schema.getFieldIndex(attr).map { index => - val field = schema(index) - BoundReference(schema.fieldIndex(attr), field.dataType, field.nullable) - } + // Finds a filter attribute in the read schema and converts it to a `BoundReference` + private def toRef(attr: String): Option[BoundReference] = { + readSchema.getFieldIndex(attr).map { index => + val field = readSchema(index) + BoundReference(readSchema.fieldIndex(attr), field.dataType, field.nullable) } + } + + // Checks that all filters refer to an field in the data schema + private def checkFilters(): Boolean = { + val refs = filters.flatMap(_.references).toSet + val fieldNames = dataSchema.fields.map(_.name).toSet + refs.forall(fieldNames.contains(_)) + } +} + +object CSVFilters { + + def unsupportedFilters(filters: Array[sources.Filter]): Array[sources.Filter] = { + filters.filter { + case sources.AlwaysFalse | sources.AlwaysTrue => true + case _ => !SQLConf.get.csvFilterPushDown + } + } + + private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { + a.zip(b).headOption + } + + private def toLiteral(value: Any): Option[Literal] = { + Try(Literal(value)).toOption + } + + def filterToExpression( + filter: sources.Filter, + toRef: String => Option[BoundReference]): Option[Expression] = { def zipAttributeAndValue(name: String, value: Any): Option[(BoundReference, Literal)] = { zip(toRef(name), toLiteral(value)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 9f00460ef80e3..590910365dada 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -58,13 +58,28 @@ class UnivocityParser( // A `ValueConverter` is responsible for converting the given value to a desired type. private type ValueConverter = String => Any + private val csvFilters = new CSVFilters(filters, dataSchema, requiredSchema) + + val readSchema = csvFilters.readSchema + // This index is used to reorder parsed tokens private val tokenIndexArr = - requiredSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray + readSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray + + private val readToRequiredIndex: Array[Int] = { + val arr = Array.fill(readSchema.length)(-1) + for { + readIndex <- 0 until readSchema.length + reqIndex <- requiredSchema.getFieldIndex(readSchema(readIndex).name) + } { + arr(readIndex) = reqIndex + } + arr + } // When column pruning is enabled, the parser only parses the required columns based on // their positions in the data schema. - private val parsedSchema = if (options.columnPruning) requiredSchema else dataSchema + private val parsedSchema = if (options.columnPruning) readSchema else dataSchema val tokenizer = { val parserSetting = options.asParserSettings @@ -77,7 +92,8 @@ class UnivocityParser( new CsvParser(parserSetting) } - private val singleRow = Seq(new GenericInternalRow(requiredSchema.length)) + private val readRow = new GenericInternalRow(readSchema.length) + private val requiredRow = Seq(new GenericInternalRow(requiredSchema.length)) private val noRows = Seq.empty[InternalRow] private val timestampFormatter = TimestampFormatter( @@ -89,8 +105,6 @@ class UnivocityParser( options.zoneId, options.locale) - private val csvFilters = new CSVFilters(filters, requiredSchema) - // Retrieve the raw record string. private def getCurrentInput: UTF8String = { UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) @@ -116,7 +130,7 @@ class UnivocityParser( // // output row - ["A", 2] private val valueConverters: Array[ValueConverter] = { - requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray + readSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray } private val decimalParser = ExprUtils.getDecimalParser(options.locale) @@ -199,7 +213,7 @@ class UnivocityParser( } } - private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { + private val doParse = if (options.columnPruning && readSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. (_: String) => Seq(InternalRow.empty) @@ -252,37 +266,43 @@ class UnivocityParser( // When the length of the returned tokens is identical to the length of the parsed schema, // we just need to convert the tokens that correspond to the required columns. var i = 0 - val r = singleRow.head var skipValueConversion = false var badRecordException: Option[Throwable] = None + val requiredSingleRow = requiredRow.head while (i < requiredSchema.length) { - if (skipValueConversion) { - r.setNullAt(i) - } else { - try { - r(i) = valueConverters(i).apply(getToken(tokens, i)) - if (csvFilters.skipRow(r, i)) { - skipValueConversion = true + requiredSingleRow.setNullAt(i) + i += 1 + } + i = 0 + while (!skipValueConversion && i < readSchema.length) { + try { + val convertedValue = valueConverters(i).apply(getToken(tokens, i)) + readRow(i) = convertedValue + if (csvFilters.skipRow(readRow, i)) { + skipValueConversion = true + } else { + val requiredIndex = readToRequiredIndex(i) + if (requiredIndex != -1) { + requiredSingleRow(requiredIndex) = convertedValue } - } catch { - case NonFatal(e) => - badRecordException = Some(e) - skipValueConversion = true - r.setNullAt(i) } + } catch { + case NonFatal(e) => + badRecordException = Some(e) + skipValueConversion = true } i += 1 } if (skipValueConversion) { if (badRecordException.isDefined) { throw BadRecordException( - () => getCurrentInput, () => singleRow.headOption, badRecordException.get) + () => getCurrentInput, () => requiredRow.headOption, badRecordException.get) } else { noRows } } else { - singleRow + requiredRow } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 90a6f9bbd8cce..54822cbfe724a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2613,6 +2613,8 @@ class SQLConf extends Serializable with Logging { def ignoreDataLocality: Boolean = getConf(SQLConf.IGNORE_DATA_LOCALITY) + def csvFilterPushDown: Boolean = getConf(CSV_FILTER_PUSHDOWN_ENABLED) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 00d90f19ddb7f..45391c6e71110 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -126,8 +126,6 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { "df.filter($\"_corrupt_record\".isNotNull).count()." ) } - val columnPruning = sparkSession.sessionState.conf.csvColumnPruning - (file: PartitionedFile) => { val conf = broadcastedHadoopConf.value.value val actualDataSchema = StructType( @@ -139,10 +137,9 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { actualRequiredSchema, parsedOptions, filters) - val schema = if (columnPruning) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( - schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + parser.readSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) CSVDataSource(parsedOptions).readFile( conf, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 31d31bd43f453..d2b5482f084b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -46,8 +46,6 @@ case class CSVPartitionReaderFactory( partitionSchema: StructType, parsedOptions: CSVOptions, filters: Seq[Filter]) extends FilePartitionReaderFactory { - private val columnPruning = sqlConf.csvColumnPruning - override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value val actualDataSchema = StructType( @@ -59,10 +57,9 @@ case class CSVPartitionReaderFactory( actualReadDataSchema, parsedOptions, filters) - val schema = if (columnPruning) actualReadDataSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( - schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + parser.readSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) val iter = CSVDataSource(parsedOptions).readFile( conf, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index 1839f8f473527..ae2c1f4036c3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.datasources.v2.csv import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.csv.CSVFilters import org.apache.spark.sql.connector.read.{Scan, SupportsPushDownFilters} import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder @@ -48,7 +49,7 @@ case class CSVScanBuilder( override def pushFilters(filters: Array[Filter]): Array[Filter] = { _pushedFilters = filters - Array.empty + CSVFilters.unsupportedFilters(filters) } override def pushedFilters(): Array[Filter] = _pushedFilters diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f3b1e6ba15418..54b514cd6ba49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2221,6 +2221,9 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { .schema("c1 integer, c2 timestamp") .csv(path.getAbsolutePath) .where($"c1" === 2) + // count() pushes empty schema. This checks handling of a filter + // which refers to not existed field. + assert(readback.count() === 1) checkAnswer(readback, Row(2, Timestamp.valueOf(t))) } } From d08fe58b012ec376b605dd001919224fb4f0bb4b Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 23:02:56 +0300 Subject: [PATCH 15/43] Add a comment for `predicates` --- .../apache/spark/sql/catalyst/csv/CSVFilters.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 92dcc088d81ec..7018e73402edb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -40,6 +40,18 @@ class CSVFilters( StructType(readFields) } + /** + * Converted filters to predicates and grouped by maximum field index + * in the read schema. For example, if an filter refers to 2 attributes + * attrA with field index 5 and attrB with field index 10 in the read schema: + * 0 === $"attrA" or $"attrB" < 100 + * the filter is compiled to a predicate, and placed to the `predicates` + * array at the position 10. In this way, if there is a row with initialized + * fields from the 0 to 10 index, the predicate can be applied to the row + * to check that the row should be skipped or not. + * Multiple predicates with the same maximum reference index are combined + * by the `And` expression. + */ private val predicates: Array[BasePredicate] = { val len = readSchema.fields.length val groupedPredicates = Array.fill[BasePredicate](len)(null) @@ -60,7 +72,7 @@ class CSVFilters( } /** - * Apply all filters that refer to row fields at the positions from 0 to index. + * Applies all filters that refer to row fields at the positions from 0 to index. * @param row The internal row to check. * @param index Maximum field index. The function assumes that all fields * from 0 to index position are set. From b0a34b37996381fc2272e7e2617fc38a90d35c13 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 23:09:31 +0300 Subject: [PATCH 16/43] Add a comment for CSVFilters --- .../org/apache/spark/sql/catalyst/csv/CSVFilters.scala | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 7018e73402edb..da8b99b01835d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -25,6 +25,15 @@ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources import org.apache.spark.sql.types.StructType +/** + * An instance of the class compiles filters to predicates and allows to + * apply the predicates to an internal row with partially initialized values + * converted from parsed CSV fields. + * + * @param filters The filters pushed down to CSV datasource. + * @param dataSchema The full schema with all fields in CSV files. + * @param requiredSchema The schema with only fields requested by the upper layer. + */ class CSVFilters( filters: Seq[sources.Filter], dataSchema: StructType, From 5fe560077cd0b4e440cf0537f45a32c4a41f11d0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 18 Dec 2019 23:22:54 +0300 Subject: [PATCH 17/43] Add a comment for `unsupportedFilters` --- .../apache/spark/sql/catalyst/csv/CSVFilters.scala | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index da8b99b01835d..8363b6de77a41 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -111,7 +111,11 @@ class CSVFilters( } object CSVFilters { - + /** + * Returns the filters currently not supported by CSV datasource. + * @param filters The filters pushed down to CSV datasource. + * @return a sub-set of `filters` that cannot be handled by CSV datasource. + */ def unsupportedFilters(filters: Array[sources.Filter]): Array[sources.Filter] = { filters.filter { case sources.AlwaysFalse | sources.AlwaysTrue => true @@ -127,6 +131,14 @@ object CSVFilters { Try(Literal(value)).toOption } + /** + * Converts a filter to an expression and binds it to row positions. + * + * @param filter The filter to convert. + * @param toRef The function converts a filter attribute to a bound reference. + * @return some expression with resolved attributes or None if the conversion + * of the given filter to an expression is impossible. + */ def filterToExpression( filter: sources.Filter, toRef: String => Option[BoundReference]): Option[Expression] = { From a7f30065fd3e0f4523d9c0d3487acce01f6d1bc6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Dec 2019 13:04:37 +0300 Subject: [PATCH 18/43] Add comments --- .../spark/sql/catalyst/csv/CSVFilters.scala | 9 ++++-- .../sql/catalyst/csv/UnivocityParser.scala | 29 ++++++++++++++----- 2 files changed, 28 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 8363b6de77a41..73ba43b62fb3d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -38,9 +38,12 @@ class CSVFilters( filters: Seq[sources.Filter], dataSchema: StructType, requiredSchema: StructType) { - require(checkFilters(), "All filters must be applicable to dataSchema") + assert(checkFilters(), "All filters must be applicable to dataSchema") - /** The schema to read from the underlying CSV parser */ + /** + * The schema to read from the underlying CSV parser. + * It combines the required schema and the fields referenced by filters. + */ val readSchema: StructType = { val refs = filters.flatMap(_.references).toSet val readFields = dataSchema.filter { field => @@ -67,6 +70,8 @@ class CSVFilters( if (SQLConf.get.csvFilterPushDown) { val groupedExprs = Array.fill(len)(Seq.empty[Expression]) for (filter <- filters) { + // readSchema must contain attributes of all filters. + // Accordingly, fieldIndex() returns a valid index always. val index = filter.references.map(readSchema.fieldIndex).max groupedExprs(index) ++= CSVFilters.filterToExpression(filter, toRef) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 590910365dada..a4abac0b1b694 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -40,6 +40,7 @@ import org.apache.spark.unsafe.types.UTF8String * @param requiredSchema The schema of the data that should be output for each row. This should be a * subset of the columns in dataSchema. * @param options Configuration options for a CSV parser. + * @param filters The pushdown filters that should be applied to converted values. */ class UnivocityParser( dataSchema: StructType, @@ -60,12 +61,12 @@ class UnivocityParser( private val csvFilters = new CSVFilters(filters, dataSchema, requiredSchema) + // The "minimal" schema to be read from Univocity parser. + // It includes `requiredSchema` + the fields referenced by pushed down filters. val readSchema = csvFilters.readSchema - // This index is used to reorder parsed tokens - private val tokenIndexArr = - readSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray - + // Mapping of field indexes of `readSchema` to indexes of `requiredSchema`. + // It returns -1 if `requiredSchema` doesn't contain a field from `readSchema`. private val readToRequiredIndex: Array[Int] = { val arr = Array.fill(readSchema.length)(-1) for { @@ -77,6 +78,10 @@ class UnivocityParser( arr } + // This index is used to reorder parsed tokens + private val tokenIndexArr = + readSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray + // When column pruning is enabled, the parser only parses the required columns based on // their positions in the data schema. private val parsedSchema = if (options.columnPruning) readSchema else dataSchema @@ -92,8 +97,13 @@ class UnivocityParser( new CsvParser(parserSetting) } + // The row is used as a temporary placeholder of parsed and converted values. + // It is needed for applying the pushdown filters. private val readRow = new GenericInternalRow(readSchema.length) + // Pre-allocated Seq to avoid the overhead of the seq builder. private val requiredRow = Seq(new GenericInternalRow(requiredSchema.length)) + // Pre-allocated empty sequence returned when the parsed row cannot pass filters. + // We preallocate it avoid unnecessary invokes of the seq builder. private val noRows = Seq.empty[InternalRow] private val timestampFormatter = TimestampFormatter( @@ -264,16 +274,19 @@ class UnivocityParser( new RuntimeException("Malformed CSV record")) } else { // When the length of the returned tokens is identical to the length of the parsed schema, - // we just need to convert the tokens that correspond to the required columns. + // we just need to: + // 1. Convert the tokens that correspond to the read schema. + // 2. Apply the pushdown filters to `readRow`. + // 3. Convert `readRow` to `requiredRow` by stripping non-required fields. var i = 0 - var skipValueConversion = false - var badRecordException: Option[Throwable] = None val requiredSingleRow = requiredRow.head - while (i < requiredSchema.length) { requiredSingleRow.setNullAt(i) i += 1 } + + var skipValueConversion = false + var badRecordException: Option[Throwable] = None i = 0 while (!skipValueConversion && i < readSchema.length) { try { From c989bee64ee37bd3f6203845a4fb87738a1ed496 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Dec 2019 15:26:45 +0300 Subject: [PATCH 19/43] Add tests to UnivocityParserSuite --- .../spark/sql/catalyst/csv/CSVFilters.scala | 2 +- .../catalyst/csv/UnivocityParserSuite.scala | 64 +++++++++++++++++++ 2 files changed, 65 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 73ba43b62fb3d..9934ad408c9fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -38,7 +38,7 @@ class CSVFilters( filters: Seq[sources.Filter], dataSchema: StructType, requiredSchema: StructType) { - assert(checkFilters(), "All filters must be applicable to dataSchema") + require(checkFilters(), "All filters must be applicable to the data schema.") /** * The schema to read from the underlying CSV parser. diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 31601f787f1a9..ba2aefa9d3d85 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -24,9 +24,11 @@ import java.util.{Locale, TimeZone} import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.sources.{EqualTo, Filter, LessThan, StringStartsWith} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -267,4 +269,66 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { assert(convertedValue.isInstanceOf[UTF8String]) assert(convertedValue == expected) } + + test("skipping rows using pushdown filters") { + def check( + input: String = "1,a", + dataSchema: String = "i INTEGER, s STRING", + requiredSchema: String = "i INTEGER", + filters: Seq[Filter], + expected: Seq[InternalRow]): Unit = { + def getSchema(str: String): StructType = str match { + case "" => new StructType() + case _ => StructType.fromDDL(str) + } + Seq(false, true).foreach { columnPruning => + val options = new CSVOptions(Map.empty[String, String], columnPruning, "GMT") + val parser = new UnivocityParser( + getSchema(dataSchema), + getSchema(requiredSchema), + options, + filters) + val actual = parser.parse(input) + assert(actual === expected) + } + } + + check(filters = Seq(), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) + check(filters = Seq(EqualTo("i", 2)), expected = Seq()) + check(filters = Seq(StringStartsWith("s", "b")), expected = Seq()) + check( + requiredSchema = "i INTEGER, s STRING", + filters = Seq(StringStartsWith("s", "a")), + expected = Seq(InternalRow(1, UTF8String.fromString("a")))) + check( + requiredSchema = "", + filters = Seq(LessThan("i", 10)), + expected = Seq(InternalRow.empty)) + check( + input = "1,a,3.14", + dataSchema = "i INTEGER, s STRING, d DOUBLE", + filters = Seq(EqualTo("d", 3.14)), + expected = Seq(InternalRow(1))) + + try { + check(filters = Seq(EqualTo("invalid attr", 1)), expected = Seq()) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("All filters must be applicable to the data schema")) + } + + try { + check( + dataSchema = "", + requiredSchema = "", + filters = Seq(EqualTo("i", 1)), + expected = Seq(InternalRow.empty)) + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("All filters must be applicable to the data schema")) + } + } } From 124c45ddafc2a5d231d50f906dd5745356d96faa Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Dec 2019 19:42:57 +0300 Subject: [PATCH 20/43] Support AlwaysTrue and AlwaysFalse filters --- .../apache/spark/sql/catalyst/csv/CSVFilters.scala | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 9934ad408c9fd..c6bbf8c53605b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types.{BooleanType, StructType} /** * An instance of the class compiles filters to predicates and allows to @@ -122,9 +122,10 @@ object CSVFilters { * @return a sub-set of `filters` that cannot be handled by CSV datasource. */ def unsupportedFilters(filters: Array[sources.Filter]): Array[sources.Filter] = { - filters.filter { - case sources.AlwaysFalse | sources.AlwaysTrue => true - case _ => !SQLConf.get.csvFilterPushDown + if (SQLConf.get.csvFilterPushDown) { + Array.empty + } else { + filters } } @@ -186,7 +187,10 @@ object CSVFilters { zipAttributeAndValue(attribute, value).map(StartsWith.tupled) case sources.StringEndsWith(attribute, value) => zipAttributeAndValue(attribute, value).map(EndsWith.tupled) - case _ => None + case sources.AlwaysTrue() => + Some(Literal(true, BooleanType)) + case sources.AlwaysFalse() => + Some(Literal(false, BooleanType)) } translate(filter) } From d7932c2a7aaad6c5ae9cb77ef6b47733b9c42c6f Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Dec 2019 22:17:00 +0300 Subject: [PATCH 21/43] Add tests for filterToExpression() --- .../sql/catalyst/csv/CSVFiltersSuite.scala | 51 +++++++++++++++++++ 1 file changed, 51 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala new file mode 100644 index 0000000000000..5f8a66251971d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.csv + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types.IntegerType + +class CSVFiltersSuite extends SparkFunSuite { + test("filter to expression conversion") { + val ref = BoundReference(0, IntegerType, true) + def check(f: Filter, expr: Expression): Unit = { + assert(CSVFilters.filterToExpression(f, _ => Some(ref)).get === expr) + } + + check(sources.AlwaysTrue, Literal(true)) + check(sources.AlwaysFalse, Literal(false)) + check(sources.IsNull("a"), IsNull(ref)) + check(sources.Not(sources.IsNull("a")), Not(IsNull(ref))) + check(sources.IsNotNull("a"), IsNotNull(ref)) + check(sources.EqualTo("a", "b"), EqualTo(ref, Literal("b"))) + check(sources.EqualNullSafe("a", "b"), EqualNullSafe(ref, Literal("b"))) + check(sources.StringStartsWith("a", "b"), StartsWith(ref, Literal("b"))) + check(sources.StringEndsWith("a", "b"), EndsWith(ref, Literal("b"))) + check(sources.StringContains("a", "b"), Contains(ref, Literal("b"))) + check(sources.LessThanOrEqual("a", 1), LessThanOrEqual(ref, Literal(1))) + check(sources.LessThan("a", 1), LessThan(ref, Literal(1))) + check(sources.GreaterThanOrEqual("a", 1), GreaterThanOrEqual(ref, Literal(1))) + check(sources.GreaterThan("a", 1), GreaterThan(ref, Literal(1))) + check(sources.And(sources.AlwaysTrue, sources.AlwaysTrue), And(Literal(true), Literal(true))) + check(sources.Or(sources.AlwaysTrue, sources.AlwaysTrue), Or(Literal(true), Literal(true))) + check(sources.In("a", Array(1)), In(ref, Seq(Literal(1)))) + } +} From bb0abf4c4a58f6bab77723af07a6ddd597908b99 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 19 Dec 2019 22:33:34 +0300 Subject: [PATCH 22/43] Add tests for readSchema --- .../sql/catalyst/csv/CSVFiltersSuite.scala | 37 ++++++++++++++++++- 1 file changed, 36 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index 5f8a66251971d..cbb0a4d739f61 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.sources import org.apache.spark.sql.sources.Filter -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{IntegerType, StructType} class CSVFiltersSuite extends SparkFunSuite { test("filter to expression conversion") { @@ -48,4 +48,39 @@ class CSVFiltersSuite extends SparkFunSuite { check(sources.Or(sources.AlwaysTrue, sources.AlwaysTrue), Or(Literal(true), Literal(true))) check(sources.In("a", Array(1)), In(ref, Seq(Literal(1)))) } + + private def getSchema(str: String): StructType = str match { + case "" => new StructType() + case _ => StructType.fromDDL(str) + } + + test("read schema is based on required schema and filters") { + def check( + dataSchema: String = "i INTEGER, d DOUBLE, s STRING", + requiredSchema: String = "s STRING", + filters: Seq[sources.Filter], + expected: String): Unit = { + val csvFilters = new CSVFilters(filters, getSchema(dataSchema), getSchema(requiredSchema)) + assert(csvFilters.readSchema === getSchema(expected)) + } + + check(filters = Seq(), expected = "s STRING") + check(filters = Seq(sources.EqualTo("d", 3.14)), expected = "d DOUBLE, s STRING") + check( + filters = Seq(sources.And(sources.EqualTo("d", 3.14), sources.StringEndsWith("s", "a"))), + expected = "d DOUBLE, s STRING") + check( + filters = Seq( + sources.And(sources.EqualTo("d", 3.14), sources.StringEndsWith("s", "a")), + sources.GreaterThan("i", 100)), + expected = "i INTEGER, d DOUBLE, s STRING") + + try { + check(filters = Seq(sources.EqualTo("invalid", 3.14)), expected = "d DOUBLE, s STRING") + fail("Expected to throw an exception for the invalid input") + } catch { + case e: IllegalArgumentException => + assert(e.getMessage.contains("All filters must be applicable to the data schema")) + } + } } From 1c707e5b29a224ee060f1e54ad40bac90aa83037 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Dec 2019 00:19:13 +0300 Subject: [PATCH 23/43] Add tests for skipRow() --- .../spark/sql/catalyst/csv/CSVFilters.scala | 17 +++- .../sql/catalyst/csv/CSVFiltersSuite.scala | 77 ++++++++++++++++++- 2 files changed, 88 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index c6bbf8c53605b..8f8f86b403635 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -70,10 +70,19 @@ class CSVFilters( if (SQLConf.get.csvFilterPushDown) { val groupedExprs = Array.fill(len)(Seq.empty[Expression]) for (filter <- filters) { - // readSchema must contain attributes of all filters. - // Accordingly, fieldIndex() returns a valid index always. - val index = filter.references.map(readSchema.fieldIndex).max - groupedExprs(index) ++= CSVFilters.filterToExpression(filter, toRef) + val expr = CSVFilters.filterToExpression(filter, toRef) + val refs = filter.references + if (refs.isEmpty) { + // For example, AlwaysTrue and AlwaysFalse doesn't have any references + for (i <- 0 until len) { + groupedExprs(i) ++= expr + } + } else { + // readSchema must contain attributes of all filters. + // Accordingly, fieldIndex() returns a valid index always. + val index = refs.map(readSchema.fieldIndex).max + groupedExprs(index) ++= expr + } } for (i <- 0 until len) { if (!groupedExprs(i).isEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index cbb0a4d739f61..7a8d402d682eb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -18,10 +18,12 @@ package org.apache.spark.sql.catalyst.csv import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.sources -import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.{AlwaysFalse, AlwaysTrue, Filter} import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.unsafe.types.UTF8String class CSVFiltersSuite extends SparkFunSuite { test("filter to expression conversion") { @@ -58,7 +60,7 @@ class CSVFiltersSuite extends SparkFunSuite { def check( dataSchema: String = "i INTEGER, d DOUBLE, s STRING", requiredSchema: String = "s STRING", - filters: Seq[sources.Filter], + filters: Seq[Filter], expected: String): Unit = { val csvFilters = new CSVFilters(filters, getSchema(dataSchema), getSchema(requiredSchema)) assert(csvFilters.readSchema === getSchema(expected)) @@ -83,4 +85,75 @@ class CSVFiltersSuite extends SparkFunSuite { assert(e.getMessage.contains("All filters must be applicable to the data schema")) } } + + test("skipping rows") { + def check( + dataSchema: String = "i INTEGER, d DOUBLE, s STRING", + requiredSchema: String = "d DOUBLE", + filters: Seq[Filter], + row: InternalRow, + pos: Int, + skip: Boolean): Unit = { + val csvFilters = new CSVFilters(filters, getSchema(dataSchema), getSchema(requiredSchema)) + assert(csvFilters.skipRow(row, pos) === skip) + } + + check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) + check(filters = Seq(AlwaysTrue), row = InternalRow(1), pos = 0, skip = false) + check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) + check( + filters = Seq(sources.EqualTo("i", 10)), + row = InternalRow(10, 3.14), + pos = 0, + skip = false) + check( + filters = Seq(sources.IsNotNull("d"), sources.GreaterThanOrEqual("d", 2.96)), + row = InternalRow(3.14), + pos = 0, + skip = false) + check( + filters = Seq(sources.In("i", Array(10, 20)), sources.LessThanOrEqual("d", 2.96)), + row = InternalRow(10, 3.14), + pos = 1, + skip = true) + val filters1 = Seq( + sources.Or( + sources.AlwaysTrue, + sources.And( + sources.Not(sources.IsNull("i")), + sources.Not( + sources.And( + sources.StringEndsWith("s", "ab"), + sources.StringEndsWith("s", "cd") + ) + ) + ) + ), + sources.GreaterThan("d", 0), + sources.LessThan("i", 500) + ) + val filters2 = Seq( + sources.And( + sources.StringContains("s", "abc"), + sources.And( + sources.Not(sources.IsNull("i")), + sources.And( + sources.StringEndsWith("s", "ab"), + sources.StringEndsWith("s", "bc") + ) + ) + ), + sources.GreaterThan("d", 100), + sources.LessThan("i", 0) + ) + Seq(filters1 -> false, filters2 -> true).foreach { case (filters, skip) => + for (p <- 0 until 3) { + check( + filters = filters, + row = InternalRow(10, 3.14, UTF8String.fromString("abc")), + pos = p, + skip = skip) + } + } + } } From 11bcbc67e22d9efd61034a4705a5b7a31dde3b89 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Dec 2019 13:29:47 +0300 Subject: [PATCH 24/43] Benchmarks at the commit 67b644c3d7 --- .../benchmarks/CSVBenchmark-jdk11-results.txt | 78 +++++++++---------- sql/core/benchmarks/CSVBenchmark-results.txt | 78 +++++++++---------- 2 files changed, 78 insertions(+), 78 deletions(-) diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index b65b236fd71f2..b37f1bfc0004f 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -2,58 +2,58 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 56894 57106 184 0.0 1137889.9 1.0X +One quoted string 33709 34840 979 0.0 674182.3 1.0X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 220825 222234 2018 0.0 220825.5 1.0X -Select 100 columns 50507 50723 278 0.0 50506.6 4.4X -Select one column 38629 38642 16 0.0 38628.6 5.7X -count() 8549 8597 51 0.1 8549.2 25.8X -Select 100 columns, one bad input field 68309 68474 182 0.0 68309.2 3.2X -Select 100 columns, corrupt record field 74551 74701 136 0.0 74551.5 3.0X +Select 1000 columns 114621 116640 NaN 0.0 114621.0 1.0X +Select 100 columns 33770 34721 947 0.0 33770.1 3.4X +Select one column 28932 29123 257 0.0 28932.1 4.0X +count() 5857 5880 21 0.2 5856.9 19.6X +Select 100 columns, one bad input field 45562 45631 61 0.0 45561.8 2.5X +Select 100 columns, corrupt record field 50106 50182 67 0.0 50105.7 2.3X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 27745 28050 276 0.4 2774.5 1.0X -Select 1 column + count() 19989 20315 319 0.5 1998.9 1.4X -count() 6091 6109 25 1.6 609.1 4.6X +Select 10 columns + count() 17164 17212 43 0.6 1716.4 1.0X +Select 1 column + count() 12837 12896 64 0.8 1283.7 1.3X +count() 4705 4750 41 2.1 470.5 3.6X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2235 2301 59 4.5 223.5 1.0X -to_csv(timestamp) 16033 16205 153 0.6 1603.3 0.1X -write timestamps to files 13556 13685 167 0.7 1355.6 0.2X -Create a dataset of dates 2262 2290 44 4.4 226.2 1.0X -to_csv(date) 11122 11160 33 0.9 1112.2 0.2X -write dates to files 8436 8486 76 1.2 843.6 0.3X +Create a dataset of timestamps 1300 1307 6 7.7 130.0 1.0X +to_csv(timestamp) 10720 10924 350 0.9 1072.0 0.1X +write timestamps to files 9503 9801 358 1.1 950.3 0.1X +Create a dataset of dates 1275 1290 15 7.8 127.5 1.0X +to_csv(date) 6225 6456 373 1.6 622.5 0.2X +write dates to files 5968 6008 40 1.7 596.8 0.2X -OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2617 2644 26 3.8 261.7 1.0X -read timestamps from files 53245 53381 149 0.2 5324.5 0.0X -infer timestamps from files 103797 104026 257 0.1 10379.7 0.0X -read date text from files 2371 2378 7 4.2 237.1 1.1X -read date from files 41808 41929 177 0.2 4180.8 0.1X -infer date from files 35069 35336 458 0.3 3506.9 0.1X -timestamp strings 3104 3127 21 3.2 310.4 0.8X -parse timestamps from Dataset[String] 61888 62132 342 0.2 6188.8 0.0X -infer timestamps from Dataset[String] 112494 114609 1949 0.1 11249.4 0.0X -date strings 3558 3603 41 2.8 355.8 0.7X -parse dates from Dataset[String] 45871 46000 120 0.2 4587.1 0.1X -from_csv(timestamp) 56975 57035 53 0.2 5697.5 0.0X -from_csv(date) 43711 43795 74 0.2 4371.1 0.1X +read timestamp text from files 2294 2315 35 4.4 229.4 1.0X +read timestamps from files 28261 28293 51 0.4 2826.1 0.1X +infer timestamps from files 53355 53453 108 0.2 5335.5 0.0X +read date text from files 1897 1923 26 5.3 189.7 1.2X +read date from files 21504 21666 215 0.5 2150.4 0.1X +infer date from files 20932 21041 105 0.5 2093.2 0.1X +timestamp strings 2085 2091 9 4.8 208.5 1.1X +parse timestamps from Dataset[String] 30174 30784 887 0.3 3017.4 0.1X +infer timestamps from Dataset[String] 52927 56366 1154 0.2 5292.7 0.0X +date strings 2334 2350 14 4.3 233.4 1.0X +parse dates from Dataset[String] 23282 23399 125 0.4 2328.2 0.1X +from_csv(timestamp) 26733 27472 718 0.4 2673.3 0.1X +from_csv(date) 22017 22674 694 0.5 2201.7 0.1X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index d2037e86a3a71..7309520afabc4 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,58 +2,58 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 62603 62755 133 0.0 1252055.6 1.0X +One quoted string 35730 36630 1107 0.0 714593.4 1.0X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 225032 225919 782 0.0 225031.7 1.0X -Select 100 columns 51982 52290 286 0.0 51982.1 4.3X -Select one column 40167 40283 133 0.0 40167.4 5.6X -count() 11435 11593 176 0.1 11435.1 19.7X -Select 100 columns, one bad input field 66864 66968 174 0.0 66864.1 3.4X -Select 100 columns, corrupt record field 79570 80418 1080 0.0 79569.5 2.8X +Select 1000 columns 119121 122140 684 0.0 119120.7 1.0X +Select 100 columns 33691 34213 677 0.0 33690.8 3.5X +Select one column 26751 26839 79 0.0 26750.9 4.5X +count() 7166 7196 36 0.1 7166.0 16.6X +Select 100 columns, one bad input field 40270 40333 70 0.0 40269.7 3.0X +Select 100 columns, corrupt record field 46937 47831 904 0.0 46937.3 2.5X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 23271 23389 103 0.4 2327.1 1.0X -Select 1 column + count() 18206 19772 NaN 0.5 1820.6 1.3X -count() 8500 8521 18 1.2 850.0 2.7X +Select 10 columns + count() 15890 15964 64 0.6 1589.0 1.0X +Select 1 column + count() 10267 10316 45 1.0 1026.7 1.5X +count() 4453 4465 17 2.2 445.3 3.6X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 2025 2068 66 4.9 202.5 1.0X -to_csv(timestamp) 22192 22983 879 0.5 2219.2 0.1X -write timestamps to files 15949 16030 72 0.6 1594.9 0.1X -Create a dataset of dates 2200 2234 32 4.5 220.0 0.9X -to_csv(date) 18268 18341 73 0.5 1826.8 0.1X -write dates to files 10495 10722 214 1.0 1049.5 0.2X +Create a dataset of timestamps 1071 1097 38 9.3 107.1 1.0X +to_csv(timestamp) 10080 10329 379 1.0 1008.0 0.1X +write timestamps to files 9057 9142 76 1.1 905.7 0.1X +Create a dataset of dates 1254 1260 5 8.0 125.4 0.9X +to_csv(date) 6519 6534 14 1.5 651.9 0.2X +write dates to files 5140 5164 26 1.9 514.0 0.2X -OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 -Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 6491 6503 18 1.5 649.1 1.0X -read timestamps from files 56069 56795 874 0.2 5606.9 0.1X -infer timestamps from files 113383 114203 825 0.1 11338.3 0.1X -read date text from files 6411 6419 10 1.6 641.1 1.0X -read date from files 46245 46371 138 0.2 4624.5 0.1X -infer date from files 43623 43906 291 0.2 4362.3 0.1X -timestamp strings 4951 4959 7 2.0 495.1 1.3X -parse timestamps from Dataset[String] 65786 66309 663 0.2 6578.6 0.1X -infer timestamps from Dataset[String] 130891 133861 1928 0.1 13089.1 0.0X -date strings 3814 3895 84 2.6 381.4 1.7X -parse dates from Dataset[String] 52259 52960 614 0.2 5225.9 0.1X -from_csv(timestamp) 63013 63306 291 0.2 6301.3 0.1X -from_csv(date) 49840 52352 NaN 0.2 4984.0 0.1X +read timestamp text from files 1888 1903 13 5.3 188.8 1.0X +read timestamps from files 25925 25977 76 0.4 2592.5 0.1X +infer timestamps from files 50415 50457 74 0.2 5041.5 0.0X +read date text from files 1601 1610 10 6.2 160.1 1.2X +read date from files 20057 20089 35 0.5 2005.7 0.1X +infer date from files 19180 19267 104 0.5 1918.0 0.1X +timestamp strings 2283 2313 27 4.4 228.3 0.8X +parse timestamps from Dataset[String] 28610 28716 106 0.3 2861.0 0.1X +infer timestamps from Dataset[String] 53327 53586 439 0.2 5332.7 0.0X +date strings 2682 2725 40 3.7 268.2 0.7X +parse dates from Dataset[String] 22683 22773 92 0.4 2268.3 0.1X +from_csv(timestamp) 27729 27788 57 0.4 2772.9 0.1X +from_csv(date) 22240 22289 47 0.4 2224.0 0.1X From a5088bd35abb09dbd8a5a178c67106e197bec857 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Dec 2019 21:49:40 +0300 Subject: [PATCH 25/43] Revert "Benchmarks at the commit 67b644c3d7" This reverts commit 11bcbc67e22d9efd61034a4705a5b7a31dde3b89. --- .../benchmarks/CSVBenchmark-jdk11-results.txt | 78 +++++++++---------- sql/core/benchmarks/CSVBenchmark-results.txt | 78 +++++++++---------- 2 files changed, 78 insertions(+), 78 deletions(-) diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index b37f1bfc0004f..b65b236fd71f2 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -2,58 +2,58 @@ Benchmark to measure CSV read/write performance ================================================================================================ -OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 33709 34840 979 0.0 674182.3 1.0X +One quoted string 56894 57106 184 0.0 1137889.9 1.0X -OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 114621 116640 NaN 0.0 114621.0 1.0X -Select 100 columns 33770 34721 947 0.0 33770.1 3.4X -Select one column 28932 29123 257 0.0 28932.1 4.0X -count() 5857 5880 21 0.2 5856.9 19.6X -Select 100 columns, one bad input field 45562 45631 61 0.0 45561.8 2.5X -Select 100 columns, corrupt record field 50106 50182 67 0.0 50105.7 2.3X +Select 1000 columns 220825 222234 2018 0.0 220825.5 1.0X +Select 100 columns 50507 50723 278 0.0 50506.6 4.4X +Select one column 38629 38642 16 0.0 38628.6 5.7X +count() 8549 8597 51 0.1 8549.2 25.8X +Select 100 columns, one bad input field 68309 68474 182 0.0 68309.2 3.2X +Select 100 columns, corrupt record field 74551 74701 136 0.0 74551.5 3.0X -OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 17164 17212 43 0.6 1716.4 1.0X -Select 1 column + count() 12837 12896 64 0.8 1283.7 1.3X -count() 4705 4750 41 2.1 470.5 3.6X +Select 10 columns + count() 27745 28050 276 0.4 2774.5 1.0X +Select 1 column + count() 19989 20315 319 0.5 1998.9 1.4X +count() 6091 6109 25 1.6 609.1 4.6X -OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 1300 1307 6 7.7 130.0 1.0X -to_csv(timestamp) 10720 10924 350 0.9 1072.0 0.1X -write timestamps to files 9503 9801 358 1.1 950.3 0.1X -Create a dataset of dates 1275 1290 15 7.8 127.5 1.0X -to_csv(date) 6225 6456 373 1.6 622.5 0.2X -write dates to files 5968 6008 40 1.7 596.8 0.2X +Create a dataset of timestamps 2235 2301 59 4.5 223.5 1.0X +to_csv(timestamp) 16033 16205 153 0.6 1603.3 0.1X +write timestamps to files 13556 13685 167 0.7 1355.6 0.2X +Create a dataset of dates 2262 2290 44 4.4 226.2 1.0X +to_csv(date) 11122 11160 33 0.9 1112.2 0.2X +write dates to files 8436 8486 76 1.2 843.6 0.3X -OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 11.0.4+11-LTS on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 2294 2315 35 4.4 229.4 1.0X -read timestamps from files 28261 28293 51 0.4 2826.1 0.1X -infer timestamps from files 53355 53453 108 0.2 5335.5 0.0X -read date text from files 1897 1923 26 5.3 189.7 1.2X -read date from files 21504 21666 215 0.5 2150.4 0.1X -infer date from files 20932 21041 105 0.5 2093.2 0.1X -timestamp strings 2085 2091 9 4.8 208.5 1.1X -parse timestamps from Dataset[String] 30174 30784 887 0.3 3017.4 0.1X -infer timestamps from Dataset[String] 52927 56366 1154 0.2 5292.7 0.0X -date strings 2334 2350 14 4.3 233.4 1.0X -parse dates from Dataset[String] 23282 23399 125 0.4 2328.2 0.1X -from_csv(timestamp) 26733 27472 718 0.4 2673.3 0.1X -from_csv(date) 22017 22674 694 0.5 2201.7 0.1X +read timestamp text from files 2617 2644 26 3.8 261.7 1.0X +read timestamps from files 53245 53381 149 0.2 5324.5 0.0X +infer timestamps from files 103797 104026 257 0.1 10379.7 0.0X +read date text from files 2371 2378 7 4.2 237.1 1.1X +read date from files 41808 41929 177 0.2 4180.8 0.1X +infer date from files 35069 35336 458 0.3 3506.9 0.1X +timestamp strings 3104 3127 21 3.2 310.4 0.8X +parse timestamps from Dataset[String] 61888 62132 342 0.2 6188.8 0.0X +infer timestamps from Dataset[String] 112494 114609 1949 0.1 11249.4 0.0X +date strings 3558 3603 41 2.8 355.8 0.7X +parse dates from Dataset[String] 45871 46000 120 0.2 4587.1 0.1X +from_csv(timestamp) 56975 57035 53 0.2 5697.5 0.0X +from_csv(date) 43711 43795 74 0.2 4371.1 0.1X diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index 7309520afabc4..d2037e86a3a71 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -2,58 +2,58 @@ Benchmark to measure CSV read/write performance ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -One quoted string 35730 36630 1107 0.0 714593.4 1.0X +One quoted string 62603 62755 133 0.0 1252055.6 1.0X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 1000 columns 119121 122140 684 0.0 119120.7 1.0X -Select 100 columns 33691 34213 677 0.0 33690.8 3.5X -Select one column 26751 26839 79 0.0 26750.9 4.5X -count() 7166 7196 36 0.1 7166.0 16.6X -Select 100 columns, one bad input field 40270 40333 70 0.0 40269.7 3.0X -Select 100 columns, corrupt record field 46937 47831 904 0.0 46937.3 2.5X +Select 1000 columns 225032 225919 782 0.0 225031.7 1.0X +Select 100 columns 51982 52290 286 0.0 51982.1 4.3X +Select one column 40167 40283 133 0.0 40167.4 5.6X +count() 11435 11593 176 0.1 11435.1 19.7X +Select 100 columns, one bad input field 66864 66968 174 0.0 66864.1 3.4X +Select 100 columns, corrupt record field 79570 80418 1080 0.0 79569.5 2.8X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Count a dataset with 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Select 10 columns + count() 15890 15964 64 0.6 1589.0 1.0X -Select 1 column + count() 10267 10316 45 1.0 1026.7 1.5X -count() 4453 4465 17 2.2 445.3 3.6X +Select 10 columns + count() 23271 23389 103 0.4 2327.1 1.0X +Select 1 column + count() 18206 19772 NaN 0.5 1820.6 1.3X +count() 8500 8521 18 1.2 850.0 2.7X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -Create a dataset of timestamps 1071 1097 38 9.3 107.1 1.0X -to_csv(timestamp) 10080 10329 379 1.0 1008.0 0.1X -write timestamps to files 9057 9142 76 1.1 905.7 0.1X -Create a dataset of dates 1254 1260 5 8.0 125.4 0.9X -to_csv(date) 6519 6534 14 1.5 651.9 0.2X -write dates to files 5140 5164 26 1.9 514.0 0.2X +Create a dataset of timestamps 2025 2068 66 4.9 202.5 1.0X +to_csv(timestamp) 22192 22983 879 0.5 2219.2 0.1X +write timestamps to files 15949 16030 72 0.6 1594.9 0.1X +Create a dataset of dates 2200 2234 32 4.5 220.0 0.9X +to_csv(date) 18268 18341 73 0.5 1826.8 0.1X +write dates to files 10495 10722 214 1.0 1049.5 0.2X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 -Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +OpenJDK 64-Bit Server VM 1.8.0_222-b10 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------------------------------ -read timestamp text from files 1888 1903 13 5.3 188.8 1.0X -read timestamps from files 25925 25977 76 0.4 2592.5 0.1X -infer timestamps from files 50415 50457 74 0.2 5041.5 0.0X -read date text from files 1601 1610 10 6.2 160.1 1.2X -read date from files 20057 20089 35 0.5 2005.7 0.1X -infer date from files 19180 19267 104 0.5 1918.0 0.1X -timestamp strings 2283 2313 27 4.4 228.3 0.8X -parse timestamps from Dataset[String] 28610 28716 106 0.3 2861.0 0.1X -infer timestamps from Dataset[String] 53327 53586 439 0.2 5332.7 0.0X -date strings 2682 2725 40 3.7 268.2 0.7X -parse dates from Dataset[String] 22683 22773 92 0.4 2268.3 0.1X -from_csv(timestamp) 27729 27788 57 0.4 2772.9 0.1X -from_csv(date) 22240 22289 47 0.4 2224.0 0.1X +read timestamp text from files 6491 6503 18 1.5 649.1 1.0X +read timestamps from files 56069 56795 874 0.2 5606.9 0.1X +infer timestamps from files 113383 114203 825 0.1 11338.3 0.1X +read date text from files 6411 6419 10 1.6 641.1 1.0X +read date from files 46245 46371 138 0.2 4624.5 0.1X +infer date from files 43623 43906 291 0.2 4362.3 0.1X +timestamp strings 4951 4959 7 2.0 495.1 1.3X +parse timestamps from Dataset[String] 65786 66309 663 0.2 6578.6 0.1X +infer timestamps from Dataset[String] 130891 133861 1928 0.1 13089.1 0.0X +date strings 3814 3895 84 2.6 381.4 1.7X +parse dates from Dataset[String] 52259 52960 614 0.2 5225.9 0.1X +from_csv(timestamp) 63013 63306 291 0.2 6301.3 0.1X +from_csv(date) 49840 52352 NaN 0.2 4984.0 0.1X From f0cc83c9709d23746a7ab6b33dc7d1fa7bfac953 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 20 Dec 2019 21:51:38 +0300 Subject: [PATCH 26/43] Update benchmarks results --- sql/core/benchmarks/CSVBenchmark-jdk11-results.txt | 8 ++++++++ sql/core/benchmarks/CSVBenchmark-results.txt | 8 ++++++++ 2 files changed, 16 insertions(+) diff --git a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt index b65b236fd71f2..57bba7091212f 100644 --- a/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-jdk11-results.txt @@ -56,4 +56,12 @@ parse dates from Dataset[String] 45871 46000 1 from_csv(timestamp) 56975 57035 53 0.2 5697.5 0.0X from_csv(date) 43711 43795 74 0.2 4371.1 0.1X +OpenJDK 64-Bit Server VM 11.0.5+10 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 11889 11945 52 0.0 118893.1 1.0X +pushdown disabled 11790 11860 115 0.0 117902.3 1.0X +w/ filters 1240 1278 33 0.1 12400.8 9.6X + diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt index d2037e86a3a71..f5bebcaa32d57 100644 --- a/sql/core/benchmarks/CSVBenchmark-results.txt +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -56,4 +56,12 @@ parse dates from Dataset[String] 52259 52960 6 from_csv(timestamp) 63013 63306 291 0.2 6301.3 0.1X from_csv(date) 49840 52352 NaN 0.2 4984.0 0.1X +Java HotSpot(TM) 64-Bit Server VM 1.8.0_231-b11 on Mac OS X 10.15.2 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Filters pushdown: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +w/o filters 12557 12634 78 0.0 125572.9 1.0X +pushdown disabled 12449 12509 65 0.0 124486.4 1.0X +w/ filters 1372 1393 18 0.1 13724.8 9.1X + From f24e8734698c65cc17cbd33efc27962f33fc1ca0 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sat, 21 Dec 2019 01:17:49 +0300 Subject: [PATCH 27/43] Add equals(), hashCode() and description() to CSVScan --- .../execution/datasources/v2/csv/CSVScan.scala | 18 ++++++++++++++++-- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 41d43335d55ac..166537b3c54b7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -39,7 +39,7 @@ case class CSVScan( readDataSchema: StructType, readPartitionSchema: StructType, options: CaseInsensitiveStringMap, - filters: Seq[Filter]) + pushedFilters: Array[Filter]) extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { private lazy val parsedOptions: CSVOptions = new CSVOptions( @@ -87,6 +87,20 @@ case class CSVScan( // The partition values are already truncated in `FileScan.partitions`. // We should use `readPartitionSchema` as the partition schema here. CSVPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, - dataSchema, readDataSchema, readPartitionSchema, parsedOptions, filters) + dataSchema, readDataSchema, readPartitionSchema, parsedOptions, pushedFilters) + } + + override def equals(obj: Any): Boolean = obj match { + case o: CSVScan => + fileIndex == o.fileIndex && dataSchema == o.dataSchema && + readDataSchema == o.readDataSchema && readPartitionSchema == o.readPartitionSchema && + options == o.options && equivalentFilters(pushedFilters, o.pushedFilters) + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") } } From 55ebb6080f65f909ee423adb346589fb8acd8bbf Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Dec 2019 11:59:40 +0300 Subject: [PATCH 28/43] Tests for column pruning on/off + refactoring --- .../spark/sql/catalyst/csv/CSVFilters.scala | 16 ++++-- .../sql/catalyst/csv/UnivocityParser.scala | 50 +++++++++---------- .../sql/catalyst/csv/CSVFiltersSuite.scala | 21 ++++++-- .../datasources/csv/CSVFileFormat.scala | 2 +- .../v2/csv/CSVPartitionReaderFactory.scala | 2 +- .../execution/datasources/csv/CSVSuite.scala | 44 +++++++++------- 6 files changed, 81 insertions(+), 54 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 8f8f86b403635..d7643266338da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -33,11 +33,13 @@ import org.apache.spark.sql.types.{BooleanType, StructType} * @param filters The filters pushed down to CSV datasource. * @param dataSchema The full schema with all fields in CSV files. * @param requiredSchema The schema with only fields requested by the upper layer. + * @param columnPruning true if CSV parser can read sub-set of columns otherwise false. */ class CSVFilters( filters: Seq[sources.Filter], dataSchema: StructType, - requiredSchema: StructType) { + requiredSchema: StructType, + columnPruning: Boolean) { require(checkFilters(), "All filters must be applicable to the data schema.") /** @@ -45,11 +47,15 @@ class CSVFilters( * It combines the required schema and the fields referenced by filters. */ val readSchema: StructType = { - val refs = filters.flatMap(_.references).toSet - val readFields = dataSchema.filter { field => - requiredSchema.contains(field) || refs.contains(field.name) + if (columnPruning) { + val refs = filters.flatMap(_.references).toSet + val readFields = dataSchema.filter { field => + requiredSchema.contains(field) || refs.contains(field.name) + } + StructType(readFields) + } else { + dataSchema } - StructType(readFields) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index a4abac0b1b694..cd73b32f92d3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -59,19 +59,23 @@ class UnivocityParser( // A `ValueConverter` is responsible for converting the given value to a desired type. private type ValueConverter = String => Any - private val csvFilters = new CSVFilters(filters, dataSchema, requiredSchema) + private val csvFilters = new CSVFilters( + filters, + dataSchema, + requiredSchema, + options.columnPruning) - // The "minimal" schema to be read from Univocity parser. - // It includes `requiredSchema` + the fields referenced by pushed down filters. - val readSchema = csvFilters.readSchema + // When column pruning is enabled, the parser only parses the required columns based on + // their positions in the data schema. + private[sql] val parsedSchema = if (options.columnPruning) csvFilters.readSchema else dataSchema - // Mapping of field indexes of `readSchema` to indexes of `requiredSchema`. - // It returns -1 if `requiredSchema` doesn't contain a field from `readSchema`. - private val readToRequiredIndex: Array[Int] = { - val arr = Array.fill(readSchema.length)(-1) + // Mapping of field indexes of `parsedSchema` to indexes of `requiredSchema`. + // It returns -1 if `requiredSchema` doesn't contain a field from `parsedSchema`. + private val parsedToRequiredIndex: Array[Int] = { + val arr = Array.fill(parsedSchema.length)(-1) for { - readIndex <- 0 until readSchema.length - reqIndex <- requiredSchema.getFieldIndex(readSchema(readIndex).name) + readIndex <- 0 until parsedSchema.length + reqIndex <- requiredSchema.getFieldIndex(parsedSchema(readIndex).name) } { arr(readIndex) = reqIndex } @@ -80,11 +84,7 @@ class UnivocityParser( // This index is used to reorder parsed tokens private val tokenIndexArr = - readSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray - - // When column pruning is enabled, the parser only parses the required columns based on - // their positions in the data schema. - private val parsedSchema = if (options.columnPruning) readSchema else dataSchema + parsedSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray val tokenizer = { val parserSetting = options.asParserSettings @@ -99,7 +99,7 @@ class UnivocityParser( // The row is used as a temporary placeholder of parsed and converted values. // It is needed for applying the pushdown filters. - private val readRow = new GenericInternalRow(readSchema.length) + private val parsedRow = new GenericInternalRow(parsedSchema.length) // Pre-allocated Seq to avoid the overhead of the seq builder. private val requiredRow = Seq(new GenericInternalRow(requiredSchema.length)) // Pre-allocated empty sequence returned when the parsed row cannot pass filters. @@ -140,7 +140,7 @@ class UnivocityParser( // // output row - ["A", 2] private val valueConverters: Array[ValueConverter] = { - readSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray + parsedSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray } private val decimalParser = ExprUtils.getDecimalParser(options.locale) @@ -223,7 +223,7 @@ class UnivocityParser( } } - private val doParse = if (options.columnPruning && readSchema.isEmpty) { + private val doParse = if (options.columnPruning && parsedSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. (_: String) => Seq(InternalRow.empty) @@ -275,9 +275,9 @@ class UnivocityParser( } else { // When the length of the returned tokens is identical to the length of the parsed schema, // we just need to: - // 1. Convert the tokens that correspond to the read schema. - // 2. Apply the pushdown filters to `readRow`. - // 3. Convert `readRow` to `requiredRow` by stripping non-required fields. + // 1. Convert the tokens that correspond to the parsed schema. + // 2. Apply the pushdown filters to `parsedRow`. + // 3. Convert `parsedRow` to `requiredRow` by stripping non-required fields. var i = 0 val requiredSingleRow = requiredRow.head while (i < requiredSchema.length) { @@ -288,14 +288,14 @@ class UnivocityParser( var skipValueConversion = false var badRecordException: Option[Throwable] = None i = 0 - while (!skipValueConversion && i < readSchema.length) { + while (!skipValueConversion && i < parsedSchema.length) { try { val convertedValue = valueConverters(i).apply(getToken(tokens, i)) - readRow(i) = convertedValue - if (csvFilters.skipRow(readRow, i)) { + parsedRow(i) = convertedValue + if (csvFilters.skipRow(parsedRow, i)) { skipValueConversion = true } else { - val requiredIndex = readToRequiredIndex(i) + val requiredIndex = parsedToRequiredIndex(i) if (requiredIndex != -1) { requiredSingleRow(requiredIndex) = convertedValue } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index 7a8d402d682eb..a9478bd950c07 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -60,14 +60,23 @@ class CSVFiltersSuite extends SparkFunSuite { def check( dataSchema: String = "i INTEGER, d DOUBLE, s STRING", requiredSchema: String = "s STRING", + columnPruning: Boolean = true, filters: Seq[Filter], expected: String): Unit = { - val csvFilters = new CSVFilters(filters, getSchema(dataSchema), getSchema(requiredSchema)) + val csvFilters = new CSVFilters( + filters, + getSchema(dataSchema), + getSchema(requiredSchema), + columnPruning) assert(csvFilters.readSchema === getSchema(expected)) } check(filters = Seq(), expected = "s STRING") check(filters = Seq(sources.EqualTo("d", 3.14)), expected = "d DOUBLE, s STRING") + check( + filters = Seq(sources.EqualTo("d", 3.14)), + columnPruning = false, + expected = "i INTEGER, d DOUBLE, s STRING") check( filters = Seq(sources.And(sources.EqualTo("d", 3.14), sources.StringEndsWith("s", "a"))), expected = "d DOUBLE, s STRING") @@ -94,8 +103,14 @@ class CSVFiltersSuite extends SparkFunSuite { row: InternalRow, pos: Int, skip: Boolean): Unit = { - val csvFilters = new CSVFilters(filters, getSchema(dataSchema), getSchema(requiredSchema)) - assert(csvFilters.skipRow(row, pos) === skip) + Seq(true, false).foreach { columnPruning => + val csvFilters = new CSVFilters( + filters, + getSchema(dataSchema), + getSchema(requiredSchema), + columnPruning) + assert(csvFilters.skipRow(row, pos) === skip) + } } check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 45391c6e71110..aa552764e7eda 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -139,7 +139,7 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { filters) val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( - parser.readSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + parser.parsedSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) CSVDataSource(parsedOptions).readFile( conf, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index d2b5482f084b6..4f3d309faeb4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -59,7 +59,7 @@ case class CSVPartitionReaderFactory( filters) val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( - parser.readSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + parser.parsedSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) val iter = CSVDataSource(parsedOptions).readFile( conf, file, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 54b514cd6ba49..a04baa96cf915 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2206,25 +2206,31 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } test("filters push down") { - withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> "true") { - withTempPath { path => - val t = "2019-12-17 00:01:02" - Seq( - "c1,c2", - "1,2019-11-14 20:35:30", - s"2,$t").toDF("data") - .repartition(1) - .write.text(path.getAbsolutePath) - val readback = spark.read - .option("header", true) - .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") - .schema("c1 integer, c2 timestamp") - .csv(path.getAbsolutePath) - .where($"c1" === 2) - // count() pushes empty schema. This checks handling of a filter - // which refers to not existed field. - assert(readback.count() === 1) - checkAnswer(readback, Row(2, Timestamp.valueOf(t))) + Seq(true, false).foreach { columnPruning => + withSQLConf( + SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> columnPruning.toString) { + + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + "c0,c1,c2", + "abc,1,2019-11-14 20:35:30", + s"def,2,$t").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + val readback = spark.read + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .schema("c0 string, c1 integer, c2 timestamp") + .csv(path.getAbsolutePath) + .where($"c1" === 2) + .select($"c2") + // count() pushes empty schema. This checks handling of a filter + // which refers to not existed field. + assert(readback.count() === 1) + checkAnswer(readback, Row(Timestamp.valueOf(t))) + } } } } From 170944c488132f5f5e51d8e1a080f52f48c41df6 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 22 Dec 2019 12:18:08 +0300 Subject: [PATCH 29/43] Simplifying parsedSchema initialization --- .../org/apache/spark/sql/catalyst/csv/UnivocityParser.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index cd73b32f92d3a..bf0f90aab21d2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -65,9 +65,7 @@ class UnivocityParser( requiredSchema, options.columnPruning) - // When column pruning is enabled, the parser only parses the required columns based on - // their positions in the data schema. - private[sql] val parsedSchema = if (options.columnPruning) csvFilters.readSchema else dataSchema + private[sql] val parsedSchema = csvFilters.readSchema // Mapping of field indexes of `parsedSchema` to indexes of `requiredSchema`. // It returns -1 if `requiredSchema` doesn't contain a field from `parsedSchema`. From f2962599985b78050d96116853ca80eb5284eb3a Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 9 Jan 2020 22:32:02 +0300 Subject: [PATCH 30/43] Test the multiLine mode --- .../execution/datasources/csv/CSVSuite.scala | 51 ++++++++++--------- 1 file changed, 27 insertions(+), 24 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index a04baa96cf915..4ace7c02cbda7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2206,30 +2206,33 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } test("filters push down") { - Seq(true, false).foreach { columnPruning => - withSQLConf( - SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> columnPruning.toString) { - - withTempPath { path => - val t = "2019-12-17 00:01:02" - Seq( - "c0,c1,c2", - "abc,1,2019-11-14 20:35:30", - s"def,2,$t").toDF("data") - .repartition(1) - .write.text(path.getAbsolutePath) - val readback = spark.read - .option("header", true) - .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") - .schema("c0 string, c1 integer, c2 timestamp") - .csv(path.getAbsolutePath) - .where($"c1" === 2) - .select($"c2") - // count() pushes empty schema. This checks handling of a filter - // which refers to not existed field. - assert(readback.count() === 1) - checkAnswer(readback, Row(Timestamp.valueOf(t))) + Seq(true, false).foreach { multiLine => + Seq(true, false).foreach { columnPruning => + withSQLConf( + SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> columnPruning.toString) { + + withTempPath { path => + val t = "2019-12-17 00:01:02" + Seq( + "c0,c1,c2", + "abc,1,2019-11-14 20:35:30", + s"def,2,$t").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + val readback = spark.read + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .option("multiLine", multiLine) + .schema("c0 string, c1 integer, c2 timestamp") + .csv(path.getAbsolutePath) + .where($"c1" === 2) + .select($"c2") + // count() pushes empty schema. This checks handling of a filter + // which refers to not existed field. + assert(readback.count() === 1) + checkAnswer(readback, Row(Timestamp.valueOf(t))) + } } } } From 61eaa368398942f6525206140ed9cd68b48c75e1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 10 Jan 2020 11:10:27 +0300 Subject: [PATCH 31/43] Follow-up merging --- .../sql/execution/datasources/v2/csv/CSVScan.scala | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index 3066357f665a1..e0531c49ff180 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -41,7 +41,7 @@ case class CSVScan( options: CaseInsensitiveStringMap, pushedFilters: Array[Filter], partitionFilters: Seq[Expression] = Seq.empty) - extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { + extends TextBasedFileScan(sparkSession, options) { private lazy val parsedOptions: CSVOptions = new CSVOptions( options.asScala.toMap, @@ -91,6 +91,9 @@ case class CSVScan( dataSchema, readDataSchema, readPartitionSchema, parsedOptions, pushedFilters) } + override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = + this.copy(partitionFilters = partitionFilters) + override def equals(obj: Any): Boolean = obj match { case o: CSVScan => fileIndex == o.fileIndex && dataSchema == o.dataSchema && @@ -99,13 +102,9 @@ case class CSVScan( case _ => false } - override def hashCode(): Int = getClass.hashCode() + override def hashCode(): Int = super.hashCode() override def description(): String = { super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") } - - override def withPartitionFilters(partitionFilters: Seq[Expression]): FileScan = - this.copy(partitionFilters = partitionFilters) - } From f0aa0a88bfa0c87007f8781ba7fac8f9cd3057ba Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Sun, 12 Jan 2020 00:10:34 +0300 Subject: [PATCH 32/43] Bug fix --- .../spark/sql/catalyst/csv/CSVFilters.scala | 22 +++++++++---------- .../datasources/v2/csv/CSVScan.scala | 6 ++--- .../datasources/v2/csv/CSVScanBuilder.scala | 6 +++-- 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index d7643266338da..16fe382dece32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -124,24 +124,24 @@ class CSVFilters( // Checks that all filters refer to an field in the data schema private def checkFilters(): Boolean = { - val refs = filters.flatMap(_.references).toSet - val fieldNames = dataSchema.fields.map(_.name).toSet - refs.forall(fieldNames.contains(_)) + filters.forall(CSVFilters.checkFilterRefs(_, dataSchema)) } } object CSVFilters { + private def checkFilterRefs(filter: sources.Filter, schema: StructType): Boolean = { + val fieldNames = schema.fields.map(_.name).toSet + filter.references.forall(fieldNames.contains(_)) + } + /** - * Returns the filters currently not supported by CSV datasource. + * Returns the filters currently supported by CSV datasource. * @param filters The filters pushed down to CSV datasource. - * @return a sub-set of `filters` that cannot be handled by CSV datasource. + * @param schema data schema of CSV files. + * @return a sub-set of `filters` that can be handled by CSV datasource. */ - def unsupportedFilters(filters: Array[sources.Filter]): Array[sources.Filter] = { - if (SQLConf.get.csvFilterPushDown) { - Array.empty - } else { - filters - } + def pushedFilters(filters: Array[sources.Filter], schema: StructType): Array[sources.Filter] = { + filters.filter(checkFilterRefs(_, schema)) } private def zip[A, B](a: Option[A], b: Option[B]): Option[(A, B)] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala index e0531c49ff180..690d66908e613 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -95,10 +95,8 @@ case class CSVScan( this.copy(partitionFilters = partitionFilters) override def equals(obj: Any): Boolean = obj match { - case o: CSVScan => - fileIndex == o.fileIndex && dataSchema == o.dataSchema && - readDataSchema == o.readDataSchema && readPartitionSchema == o.readPartitionSchema && - options == o.options && equivalentFilters(pushedFilters, o.pushedFilters) + case c: CSVScan => super.equals(c) && dataSchema == c.dataSchema && options == c.options && + equivalentFilters(pushedFilters, c.pushedFilters) case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala index ae2c1f4036c3e..81a234e254000 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala @@ -48,8 +48,10 @@ case class CSVScanBuilder( private var _pushedFilters: Array[Filter] = Array.empty override def pushFilters(filters: Array[Filter]): Array[Filter] = { - _pushedFilters = filters - CSVFilters.unsupportedFilters(filters) + if (sparkSession.sessionState.conf.csvFilterPushDown) { + _pushedFilters = CSVFilters.pushedFilters(filters, dataSchema) + } + filters } override def pushedFilters(): Array[Filter] = _pushedFilters From 711a703b32d32f2ef1d7b95cae07af88535b4aaf Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 13 Jan 2020 11:20:26 +0300 Subject: [PATCH 33/43] Apply filters w/o refs at pos 0 only --- .../apache/spark/sql/catalyst/csv/CSVFilters.scala | 13 +++++++------ .../spark/sql/catalyst/csv/CSVFiltersSuite.scala | 6 ++++++ 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 16fe382dece32..28a03ab17f0dd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -78,17 +78,18 @@ class CSVFilters( for (filter <- filters) { val expr = CSVFilters.filterToExpression(filter, toRef) val refs = filter.references - if (refs.isEmpty) { + val index = if (refs.isEmpty) { // For example, AlwaysTrue and AlwaysFalse doesn't have any references - for (i <- 0 until len) { - groupedExprs(i) ++= expr - } + // Filters w/o refs always return the same result. Taking into account + // that predicates are combined via And, we can apply such filters only + // once at the position 0. + 0 } else { // readSchema must contain attributes of all filters. // Accordingly, fieldIndex() returns a valid index always. - val index = refs.map(readSchema.fieldIndex).max - groupedExprs(index) ++= expr + refs.map(readSchema.fieldIndex).max } + groupedExprs(index) ++= expr } for (i <- 0 until len) { if (!groupedExprs(i).isEmpty) { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index a9478bd950c07..956c3e3c9e068 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -116,6 +116,12 @@ class CSVFiltersSuite extends SparkFunSuite { check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) check(filters = Seq(AlwaysTrue), row = InternalRow(1), pos = 0, skip = false) check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) + check( + filters = Seq(sources.LessThan("d", 10), sources.AlwaysFalse), + requiredSchema = "i INTEGER, d DOUBLE", + row = InternalRow(1, 3.14), + pos = 0, + skip = true) check( filters = Seq(sources.EqualTo("i", 10)), row = InternalRow(10, 3.14), From 4a25815e66dff72c99f1f181628cf5b4d45dda15 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 13 Jan 2020 11:47:13 +0300 Subject: [PATCH 34/43] Fix build error --- .../spark/sql/execution/datasources/csv/CSVBenchmark.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala index eb4150ee3a655..e2abb39c986a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -317,12 +317,12 @@ object CSVBenchmark extends SqlBasedBenchmark { } benchmark.addCase(s"w/o filters", numIters) { _ => - toNoop(readback) + readback.noop() } def withFilter(configEnabled: Boolean): Unit = { withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> configEnabled.toString()) { - toNoop(readback.filter($"key" === 0)) + readback.filter($"key" === 0).noop() } } From c03ae069d738c6aa526cc1a1216d079bc8b5ec3e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 13 Jan 2020 12:19:51 +0300 Subject: [PATCH 35/43] Put literal filters in front of others --- .../spark/sql/catalyst/csv/CSVFilters.scala | 17 ++++++++++++----- .../sql/catalyst/csv/CSVFiltersSuite.scala | 2 +- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 28a03ab17f0dd..7222f9f050348 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -74,9 +74,8 @@ class CSVFilters( val len = readSchema.fields.length val groupedPredicates = Array.fill[BasePredicate](len)(null) if (SQLConf.get.csvFilterPushDown) { - val groupedExprs = Array.fill(len)(Seq.empty[Expression]) + val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) for (filter <- filters) { - val expr = CSVFilters.filterToExpression(filter, toRef) val refs = filter.references val index = if (refs.isEmpty) { // For example, AlwaysTrue and AlwaysFalse doesn't have any references @@ -89,11 +88,19 @@ class CSVFilters( // Accordingly, fieldIndex() returns a valid index always. refs.map(readSchema.fieldIndex).max } - groupedExprs(index) ++= expr + groupedFilters(index) :+= filter + } + if (len > 0 && !groupedFilters(0).isEmpty) { + // We assume that filters w/o refs like AlwaysTrue and AlwaysFalse + // can be evaluated faster that others. We put them in front of others. + val (literals, others) = groupedFilters(0).partition(_.references.isEmpty) + groupedFilters(0) = literals ++ others } for (i <- 0 until len) { - if (!groupedExprs(i).isEmpty) { - val reducedExpr = groupedExprs(i).reduce(And) + if (!groupedFilters(i).isEmpty) { + val reducedExpr = groupedFilters(i) + .flatMap(CSVFilters.filterToExpression(_, toRef)) + .reduce(And) groupedPredicates(i) = Predicate.create(reducedExpr) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index 956c3e3c9e068..9268877964398 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -117,7 +117,7 @@ class CSVFiltersSuite extends SparkFunSuite { check(filters = Seq(AlwaysTrue), row = InternalRow(1), pos = 0, skip = false) check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) check( - filters = Seq(sources.LessThan("d", 10), sources.AlwaysFalse), + filters = Seq(sources.EqualTo("i", 1), sources.LessThan("d", 10), sources.AlwaysFalse), requiredSchema = "i INTEGER, d DOUBLE", row = InternalRow(1, 3.14), pos = 0, From 18389b0c85e5f933d1a1bcad4c17638f596b0e9e Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 13 Jan 2020 16:24:26 +0300 Subject: [PATCH 36/43] Test more options/modes in the end-to-end test --- .../execution/datasources/csv/CSVSuite.scala | 33 +++++++++++-------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 4ace7c02cbda7..a5bf47337f800 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -2206,10 +2206,10 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } test("filters push down") { - Seq(true, false).foreach { multiLine => + Seq(true, false).foreach { filterPushdown => Seq(true, false).foreach { columnPruning => withSQLConf( - SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString, SQLConf.CSV_PARSER_COLUMN_PRUNING.key -> columnPruning.toString) { withTempPath { path => @@ -2220,18 +2220,23 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { s"def,2,$t").toDF("data") .repartition(1) .write.text(path.getAbsolutePath) - val readback = spark.read - .option("header", true) - .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") - .option("multiLine", multiLine) - .schema("c0 string, c1 integer, c2 timestamp") - .csv(path.getAbsolutePath) - .where($"c1" === 2) - .select($"c2") - // count() pushes empty schema. This checks handling of a filter - // which refers to not existed field. - assert(readback.count() === 1) - checkAnswer(readback, Row(Timestamp.valueOf(t))) + Seq(true, false).foreach { multiLine => + Seq("PERMISSIVE", "DROPMALFORMED", "FAILFAST").foreach { mode => + val readback = spark.read + .option("mode", mode) + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .option("multiLine", multiLine) + .schema("c0 string, c1 integer, c2 timestamp") + .csv(path.getAbsolutePath) + .where($"c1" === 2) + .select($"c2") + // count() pushes empty schema. This checks handling of a filter + // which refers to not existed field. + assert(readback.count() === 1) + checkAnswer(readback, Row(Timestamp.valueOf(t))) + } + } } } } From e302fa466641999646596dd774e1566805a1ff36 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 13 Jan 2020 17:30:45 +0300 Subject: [PATCH 37/43] Bug fix: malformed input + permissive mode + columnNameOfCorruptRecord option --- .../sql/catalyst/csv/UnivocityParser.scala | 18 ++++----- .../execution/datasources/csv/CSVSuite.scala | 39 ++++++++++++++++++- 2 files changed, 47 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index bf0f90aab21d2..b1bf5e9a2a12b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -283,15 +283,15 @@ class UnivocityParser( i += 1 } - var skipValueConversion = false + var skipRow = false var badRecordException: Option[Throwable] = None i = 0 - while (!skipValueConversion && i < parsedSchema.length) { + while (!skipRow && i < parsedSchema.length) { try { val convertedValue = valueConverters(i).apply(getToken(tokens, i)) parsedRow(i) = convertedValue if (csvFilters.skipRow(parsedRow, i)) { - skipValueConversion = true + skipRow = true } else { val requiredIndex = parsedToRequiredIndex(i) if (requiredIndex != -1) { @@ -300,20 +300,20 @@ class UnivocityParser( } } catch { case NonFatal(e) => - badRecordException = Some(e) - skipValueConversion = true + badRecordException = badRecordException.orElse(Some(e)) + requiredSingleRow.setNullAt(i) } i += 1 } - if (skipValueConversion) { + if (skipRow) { + noRows + } else { if (badRecordException.isDefined) { throw BadRecordException( () => getCurrentInput, () => requiredRow.headOption, badRecordException.get) } else { - noRows + requiredRow } - } else { - requiredRow } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index a5bf47337f800..80f5bb4c86377 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -36,7 +36,7 @@ import org.apache.log4j.{AppenderSkeleton, LogManager} import org.apache.log4j.spi.LoggingEvent import org.apache.spark.{SparkException, TestUtils} -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, Column, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession @@ -2242,4 +2242,41 @@ class CSVSuite extends QueryTest with SharedSparkSession with TestCsvData { } } } + + test("filters push down - malformed input in PERMISSIVE mode") { + val invalidTs = "2019-123-14 20:35:30" + val invalidRow = s"0,$invalidTs,999" + val validTs = "2019-12-14 20:35:30" + Seq(true, false).foreach { filterPushdown => + withSQLConf(SQLConf.CSV_FILTER_PUSHDOWN_ENABLED.key -> filterPushdown.toString) { + withTempPath { path => + Seq( + "c0,c1,c2", + invalidRow, + s"1,$validTs,999").toDF("data") + .repartition(1) + .write.text(path.getAbsolutePath) + def checkReadback(condition: Column, expected: Seq[Row]): Unit = { + val readback = spark.read + .option("mode", "PERMISSIVE") + .option("columnNameOfCorruptRecord", "c3") + .option("header", true) + .option("timestampFormat", "uuuu-MM-dd HH:mm:ss") + .schema("c0 integer, c1 timestamp, c2 integer, c3 string") + .csv(path.getAbsolutePath) + .where(condition) + .select($"c0", $"c1", $"c3") + checkAnswer(readback, expected) + } + + checkReadback( + condition = $"c2" === 999, + expected = Seq(Row(0, null, invalidRow), Row(1, Timestamp.valueOf(validTs), null))) + checkReadback( + condition = $"c2" === 999 && $"c1" > "1970-01-01 00:00:00", + expected = Seq(Row(1, Timestamp.valueOf(validTs), null))) + } + } + } + } } From 96e95542a831d1ed23fedfd037e23511fcfb0521 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Mon, 13 Jan 2020 18:20:31 +0300 Subject: [PATCH 38/43] Remove unnecessary setNullAt --- .../org/apache/spark/sql/catalyst/csv/UnivocityParser.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index b1bf5e9a2a12b..85a99199e6c49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -301,7 +301,6 @@ class UnivocityParser( } catch { case NonFatal(e) => badRecordException = badRecordException.orElse(Some(e)) - requiredSingleRow.setNullAt(i) } i += 1 } From 1be5534e0454498eef975c91b4e8696932a99ebc Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 11:15:09 +0300 Subject: [PATCH 39/43] Remove checkFilters() --- .../org/apache/spark/sql/catalyst/csv/CSVFilters.scala | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index 7222f9f050348..fb322eb936fc4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -40,7 +40,9 @@ class CSVFilters( dataSchema: StructType, requiredSchema: StructType, columnPruning: Boolean) { - require(checkFilters(), "All filters must be applicable to the data schema.") + + require(filters.forall(CSVFilters.checkFilterRefs(_, dataSchema)), + "All filters must be applicable to the data schema.") /** * The schema to read from the underlying CSV parser. @@ -129,11 +131,6 @@ class CSVFilters( BoundReference(readSchema.fieldIndex(attr), field.dataType, field.nullable) } } - - // Checks that all filters refer to an field in the data schema - private def checkFilters(): Boolean = { - filters.forall(CSVFilters.checkFilterRefs(_, dataSchema)) - } } object CSVFilters { From 9217536093f58060209c7a2c0067a5319be05b41 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 11:16:29 +0300 Subject: [PATCH 40/43] Remove private[sql] for parsedSchema --- .../org/apache/spark/sql/catalyst/csv/UnivocityParser.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 85a99199e6c49..0dd33286c6afd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -65,7 +65,7 @@ class UnivocityParser( requiredSchema, options.columnPruning) - private[sql] val parsedSchema = csvFilters.readSchema + val parsedSchema = csvFilters.readSchema // Mapping of field indexes of `parsedSchema` to indexes of `requiredSchema`. // It returns -1 if `requiredSchema` doesn't contain a field from `parsedSchema`. From 15c9648a547d6afd4e5f0dbe834047a37f0b7ce1 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 14 Jan 2020 21:36:01 +0300 Subject: [PATCH 41/43] Simplify code assuming that requireSchema contains all filter refs --- .../spark/sql/catalyst/csv/CSVFilters.scala | 38 ++--------- .../sql/catalyst/csv/UnivocityParser.scala | 67 ++++++------------- .../sql/catalyst/csv/CSVFiltersSuite.scala | 54 ++------------- .../catalyst/csv/UnivocityParserSuite.scala | 13 ++-- .../datasources/csv/CSVFileFormat.scala | 10 +-- .../v2/csv/CSVPartitionReaderFactory.scala | 5 +- 6 files changed, 45 insertions(+), 142 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala index fb322eb936fc4..b50a76a496556 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVFilters.scala @@ -31,35 +31,9 @@ import org.apache.spark.sql.types.{BooleanType, StructType} * converted from parsed CSV fields. * * @param filters The filters pushed down to CSV datasource. - * @param dataSchema The full schema with all fields in CSV files. * @param requiredSchema The schema with only fields requested by the upper layer. - * @param columnPruning true if CSV parser can read sub-set of columns otherwise false. */ -class CSVFilters( - filters: Seq[sources.Filter], - dataSchema: StructType, - requiredSchema: StructType, - columnPruning: Boolean) { - - require(filters.forall(CSVFilters.checkFilterRefs(_, dataSchema)), - "All filters must be applicable to the data schema.") - - /** - * The schema to read from the underlying CSV parser. - * It combines the required schema and the fields referenced by filters. - */ - val readSchema: StructType = { - if (columnPruning) { - val refs = filters.flatMap(_.references).toSet - val readFields = dataSchema.filter { field => - requiredSchema.contains(field) || refs.contains(field.name) - } - StructType(readFields) - } else { - dataSchema - } - } - +class CSVFilters(filters: Seq[sources.Filter], requiredSchema: StructType) { /** * Converted filters to predicates and grouped by maximum field index * in the read schema. For example, if an filter refers to 2 attributes @@ -73,7 +47,7 @@ class CSVFilters( * by the `And` expression. */ private val predicates: Array[BasePredicate] = { - val len = readSchema.fields.length + val len = requiredSchema.fields.length val groupedPredicates = Array.fill[BasePredicate](len)(null) if (SQLConf.get.csvFilterPushDown) { val groupedFilters = Array.fill(len)(Seq.empty[sources.Filter]) @@ -88,7 +62,7 @@ class CSVFilters( } else { // readSchema must contain attributes of all filters. // Accordingly, fieldIndex() returns a valid index always. - refs.map(readSchema.fieldIndex).max + refs.map(requiredSchema.fieldIndex).max } groupedFilters(index) :+= filter } @@ -126,9 +100,9 @@ class CSVFilters( // Finds a filter attribute in the read schema and converts it to a `BoundReference` private def toRef(attr: String): Option[BoundReference] = { - readSchema.getFieldIndex(attr).map { index => - val field = readSchema(index) - BoundReference(readSchema.fieldIndex(attr), field.dataType, field.nullable) + requiredSchema.getFieldIndex(attr).map { index => + val field = requiredSchema(index) + BoundReference(requiredSchema.fieldIndex(attr), field.dataType, field.nullable) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index 0dd33286c6afd..288179fc480da 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -59,30 +59,13 @@ class UnivocityParser( // A `ValueConverter` is responsible for converting the given value to a desired type. private type ValueConverter = String => Any - private val csvFilters = new CSVFilters( - filters, - dataSchema, - requiredSchema, - options.columnPruning) - - val parsedSchema = csvFilters.readSchema - - // Mapping of field indexes of `parsedSchema` to indexes of `requiredSchema`. - // It returns -1 if `requiredSchema` doesn't contain a field from `parsedSchema`. - private val parsedToRequiredIndex: Array[Int] = { - val arr = Array.fill(parsedSchema.length)(-1) - for { - readIndex <- 0 until parsedSchema.length - reqIndex <- requiredSchema.getFieldIndex(parsedSchema(readIndex).name) - } { - arr(readIndex) = reqIndex - } - arr - } - // This index is used to reorder parsed tokens private val tokenIndexArr = - parsedSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray + requiredSchema.map(f => java.lang.Integer.valueOf(dataSchema.indexOf(f))).toArray + + // When column pruning is enabled, the parser only parses the required columns based on + // their positions in the data schema. + private val parsedSchema = if (options.columnPruning) requiredSchema else dataSchema val tokenizer = { val parserSetting = options.asParserSettings @@ -95,9 +78,6 @@ class UnivocityParser( new CsvParser(parserSetting) } - // The row is used as a temporary placeholder of parsed and converted values. - // It is needed for applying the pushdown filters. - private val parsedRow = new GenericInternalRow(parsedSchema.length) // Pre-allocated Seq to avoid the overhead of the seq builder. private val requiredRow = Seq(new GenericInternalRow(requiredSchema.length)) // Pre-allocated empty sequence returned when the parsed row cannot pass filters. @@ -113,6 +93,8 @@ class UnivocityParser( options.zoneId, options.locale) + private val csvFilters = new CSVFilters(filters, requiredSchema) + // Retrieve the raw record string. private def getCurrentInput: UTF8String = { UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) @@ -138,7 +120,7 @@ class UnivocityParser( // // output row - ["A", 2] private val valueConverters: Array[ValueConverter] = { - parsedSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray + requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray } private val decimalParser = ExprUtils.getDecimalParser(options.locale) @@ -221,7 +203,7 @@ class UnivocityParser( } } - private val doParse = if (options.columnPruning && parsedSchema.isEmpty) { + private val doParse = if (options.columnPruning && requiredSchema.isEmpty) { // If `columnPruning` enabled and partition attributes scanned only, // `schema` gets empty. (_: String) => Seq(InternalRow.empty) @@ -273,34 +255,27 @@ class UnivocityParser( } else { // When the length of the returned tokens is identical to the length of the parsed schema, // we just need to: - // 1. Convert the tokens that correspond to the parsed schema. - // 2. Apply the pushdown filters to `parsedRow`. - // 3. Convert `parsedRow` to `requiredRow` by stripping non-required fields. + // 1. Convert the tokens that correspond to the required schema. + // 2. Apply the pushdown filters to `requiredRow`. var i = 0 - val requiredSingleRow = requiredRow.head - while (i < requiredSchema.length) { - requiredSingleRow.setNullAt(i) - i += 1 - } - + val row = requiredRow.head var skipRow = false var badRecordException: Option[Throwable] = None - i = 0 - while (!skipRow && i < parsedSchema.length) { + while (i < requiredSchema.length) { try { - val convertedValue = valueConverters(i).apply(getToken(tokens, i)) - parsedRow(i) = convertedValue - if (csvFilters.skipRow(parsedRow, i)) { - skipRow = true - } else { - val requiredIndex = parsedToRequiredIndex(i) - if (requiredIndex != -1) { - requiredSingleRow(requiredIndex) = convertedValue + if (!skipRow) { + row(i) = valueConverters(i).apply(getToken(tokens, i)) + if (csvFilters.skipRow(row, i)) { + skipRow = true } } + if (skipRow) { + row.setNullAt(i) + } } catch { case NonFatal(e) => badRecordException = badRecordException.orElse(Some(e)) + row.setNullAt(i) } i += 1 } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala index 9268877964398..499bbaf452aee 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVFiltersSuite.scala @@ -56,61 +56,15 @@ class CSVFiltersSuite extends SparkFunSuite { case _ => StructType.fromDDL(str) } - test("read schema is based on required schema and filters") { - def check( - dataSchema: String = "i INTEGER, d DOUBLE, s STRING", - requiredSchema: String = "s STRING", - columnPruning: Boolean = true, - filters: Seq[Filter], - expected: String): Unit = { - val csvFilters = new CSVFilters( - filters, - getSchema(dataSchema), - getSchema(requiredSchema), - columnPruning) - assert(csvFilters.readSchema === getSchema(expected)) - } - - check(filters = Seq(), expected = "s STRING") - check(filters = Seq(sources.EqualTo("d", 3.14)), expected = "d DOUBLE, s STRING") - check( - filters = Seq(sources.EqualTo("d", 3.14)), - columnPruning = false, - expected = "i INTEGER, d DOUBLE, s STRING") - check( - filters = Seq(sources.And(sources.EqualTo("d", 3.14), sources.StringEndsWith("s", "a"))), - expected = "d DOUBLE, s STRING") - check( - filters = Seq( - sources.And(sources.EqualTo("d", 3.14), sources.StringEndsWith("s", "a")), - sources.GreaterThan("i", 100)), - expected = "i INTEGER, d DOUBLE, s STRING") - - try { - check(filters = Seq(sources.EqualTo("invalid", 3.14)), expected = "d DOUBLE, s STRING") - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("All filters must be applicable to the data schema")) - } - } - test("skipping rows") { def check( - dataSchema: String = "i INTEGER, d DOUBLE, s STRING", - requiredSchema: String = "d DOUBLE", + requiredSchema: String = "i INTEGER, d DOUBLE", filters: Seq[Filter], row: InternalRow, pos: Int, skip: Boolean): Unit = { - Seq(true, false).foreach { columnPruning => - val csvFilters = new CSVFilters( - filters, - getSchema(dataSchema), - getSchema(requiredSchema), - columnPruning) - assert(csvFilters.skipRow(row, pos) === skip) - } + val csvFilters = new CSVFilters(filters, getSchema(requiredSchema)) + assert(csvFilters.skipRow(row, pos) === skip) } check(filters = Seq(), row = InternalRow(3.14), pos = 0, skip = false) @@ -118,7 +72,6 @@ class CSVFiltersSuite extends SparkFunSuite { check(filters = Seq(AlwaysFalse), row = InternalRow(1), pos = 0, skip = true) check( filters = Seq(sources.EqualTo("i", 1), sources.LessThan("d", 10), sources.AlwaysFalse), - requiredSchema = "i INTEGER, d DOUBLE", row = InternalRow(1, 3.14), pos = 0, skip = true) @@ -170,6 +123,7 @@ class CSVFiltersSuite extends SparkFunSuite { Seq(filters1 -> false, filters2 -> true).foreach { case (filters, skip) => for (p <- 0 until 3) { check( + requiredSchema = "i INTEGER, d DOUBLE, s STRING", filters = filters, row = InternalRow(10, 3.14, UTF8String.fromString("abc")), pos = p, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index ba2aefa9d3d85..b4b733f1350b3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -296,27 +296,24 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { check(filters = Seq(), expected = Seq(InternalRow(1))) check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) check(filters = Seq(EqualTo("i", 2)), expected = Seq()) - check(filters = Seq(StringStartsWith("s", "b")), expected = Seq()) + check(requiredSchema = "s STRING", filters = Seq(StringStartsWith("s", "b")), expected = Seq()) check( requiredSchema = "i INTEGER, s STRING", filters = Seq(StringStartsWith("s", "a")), expected = Seq(InternalRow(1, UTF8String.fromString("a")))) - check( - requiredSchema = "", - filters = Seq(LessThan("i", 10)), - expected = Seq(InternalRow.empty)) check( input = "1,a,3.14", dataSchema = "i INTEGER, s STRING, d DOUBLE", + requiredSchema = "i INTEGER, d DOUBLE", filters = Seq(EqualTo("d", 3.14)), - expected = Seq(InternalRow(1))) + expected = Seq(InternalRow(1, 3.14))) try { check(filters = Seq(EqualTo("invalid attr", 1)), expected = Seq()) fail("Expected to throw an exception for the invalid input") } catch { case e: IllegalArgumentException => - assert(e.getMessage.contains("All filters must be applicable to the data schema")) + assert(e.getMessage.contains("invalid attr does not exist")) } try { @@ -328,7 +325,7 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { fail("Expected to throw an exception for the invalid input") } catch { case e: IllegalArgumentException => - assert(e.getMessage.contains("All filters must be applicable to the data schema")) + assert(e.getMessage.contains("i does not exist")) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index aa552764e7eda..cbf9d2bac7ceb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -17,16 +17,13 @@ package org.apache.spark.sql.execution.datasources.csv -import java.nio.charset.Charset - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce._ -import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityGenerator, UnivocityParser} +import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ @@ -126,6 +123,8 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { "df.filter($\"_corrupt_record\".isNotNull).count()." ) } + val columnPruning = sparkSession.sessionState.conf.csvColumnPruning + (file: PartitionedFile) => { val conf = broadcastedHadoopConf.value.value val actualDataSchema = StructType( @@ -137,9 +136,10 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { actualRequiredSchema, parsedOptions, filters) + val schema = if (columnPruning) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( - parser.parsedSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) CSVDataSource(parsedOptions).readFile( conf, file, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala index 4f3d309faeb4b..31d31bd43f453 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -46,6 +46,8 @@ case class CSVPartitionReaderFactory( partitionSchema: StructType, parsedOptions: CSVOptions, filters: Seq[Filter]) extends FilePartitionReaderFactory { + private val columnPruning = sqlConf.csvColumnPruning + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { val conf = broadcastedConf.value.value val actualDataSchema = StructType( @@ -57,9 +59,10 @@ case class CSVPartitionReaderFactory( actualReadDataSchema, parsedOptions, filters) + val schema = if (columnPruning) actualReadDataSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( - parser.parsedSchema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) val iter = CSVDataSource(parsedOptions).readFile( conf, file, From df3043936dfe72a67812897b0fd4a16ce28955ef Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 15 Jan 2020 09:54:07 +0300 Subject: [PATCH 42/43] Use intercept in UnivocityParserSuite --- .../catalyst/csv/UnivocityParserSuite.scala | 18 ++++++------------ 1 file changed, 6 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index b4b733f1350b3..475d66f484d9a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -308,24 +308,18 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { filters = Seq(EqualTo("d", 3.14)), expected = Seq(InternalRow(1, 3.14))) - try { + val errMsg = intercept[IllegalArgumentException] { check(filters = Seq(EqualTo("invalid attr", 1)), expected = Seq()) - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("invalid attr does not exist")) - } + }.getMessage + assert(errMsg.contains("invalid attr does not exist")) - try { + val errMsg2 = intercept[IllegalArgumentException] { check( dataSchema = "", requiredSchema = "", filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow.empty)) - fail("Expected to throw an exception for the invalid input") - } catch { - case e: IllegalArgumentException => - assert(e.getMessage.contains("i does not exist")) - } + }.getMessage + assert(errMsg2.contains("i does not exist")) } } From 06be01355f266df805113aac4e3eeb952ca42ea4 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 15 Jan 2020 09:59:15 +0300 Subject: [PATCH 43/43] Remove nested getSchema() in UnivocityParserSuite --- .../catalyst/csv/UnivocityParserSuite.scala | 31 ++++++++----------- 1 file changed, 13 insertions(+), 18 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala index 475d66f484d9a..bd4b2529f8b92 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.catalyst.util.DateTimeConstants._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.sources.{EqualTo, Filter, LessThan, StringStartsWith} +import org.apache.spark.sql.sources.{EqualTo, Filter, StringStartsWith} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -273,21 +273,13 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { test("skipping rows using pushdown filters") { def check( input: String = "1,a", - dataSchema: String = "i INTEGER, s STRING", - requiredSchema: String = "i INTEGER", + dataSchema: StructType = StructType.fromDDL("i INTEGER, s STRING"), + requiredSchema: StructType = StructType.fromDDL("i INTEGER"), filters: Seq[Filter], expected: Seq[InternalRow]): Unit = { - def getSchema(str: String): StructType = str match { - case "" => new StructType() - case _ => StructType.fromDDL(str) - } Seq(false, true).foreach { columnPruning => val options = new CSVOptions(Map.empty[String, String], columnPruning, "GMT") - val parser = new UnivocityParser( - getSchema(dataSchema), - getSchema(requiredSchema), - options, - filters) + val parser = new UnivocityParser(dataSchema, requiredSchema, options, filters) val actual = parser.parse(input) assert(actual === expected) } @@ -296,15 +288,18 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { check(filters = Seq(), expected = Seq(InternalRow(1))) check(filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow(1))) check(filters = Seq(EqualTo("i", 2)), expected = Seq()) - check(requiredSchema = "s STRING", filters = Seq(StringStartsWith("s", "b")), expected = Seq()) check( - requiredSchema = "i INTEGER, s STRING", + requiredSchema = StructType.fromDDL("s STRING"), + filters = Seq(StringStartsWith("s", "b")), + expected = Seq()) + check( + requiredSchema = StructType.fromDDL("i INTEGER, s STRING"), filters = Seq(StringStartsWith("s", "a")), expected = Seq(InternalRow(1, UTF8String.fromString("a")))) check( input = "1,a,3.14", - dataSchema = "i INTEGER, s STRING, d DOUBLE", - requiredSchema = "i INTEGER, d DOUBLE", + dataSchema = StructType.fromDDL("i INTEGER, s STRING, d DOUBLE"), + requiredSchema = StructType.fromDDL("i INTEGER, d DOUBLE"), filters = Seq(EqualTo("d", 3.14)), expected = Seq(InternalRow(1, 3.14))) @@ -315,8 +310,8 @@ class UnivocityParserSuite extends SparkFunSuite with SQLHelper { val errMsg2 = intercept[IllegalArgumentException] { check( - dataSchema = "", - requiredSchema = "", + dataSchema = new StructType(), + requiredSchema = new StructType(), filters = Seq(EqualTo("i", 1)), expected = Seq(InternalRow.empty)) }.getMessage