maropu commented on a change in pull request #28841: URL: https://github.com/apache/spark/pull/28841#discussion_r468943144
########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathfilters/PathFilterStrategy.scala ########## @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.pathfilters + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileIndexFilter + +abstract class PathFilterStrategy(sparkSession: SparkSession, + conf: Configuration, + options: Map[String, String]) + extends FileIndexFilter { Review comment: Why do we need the class fields? Could we write it like this?; ``` trait PathFilterStrategy extends FileIndexFilter { def accept(path: Path): Boolean def accept(fileStatus: FileStatus): Boolean def strategy(): String } ``` ########## File path: docs/sql-data-sources-generic-options.md ########## @@ -119,3 +119,47 @@ To load all files recursively, you can use: {% include_example recursive_file_lookup r/RSparkSQLExample.R %} </div> </div> + +### Modification Time Path Filters +`modifiedBefore` and `modifiedAfter` are options that can be +applied together or separately in order to achieve greater +granularity over which files may load during a Spark batch query. <br/> +<br/>When the `timeZone` option is present, modified timestamps will be +interpreted according to the specified zone. When a timezone option +is not provided, modified timestamps will be interpreted according +to the default zone specified within the Spark configuration. Without +any timezone configuration, modified timestamps are interpreted as UTC. + +`modifiedBefore` will only allow files having last modified +timestamps occurring before the specified time to load. For example, +when`modifiedBefore`has the timestamp `2020-06-01T12:00:00` applied, + all files modified after that time will not be considered when loading from a file data source.<br/><br/> +`modifiedAfter` only allows files having last modified timestamps +occurring after the specified timestamp. For example, when`modifiedAfter` +has the timestamp `2020-06-01T12:00:00` applied, only files modified after + this time will be eligible when loading from a file data source. + <br/><br/> Review comment: Why did you use the tag `<br/>` instead of line breaks? ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathfilters/PathFilterStrategies.scala ########## @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.pathfilters + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.datasources.{FileIndexFilter, PathFilterObject} + + +case object PathFilterStrategies { Review comment: Could you move this object into `pathFilters.scala`, too? ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( + DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), + SQLConf.get.sessionLocalTimeZone) + + /* Implicitly defaults to UTC if unable to parse */ + lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId) + lazy val timeString: UTF8String = + UTF8String.fromString(options.apply(strategy())) + + def thresholdTime(): Long = { + DateTimeUtils + .stringToTimestamp(timeString, timeZone.toZoneId) + .getOrElse(throw new AnalysisException( + s"The timestamp provided for the '${strategy()}'" + + s" option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss'. " + + s" Provided timestamp: " + + s"${options.apply(strategy())}")) + } + + def localTime(micros: Long): Long = + DateTimeUtils.fromUTCTime(micros, timeZoneId) Review comment: nit: ` def localTime(micros: Long): Long = DateTimeUtils.fromUTCTime(micros, timeZoneId)` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( + DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), + SQLConf.get.sessionLocalTimeZone) + + /* Implicitly defaults to UTC if unable to parse */ + lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId) + lazy val timeString: UTF8String = + UTF8String.fromString(options.apply(strategy())) + + def thresholdTime(): Long = { + DateTimeUtils + .stringToTimestamp(timeString, timeZone.toZoneId) + .getOrElse(throw new AnalysisException( + s"The timestamp provided for the '${strategy()}'" + + s" option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss'. " + + s" Provided timestamp: " + + s"${options.apply(strategy())}")) + } + + def localTime(micros: Long): Long = + DateTimeUtils.fromUTCTime(micros, timeZoneId) + + def accept(fileStatus: FileStatus): Boolean + def accept(path: Path): Boolean + def strategy(): String +} + +/** + * Filter used to determine whether file was modified + * before the provided timestamp. + * + @param sparkSession SparkSession + @param hadoopConf Hadoop Configuration object + @param options Map containing options + */ +class ModifiedBeforeFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends ModifiedDateFilter(sparkSession, hadoopConf, options) + with FileIndexFilter { Review comment: ``` class ModifiedBeforeFilter( sparkSession: SparkSession, hadoopConf: Configuration, options: CaseInsensitiveMap[String]) extends ModifiedDateFilter(sparkSession, hadoopConf, options) with FileIndexFilter { override def accept(fileStatus: FileStatus): Boolean = ``` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathfilters/PathFilterFactory.scala ########## @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.pathfilters + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.datasources.{FileIndexFilter, ModifiedAfterFilter, ModifiedBeforeFilter, PathGlobFilter} + +object PathFilterFactory { Review comment: Could you move this object into `pathFilters.scala`, too? ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when Review comment: nit: you don't need the jira ID here; ``` /** * Provide modifiedAfter and modifiedBefore options when * filtering from a batch-based file data source. * * Example Usages ``` ########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PathFilterSuite.scala ########## @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.time.LocalDateTime + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.{stringToFile, CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PathFilterSuite + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { Review comment: nit: ``` class PathFilterSuite extends QueryTest with SharedSparkSession with AdaptiveSparkPlanHelper { import testImplicits._ ``` ########## File path: docs/sql-data-sources-generic-options.md ########## @@ -119,3 +119,47 @@ To load all files recursively, you can use: {% include_example recursive_file_lookup r/RSparkSQLExample.R %} </div> </div> + +### Modification Time Path Filters +`modifiedBefore` and `modifiedAfter` are options that can be +applied together or separately in order to achieve greater +granularity over which files may load during a Spark batch query. <br/> +<br/>When the `timeZone` option is present, modified timestamps will be +interpreted according to the specified zone. When a timezone option +is not provided, modified timestamps will be interpreted according +to the default zone specified within the Spark configuration. Without +any timezone configuration, modified timestamps are interpreted as UTC. + +`modifiedBefore` will only allow files having last modified +timestamps occurring before the specified time to load. For example, +when`modifiedBefore`has the timestamp `2020-06-01T12:00:00` applied, + all files modified after that time will not be considered when loading from a file data source.<br/><br/> +`modifiedAfter` only allows files having last modified timestamps +occurring after the specified timestamp. For example, when`modifiedAfter` Review comment: ". For example, when`modifiedAfter`" -> '. For example, when `modifiedAfter`' ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathfilters/PathFilterStrategy.scala ########## @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.pathfilters + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileIndexFilter + +abstract class PathFilterStrategy(sparkSession: SparkSession, Review comment: Could you move this class into `pathFilters.scala`, too? ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( Review comment: ``` abstract class ModifiedDateFilter( sparkSession: SparkSession, hadoopConf: Configuration, options: CaseInsensitiveMap[String]) extends PathFilterStrategy(sparkSession, hadoopConf, options) { lazy val timeZoneId: String = options.getOrElse( DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), SQLConf.get.sessionLocalTimeZone) ``` ########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PathFilterSuite.scala ########## @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.time.LocalDateTime + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.{stringToFile, CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PathFilterSuite + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + import testImplicits._ + + test("SPARK-31962: when modifiedAfter specified with a past date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file = new File(dir, "file1.csv") + stringToFile(file, "text") + file.setLastModified(DateTimeUtils.currentTimestamp()) + val df = spark.read + .option("modifiedAfter", "2019-05-10T01:11:00") + .format("csv") + .load(path.toString) + assert(df.count() == 1) + } + } + } + + test("SPARK-31962: when modifiedBefore specified with a future date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file = new File(dir, "file1.csv") + stringToFile(file, "text") + val df = spark.read + .option("modifiedBefore", "2090-05-10T01:11:00") + .format("csv") + .load(path.toString) + assert(df.count() == 1) + } + } + } + + test("SPARK-31962: when modifiedBefore specified with a past date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file = new File(dir, "file1.csv") + stringToFile(file, "text") + file.setLastModified(DateTimeUtils.currentTimestamp()) + val msg = intercept[AnalysisException] { + spark.read + .option("modifiedBefore", "1984-05-01T01:00:00") + .format("csv") + .load(path.toString) + }.getMessage + assert(msg.contains("Unable to infer schema for CSV")) + } + } + } + + test("SPARK-31962: when modifiedAfter specified with a past date, " + Review comment: indent.... ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( + DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), + SQLConf.get.sessionLocalTimeZone) + + /* Implicitly defaults to UTC if unable to parse */ + lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId) + lazy val timeString: UTF8String = + UTF8String.fromString(options.apply(strategy())) + + def thresholdTime(): Long = { + DateTimeUtils + .stringToTimestamp(timeString, timeZone.toZoneId) + .getOrElse(throw new AnalysisException( + s"The timestamp provided for the '${strategy()}'" + + s" option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss'. " + + s" Provided timestamp: " + + s"${options.apply(strategy())}")) Review comment: ``` def thresholdTime(): Long = { DateTimeUtils .stringToTimestamp(timeString, timeZone.toZoneId) .getOrElse(throw new AnalysisException(s"The timestamp provided for the '$strategy' " + "option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss'. " + s"Provided timestamp: ${options.apply(strategy)}")) } ``` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( + DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), + SQLConf.get.sessionLocalTimeZone) + + /* Implicitly defaults to UTC if unable to parse */ + lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId) + lazy val timeString: UTF8String = + UTF8String.fromString(options.apply(strategy())) Review comment: nit: you don't the line break: ` lazy val timeString: UTF8String = UTF8String.fromString(options.apply(strategy()))` ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathfilters/PathFilterFactory.scala ########## @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.pathfilters + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.datasources.{FileIndexFilter, ModifiedAfterFilter, ModifiedBeforeFilter, PathGlobFilter} + +object PathFilterFactory { + PathFilterStrategies.register(ModifiedAfterFilter) + PathFilterStrategies.register(ModifiedBeforeFilter) + PathFilterStrategies.register(PathGlobFilter) Review comment: Why do we need this register/cache mechanism? We cannot just return singletons for the filter implementations? ########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PathFilterSuite.scala ########## @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.time.LocalDateTime + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.{stringToFile, CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PathFilterSuite + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { Review comment: Also, I think you don't need `AdaptiveSparkPlanHelper`. ########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PathFilterSuite.scala ########## @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.time.LocalDateTime + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.{stringToFile, CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PathFilterSuite + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + import testImplicits._ + + test("SPARK-31962: when modifiedAfter specified with a past date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { Review comment: I think you don't set the timezone here cuz its already been fixed in the base class; https://github.com/apache/spark/blob/e7c1204f6c15fdc98d7fc8068d96572e0e868436/core/src/test/scala/org/apache/spark/SparkFunSuite.scala#L74-L77 ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( + DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), + SQLConf.get.sessionLocalTimeZone) + + /* Implicitly defaults to UTC if unable to parse */ + lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId) + lazy val timeString: UTF8String = + UTF8String.fromString(options.apply(strategy())) + + def thresholdTime(): Long = { + DateTimeUtils + .stringToTimestamp(timeString, timeZone.toZoneId) + .getOrElse(throw new AnalysisException( + s"The timestamp provided for the '${strategy()}'" + + s" option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss'. " + + s" Provided timestamp: " + + s"${options.apply(strategy())}")) + } + + def localTime(micros: Long): Long = + DateTimeUtils.fromUTCTime(micros, timeZoneId) + + def accept(fileStatus: FileStatus): Boolean + def accept(path: Path): Boolean + def strategy(): String +} + +/** + * Filter used to determine whether file was modified + * before the provided timestamp. + * + @param sparkSession SparkSession + @param hadoopConf Hadoop Configuration object + @param options Map containing options + */ +class ModifiedBeforeFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends ModifiedDateFilter(sparkSession, hadoopConf, options) + with FileIndexFilter { + override def accept(fileStatus: FileStatus): Boolean = + /* We standardize on microseconds wherever possible */ + thresholdTime - localTime( + DateTimeUtils + /* getModificationTime returns in milliseconds */ + .millisToMicros(fileStatus.getModificationTime)) > 0 Review comment: nit format: ``` override def accept(fileStatus: FileStatus): Boolean = { // Standardizes on microseconds wherever possible and `getModificationTime` // returns in milliseconds. val fileTimestamp = localTime(DateTimeUtils.millisToMicros(fileStatus.getModificationTime)) thresholdTime - fileTimestamp > 0 } ``` ########## File path: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/PathFilterSuite.scala ########## @@ -0,0 +1,557 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.io.File +import java.time.LocalDateTime + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.catalyst.util.{stringToFile, CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSparkSession + +class PathFilterSuite + extends QueryTest + with SharedSparkSession + with AdaptiveSparkPlanHelper { + import testImplicits._ + + test("SPARK-31962: when modifiedAfter specified with a past date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file = new File(dir, "file1.csv") + stringToFile(file, "text") + file.setLastModified(DateTimeUtils.currentTimestamp()) + val df = spark.read + .option("modifiedAfter", "2019-05-10T01:11:00") + .format("csv") + .load(path.toString) + assert(df.count() == 1) + } + } + } + + test("SPARK-31962: when modifiedBefore specified with a future date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file = new File(dir, "file1.csv") + stringToFile(file, "text") + val df = spark.read + .option("modifiedBefore", "2090-05-10T01:11:00") + .format("csv") + .load(path.toString) + assert(df.count() == 1) + } + } + } + + test("SPARK-31962: when modifiedBefore specified with a past date") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file = new File(dir, "file1.csv") + stringToFile(file, "text") + file.setLastModified(DateTimeUtils.currentTimestamp()) + val msg = intercept[AnalysisException] { + spark.read + .option("modifiedBefore", "1984-05-01T01:00:00") + .format("csv") + .load(path.toString) + }.getMessage + assert(msg.contains("Unable to infer schema for CSV")) + } + } + } + + test("SPARK-31962: when modifiedAfter specified with a past date, " + + "multiple files, one valid") { + withTempDir { dir => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + val path = new Path(dir.getCanonicalPath) + val file1 = new File(dir, "file1.csv") + val file2 = new File(dir, "file2.csv") + stringToFile(file1, "text") + stringToFile(file2, "text") + file1.setLastModified(DateTimeUtils.currentTimestamp()) + file2.setLastModified(0) + val df = spark.read + .option("modifiedAfter", "2019-05-10T01:11:00") + .format("csv") + .load(path.toString) + assert(df.count() == 1) + } + } + } + test("SPARK-31962: when modifiedAfter specified with a past date, " + Review comment: add a space between test units... ########## File path: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/pathFilters.scala ########## @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.{Locale, TimeZone} + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path, PathFilter} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.pathfilters.PathFilterStrategy +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.types.UTF8String + + +/** + * SPARK-31962: Provide modifiedAfter and modifiedBefore options when + * filtering from a batch-based file data source. + * + * Example Usages + * Load all CSV files modified after date: + * spark.read.format("csv").option("modifiedAfter","2020-06-15T05:00:00").load() + * + * Load all CSV files modified before date: + * spark.read.format("csv").option("modifiedBefore","2020-06-15T05:00:00").load() + * + * Load all CSV files modified between two dates: + * spark.read.format("csv").option("modifiedAfter","2019-01-15T05:00:00") + * .option("modifiedBefore","2020-06-15T05:00:00").load() + * +@param sparkSession SparkSession +@param hadoopConf Hadoop Configuration object +@param options Map containing options + */ +abstract class ModifiedDateFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends PathFilterStrategy(sparkSession, hadoopConf, options) { + lazy val timeZoneId: String = options.getOrElse( + DateTimeUtils.TIMEZONE_OPTION.toLowerCase(Locale.ROOT), + SQLConf.get.sessionLocalTimeZone) + + /* Implicitly defaults to UTC if unable to parse */ + lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId) + lazy val timeString: UTF8String = + UTF8String.fromString(options.apply(strategy())) + + def thresholdTime(): Long = { + DateTimeUtils + .stringToTimestamp(timeString, timeZone.toZoneId) + .getOrElse(throw new AnalysisException( + s"The timestamp provided for the '${strategy()}'" + + s" option is invalid. The expected format is 'YYYY-MM-DDTHH:mm:ss'. " + + s" Provided timestamp: " + + s"${options.apply(strategy())}")) + } + + def localTime(micros: Long): Long = + DateTimeUtils.fromUTCTime(micros, timeZoneId) + + def accept(fileStatus: FileStatus): Boolean + def accept(path: Path): Boolean + def strategy(): String +} + +/** + * Filter used to determine whether file was modified + * before the provided timestamp. + * + @param sparkSession SparkSession + @param hadoopConf Hadoop Configuration object + @param options Map containing options + */ +class ModifiedBeforeFilter(sparkSession: SparkSession, + hadoopConf: Configuration, + options: CaseInsensitiveMap[String]) + extends ModifiedDateFilter(sparkSession, hadoopConf, options) + with FileIndexFilter { + override def accept(fileStatus: FileStatus): Boolean = + /* We standardize on microseconds wherever possible */ + thresholdTime - localTime( + DateTimeUtils + /* getModificationTime returns in milliseconds */ + .millisToMicros(fileStatus.getModificationTime)) > 0 + + override def accept(path: Path): Boolean = true + override def strategy(): String = "modifiedbefore" +} +case object ModifiedBeforeFilter extends PathFilterObject { Review comment: add a single space between classes. ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
