LuciferYang commented on a change in pull request #33748:
URL: https://github.com/apache/spark/pull/33748#discussion_r690867601



##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.concurrent.TimeUnit
+
+import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine}
+import com.github.benmanes.caffeine.cache.stats.CacheStats
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A singleton Cache Manager to caching file meta. We cache these file metas 
in order to speed up
+ * iterated queries over the same dataset. Otherwise, each query would have to 
hit remote storage
+ * in order to fetch file meta before read files.
+ *
+ * We should implement the corresponding `FileMetaKey` for a specific file 
format, for example
+ * `ParquetFileMetaKey` or `OrcFileMetaKey`. By default, the file path is used 
as the identification
+ * of the `FileMetaKey` and the `getFileMeta` method of `FileMetaKey` is used 
to return the file
+ * meta of the corresponding file format.
+ */
+object FileMetaCacheManager extends Logging {
+
+  private lazy val cacheLoader = new CacheLoader[FileMetaKey, FileMeta]() {
+    override def load(fileMetaKey: FileMetaKey): FileMeta = {
+      logDebug(s"Loading Data File Meta ${fileMetaKey.path}")
+      fileMetaKey.getFileMeta
+    }
+  }
+
+  private lazy val ttlTime =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS)
+
+  private lazy val maximumSize =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_MAXIMUM_SIZE)
+
+  private lazy val cache = Caffeine
+    .newBuilder()
+    .expireAfterAccess(ttlTime, TimeUnit.SECONDS)
+    .maximumSize(maximumSize)
+    .recordStats()
+    .build[FileMetaKey, FileMeta](cacheLoader)
+
+  /**
+   * Returns the `FileMeta` associated with the `FileMetaKey` in the 
`FileMetaCacheManager`,
+   * obtaining that the `FileMeta` from `cacheLoader.load(FileMetaKey)` if 
necessary.
+   */
+  def get(fileMeteKey: FileMetaKey): FileMeta = cache.get(fileMeteKey)
+
+  /**
+   * Return current snapshot of FileMeta Cache's cumulative statistics
+   * include cache hitCount, missCount and so on.
+   * This method is only called when testing now.
+   */
+  private def cacheStats: CacheStats = cache.stats()
+
+  /**
+   * Use to cleanUp entries in the FileMeta Cache.
+   * This method is only called when testing now.
+   */
+  private def cleanUp(): Unit = cache.cleanUp()
+}
+
+abstract class FileMetaKey {
+  def path: Path
+  def configuration: Configuration
+  def getFileMeta: FileMeta
+  override def hashCode(): Int = path.hashCode
+  override def equals(other: Any): Boolean = other match {
+    case df: FileMetaKey => path.equals(df.path)
+    case _ => false

Review comment:
       The current design is still needed. We need to be able to construct a 
specific `FileMeta(like OrcFileMeta or ParquetFileMeta)` through `FileMetaKey 
(path and configuration)`

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.concurrent.TimeUnit
+
+import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine}
+import com.github.benmanes.caffeine.cache.stats.CacheStats
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A singleton Cache Manager to caching file meta. We cache these file metas 
in order to speed up
+ * iterated queries over the same dataset. Otherwise, each query would have to 
hit remote storage
+ * in order to fetch file meta before read files.
+ *
+ * We should implement the corresponding `FileMetaKey` for a specific file 
format, for example
+ * `ParquetFileMetaKey` or `OrcFileMetaKey`. By default, the file path is used 
as the identification
+ * of the `FileMetaKey` and the `getFileMeta` method of `FileMetaKey` is used 
to return the file
+ * meta of the corresponding file format.
+ */
+object FileMetaCacheManager extends Logging {
+
+  private lazy val cacheLoader = new CacheLoader[FileMetaKey, FileMeta]() {
+    override def load(fileMetaKey: FileMetaKey): FileMeta = {
+      logDebug(s"Loading Data File Meta ${fileMetaKey.path}")
+      fileMetaKey.getFileMeta
+    }
+  }
+
+  private lazy val ttlTime =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS)
+
+  private lazy val maximumSize =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_MAXIMUM_SIZE)
+
+  private lazy val cache = Caffeine
+    .newBuilder()
+    .expireAfterAccess(ttlTime, TimeUnit.SECONDS)
+    .maximumSize(maximumSize)
+    .recordStats()
+    .build[FileMetaKey, FileMeta](cacheLoader)
+
+  /**
+   * Returns the `FileMeta` associated with the `FileMetaKey` in the 
`FileMetaCacheManager`,
+   * obtaining that the `FileMeta` from `cacheLoader.load(FileMetaKey)` if 
necessary.
+   */
+  def get(fileMeteKey: FileMetaKey): FileMeta = cache.get(fileMeteKey)
+
+  /**
+   * Return current snapshot of FileMeta Cache's cumulative statistics
+   * include cache hitCount, missCount and so on.
+   * This method is only called when testing now.
+   */
+  private def cacheStats: CacheStats = cache.stats()
+
+  /**
+   * Use to cleanUp entries in the FileMeta Cache.
+   * This method is only called when testing now.
+   */
+  private def cleanUp(): Unit = cache.cleanUp()
+}
+
+abstract class FileMetaKey {
+  def path: Path
+  def configuration: Configuration
+  def getFileMeta: FileMeta
+  override def hashCode(): Int = path.hashCode
+  override def equals(other: Any): Boolean = other match {
+    case df: FileMetaKey => path.equals(df.path)
+    case _ => false

Review comment:
       The current design is still needed. We need to be able to construct a 
specific `FileMeta(like OrcFileMeta or ParquetFileMeta)` through `FileMetaKey 
(path and configuration)`,  and specific `FileMetaKey` used to determine 
whether to read the footer of Parquet or the tail of ORC.
   
   If a unified `FileMetaKey` is used, it seems that a fileType field needs to 
be added to `FileMetaKey` to read the corresponding `FileMeta`.
   
    
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##########
@@ -154,11 +155,17 @@ class OrcFileFormat
 
     (file: PartitionedFile) => {
       val conf = broadcastedConf.value.value
+      val metaCacheEnabled = 
conf.getBoolean(SQLConf.FILE_META_CACHE_ENABLED.key, false)

Review comment:
       It seems that there is no problem here. If `useFileMetaCacheList` config 
 is used, we can move the
   ```
   val metaCacheEnabled = xxx
   ```
   out of 
   
   ```
   (file: PartitionedFile) => {
   }
   ```

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##########
@@ -154,11 +155,17 @@ class OrcFileFormat
 
     (file: PartitionedFile) => {
       val conf = broadcastedConf.value.value
+      val metaCacheEnabled = 
conf.getBoolean(SQLConf.FILE_META_CACHE_ENABLED.key, false)

Review comment:
       It seems that there is no problem here. If `useFileMetaCacheList` config 
 is used, we can move the
   ```
   val metaCacheEnabled = useFileMetaCacheList.split().contains(shortName)
   ```
   out of 
   
   ```
   (file: PartitionedFile) => {
   }
   ```
   
   to avoid do `split + contains` for each task
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##########
@@ -154,11 +155,17 @@ class OrcFileFormat
 
     (file: PartitionedFile) => {
       val conf = broadcastedConf.value.value
+      val metaCacheEnabled = 
conf.getBoolean(SQLConf.FILE_META_CACHE_ENABLED.key, false)

Review comment:
       It seems that there is no problem here. If `useFileMetaCacheList` config 
 is used, we can move the
   ```
   val metaCacheEnabled = useFileMetaCacheList.toLowerCase(Locale.ROOT)
         .split(",").map(_.trim).contains(shortName)
   ```
   out of 
   
   ```
   (file: PartitionedFile) => {
   }
   ```
   
   to avoid do `split + contains` for each task
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##########
@@ -60,6 +60,7 @@ case class OrcPartitionReaderFactory(
   private val capacity = sqlConf.orcVectorizedReaderBatchSize
   private val orcFilterPushDown = sqlConf.orcFilterPushDown
   private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
+  private val metaCacheEnabled = sqlConf.fileMetaCacheEnabled

Review comment:
       If `useFileMetaCacheList` config is used, without change the V2 API, it 
seems that it can only be hard coded as
   ```
   val metaCacheEnabled = useFileMetaCacheList.toLowerCase(Locale.ROOT)
         .split(",").map(_.trim).contains("ORC")
   ```
   
   the `formatName("ORC")` is defined `OrcTable`, we can't get it through the 
API here at present.
   
    
   
   
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.concurrent.TimeUnit
+
+import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine}
+import com.github.benmanes.caffeine.cache.stats.CacheStats
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A singleton Cache Manager to caching file meta. We cache these file metas 
in order to speed up
+ * iterated queries over the same dataset. Otherwise, each query would have to 
hit remote storage
+ * in order to fetch file meta before read files.
+ *
+ * We should implement the corresponding `FileMetaKey` for a specific file 
format, for example
+ * `ParquetFileMetaKey` or `OrcFileMetaKey`. By default, the file path is used 
as the identification
+ * of the `FileMetaKey` and the `getFileMeta` method of `FileMetaKey` is used 
to return the file
+ * meta of the corresponding file format.
+ */
+object FileMetaCacheManager extends Logging {
+
+  private lazy val cacheLoader = new CacheLoader[FileMetaKey, FileMeta]() {
+    override def load(fileMetaKey: FileMetaKey): FileMeta = {
+      logDebug(s"Loading Data File Meta ${fileMetaKey.path}")
+      fileMetaKey.getFileMeta
+    }
+  }
+
+  private lazy val ttlTime =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS)
+
+  private lazy val maximumSize =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_MAXIMUM_SIZE)
+
+  private lazy val cache = Caffeine
+    .newBuilder()
+    .expireAfterAccess(ttlTime, TimeUnit.SECONDS)
+    .maximumSize(maximumSize)
+    .recordStats()
+    .build[FileMetaKey, FileMeta](cacheLoader)
+
+  /**
+   * Returns the `FileMeta` associated with the `FileMetaKey` in the 
`FileMetaCacheManager`,
+   * obtaining that the `FileMeta` from `cacheLoader.load(FileMetaKey)` if 
necessary.
+   */
+  def get(fileMeteKey: FileMetaKey): FileMeta = cache.get(fileMeteKey)
+
+  /**
+   * Return current snapshot of FileMeta Cache's cumulative statistics
+   * include cache hitCount, missCount and so on.
+   * This method is only called when testing now.
+   */
+  private def cacheStats: CacheStats = cache.stats()
+
+  /**
+   * Use to cleanUp entries in the FileMeta Cache.
+   * This method is only called when testing now.
+   */
+  private def cleanUp(): Unit = cache.cleanUp()
+}
+
+abstract class FileMetaKey {
+  def path: Path
+  def configuration: Configuration
+  def getFileMeta: FileMeta
+  override def hashCode(): Int = path.hashCode
+  override def equals(other: Any): Boolean = other match {
+    case df: FileMetaKey => path.equals(df.path)
+    case _ => false

Review comment:
       Do you have any suggestions ? @dongjoon-hyun 

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##########
@@ -154,11 +155,17 @@ class OrcFileFormat
 
     (file: PartitionedFile) => {
       val conf = broadcastedConf.value.value
+      val metaCacheEnabled = 
conf.getBoolean(SQLConf.FILE_META_CACHE_ENABLED.key, false)

Review comment:
       > > BTW, @viirya 's suggestion about the config is a list config like 
spark.sql.sources.useV1SourceList.
   
   It seems that there is no problem here. If `useFileMetaCacheList` config  is 
used, we can move the
   ```
   val metaCacheEnabled = useFileMetaCacheList.toLowerCase(Locale.ROOT)
         .split(",").map(_.trim).contains(shortName)
   ```
   out of 
   
   ```
   (file: PartitionedFile) => {
   }
   ```
   
   to avoid do `split + contains` for each task
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##########
@@ -60,6 +60,7 @@ case class OrcPartitionReaderFactory(
   private val capacity = sqlConf.orcVectorizedReaderBatchSize
   private val orcFilterPushDown = sqlConf.orcFilterPushDown
   private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
+  private val metaCacheEnabled = sqlConf.fileMetaCacheEnabled

Review comment:
       > BTW, @viirya 's suggestion about the config is a list config like 
spark.sql.sources.useV1SourceList.
   
   @dongjoon-hyun @viirya 
   
   If `useFileMetaCacheList` config is used, without change the V2 API, it 
seems that it can only be hard coded as
   ```
   val metaCacheEnabled = useFileMetaCacheList.toLowerCase(Locale.ROOT)
         .split(",").map(_.trim).contains("ORC")
   ```
   
   the `formatName("ORC")` is defined `OrcTable`, we can't get it through the 
API here at present.
   
   
   On the other hand, this config will not have corresponding implementations 
for all build-in data format like `spark.sql.sources.useV1SourceList`, if the 
new data format is not considered,  it may only work for `Parquet` and `Orc`, 
so is it really appropriate to use list config?
   
   
   
   
   
   
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##########
@@ -60,6 +60,7 @@ case class OrcPartitionReaderFactory(
   private val capacity = sqlConf.orcVectorizedReaderBatchSize
   private val orcFilterPushDown = sqlConf.orcFilterPushDown
   private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
+  private val metaCacheEnabled = sqlConf.fileMetaCacheEnabled

Review comment:
       > BTW, @viirya 's suggestion about the config is a list config like 
spark.sql.sources.useV1SourceList.
   
   @dongjoon-hyun @viirya 
   
   If `useFileMetaCacheList` config is used, without change the V2 API, it 
seems that it can only be hard coded as
   ```
   val metaCacheEnabled = useFileMetaCacheList.toLowerCase(Locale.ROOT)
         .split(",").map(_.trim).contains("ORC")
   ```
   
   the `formatName("ORC")` is defined `OrcTable`, we can't get it through the 
API here at present.
   
   
   On the other hand, this config will not have corresponding implementations 
for all build-in data format like `spark.sql.sources.useV1SourceList`, if the 
new data format is not considered,  it may only work for `Parquet` and `Orc`, 
so are we sure we need to use a list config?
   
   
   
   
   
   
   
   
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##########
@@ -60,6 +60,7 @@ case class OrcPartitionReaderFactory(
   private val capacity = sqlConf.orcVectorizedReaderBatchSize
   private val orcFilterPushDown = sqlConf.orcFilterPushDown
   private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
+  private val metaCacheEnabled = sqlConf.fileMetaCacheEnabled

Review comment:
       > BTW, @viirya 's suggestion about the config is a list config like 
spark.sql.sources.useV1SourceList.
   
   @dongjoon-hyun @viirya 
   
   If `useFileMetaCacheList` config is used, without change the V2 API, it 
seems that it can only be hard coded as
   ```
   val metaCacheEnabled = useFileMetaCacheList.toLowerCase(Locale.ROOT)
         .split(",").map(_.trim).contains("orc")
   ```
   
   the `formatName("ORC")` is defined `OrcTable`, we can't get it through the 
API here at present.
   
   
   On the other hand, this config will not have corresponding implementations 
for all build-in data format like `spark.sql.sources.useV1SourceList`, if the 
new data format is not considered,  it may only work for `Parquet` and `Orc`, 
so are we sure we need to use a list config?
   
   
   
   
   
   
   
   
   
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##########
@@ -0,0 +1,94 @@
+/*
+ * 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.concurrent.TimeUnit
+
+import com.github.benmanes.caffeine.cache.{CacheLoader, Caffeine}
+import com.github.benmanes.caffeine.cache.stats.CacheStats
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+
+import org.apache.spark.SparkEnv
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * A singleton Cache Manager to caching file meta. We cache these file metas 
in order to speed up
+ * iterated queries over the same dataset. Otherwise, each query would have to 
hit remote storage
+ * in order to fetch file meta before read files.
+ *
+ * We should implement the corresponding `FileMetaKey` for a specific file 
format, for example
+ * `ParquetFileMetaKey` or `OrcFileMetaKey`. By default, the file path is used 
as the identification
+ * of the `FileMetaKey` and the `getFileMeta` method of `FileMetaKey` is used 
to return the file
+ * meta of the corresponding file format.
+ */
+object FileMetaCacheManager extends Logging {
+
+  private lazy val cacheLoader = new CacheLoader[FileMetaKey, FileMeta]() {
+    override def load(fileMetaKey: FileMetaKey): FileMeta = {
+      logDebug(s"Loading Data File Meta ${fileMetaKey.path}")
+      fileMetaKey.getFileMeta
+    }
+  }
+
+  private lazy val ttlTime =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS)
+
+  private lazy val maximumSize =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_MAXIMUM_SIZE)
+
+  private lazy val cache = Caffeine
+    .newBuilder()
+    .expireAfterAccess(ttlTime, TimeUnit.SECONDS)
+    .maximumSize(maximumSize)
+    .recordStats()
+    .build[FileMetaKey, FileMeta](cacheLoader)
+
+  /**
+   * Returns the `FileMeta` associated with the `FileMetaKey` in the 
`FileMetaCacheManager`,
+   * obtaining that the `FileMeta` from `cacheLoader.load(FileMetaKey)` if 
necessary.
+   */
+  def get(fileMeteKey: FileMetaKey): FileMeta = cache.get(fileMeteKey)
+
+  /**
+   * Return current snapshot of FileMeta Cache's cumulative statistics
+   * include cache hitCount, missCount and so on.
+   * This method is only called when testing now.
+   */
+  private def cacheStats: CacheStats = cache.stats()
+
+  /**
+   * Use to cleanUp entries in the FileMeta Cache.
+   * This method is only called when testing now.
+   */
+  private def cleanUp(): Unit = cache.cleanUp()
+}
+
+abstract class FileMetaKey {
+  def path: Path
+  def configuration: Configuration
+  def getFileMeta: FileMeta
+  override def hashCode(): Int = path.hashCode
+  override def equals(other: Any): Boolean = other match {
+    case df: FileMetaKey => path.equals(df.path)
+    case _ => false

Review comment:
       @viirya 3b15a82 add same type check, I think you're right
   
   

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FileMetaCacheReadBenchmark.scala
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.benchmark
+
+import java.io.File
+
+import scala.util.Random
+
+import org.apache.spark.SparkConf
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * Benchmark to measure the performance of fileMetaCache in data source read.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt: bin/spark-submit --class <this class>
+ *        --jars <spark core test jar>,<spark catalyst test jar> <spark sql 
test jar>
+ *   2. build/sbt "sql/test:runMain <this class>"
+ *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt 
"sql/test:runMain <this class>"
+ *      Results will be written to 
"benchmarks/FileMetaCacheReadBenchmark-results.txt".
+ * }}}
+ */
+object FileMetaCacheReadBenchmark extends SqlBasedBenchmark {
+
+  override def getSparkSession: SparkSession = {
+    val conf = new SparkConf()
+      .setAppName("FileMetaCacheReadBenchmark")
+      // Since `spark.master` always exists, overrides this value
+      .set("spark.master", "local[1]")
+      .setIfMissing("spark.driver.memory", "4g")
+      .setIfMissing("spark.executor.memory", "4g")
+
+    val sparkSession = SparkSession.builder().config(conf).getOrCreate()
+
+    // Set default configs. Individual cases will change them if necessary.
+    sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
+    sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
+    sparkSession.conf.set(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS.key, 
"5")
+
+    sparkSession
+  }
+
+  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
+    try f finally tableNames.foreach(spark.catalog.dropTempView)
+  }
+
+  private def prepareTable(dir: File, df: DataFrame, fileCount: Int): Unit =
+    saveAsOrcTable( df.repartition(fileCount).write, dir.getCanonicalPath + 
"/orc")
+
+  private def saveAsOrcTable(df: DataFrameWriter[Row], dir: String): Unit = {
+    df.mode("overwrite").option("compression", "snappy").orc(dir)
+    spark.read.orc(dir).createOrReplaceTempView("orcTable")
+  }
+
+  def fileScanBenchmark(values: Int, width: Int, fileCount: Int): Unit = {
+    val benchmark = new Benchmark(
+      s"Scan from $width columns with $fileCount files",
+      values,
+      output = output)
+
+    withTempPath { dir =>
+      withTempTable("t1", "orcTable") {
+        import spark.implicits._
+        val middle = width / 2
+        val selectExpr = (1 to width).map(i => s"value as c$i")
+        val dataFrame = spark.range(values).map(_ => Random.nextLong).toDF()
+        dataFrame.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
+
+        prepareTable(dir, spark.sql("SELECT * FROM t1"), fileCount)
+
+        val filter = {
+          val rows =
+            spark.sql(s"SELECT c1, count(*) FROM orcTable group by c1")
+              .collect()
+              .sortBy(r => r.getLong(1))
+          rows.head.getLong(0)
+        }
+
+        benchmark.addCase("ORC Vectorized Full Scan: fileMetaCacheEnabled = 
false") { _ =>
+          spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop()
+        }
+
+        benchmark.addCase("ORC Vectorized Full Scan: fileMetaCacheEnabled = 
true") { _ =>
+          withSQLConf(SQLConf.FILE_META_CACHE_ENABLED.key -> "true") {
+            spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop()
+          }
+        }
+
+        benchmark.addCase("ORC Vectorized Filter Scan: fileMetaCacheEnabled = 
false") { _ =>
+          spark.sql(s"SELECT sum(c$middle) FROM orcTable where c1 = 
$filter").noop()
+        }
+
+        benchmark.addCase("ORC Vectorized Filter Scan: fileMetaCacheEnabled = 
true") { _ =>
+          withSQLConf(SQLConf.FILE_META_CACHE_ENABLED.key -> "true") {
+            spark.sql(s"SELECT sum(c$middle) FROM orcTable where c1 = 
$filter").noop()
+          }
+        }
+
+        benchmark.run()
+      }
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    for (fileCount <- List(100, 500, 1000)) {
+      runBenchmark(s"Scan From $fileCount files") {
+        for (columnWidth <- List(10, 50, 100)) {
+          fileScanBenchmark(1024 * 1024 * 5, columnWidth, fileCount)

Review comment:
       2b99983 update to use a simple `countBenchmark`

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -967,6 +967,28 @@ object SQLConf {
     .booleanConf
     .createWithDefault(false)
 
+  val FILE_META_CACHE_ENABLED = buildConf("spark.sql.fileMetaCache.enabled")
+    .doc("To indicate if enable file meta cache, it is recommended to enabled 
" +
+      "this config when multiple queries are performed on the same dataset, 
default is false.")
+    .version("3.3.0")
+    .booleanConf
+    .createWithDefault(false)
+
+  val FILE_META_CACHE_TTL_SINCE_LAST_ACCESS =
+    buildConf("spark.sql.fileMetaCache.ttlSinceLastAccess")
+      .version("3.3.0")
+      .doc("Time-to-live for file metadata cache entry after last access, the 
unit is seconds.")
+      .timeConf(TimeUnit.SECONDS)
+      .createWithDefault(3600L)

Review comment:
       82ddf4f  reduce this from `3600L` to `600L` 

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FileMetaCacheReadBenchmark.scala
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.benchmark
+
+import java.io.File
+
+import scala.util.Random
+
+import org.apache.spark.SparkConf
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * Benchmark to measure the performance of fileMetaCache in data source read.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt: bin/spark-submit --class <this class>
+ *        --jars <spark core test jar>,<spark catalyst test jar> <spark sql 
test jar>
+ *   2. build/sbt "sql/test:runMain <this class>"
+ *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt 
"sql/test:runMain <this class>"
+ *      Results will be written to 
"benchmarks/FileMetaCacheReadBenchmark-results.txt".
+ * }}}
+ */
+object FileMetaCacheReadBenchmark extends SqlBasedBenchmark {
+
+  override def getSparkSession: SparkSession = {
+    val conf = new SparkConf()
+      .setAppName("FileMetaCacheReadBenchmark")
+      // Since `spark.master` always exists, overrides this value
+      .set("spark.master", "local[1]")
+      .setIfMissing("spark.driver.memory", "4g")
+      .setIfMissing("spark.executor.memory", "4g")
+
+    val sparkSession = SparkSession.builder().config(conf).getOrCreate()
+
+    // Set default configs. Individual cases will change them if necessary.
+    sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
+    sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
+    sparkSession.conf.set(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS.key, 
"5")
+
+    sparkSession
+  }
+
+  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
+    try f finally tableNames.foreach(spark.catalog.dropTempView)
+  }
+
+  private def prepareTable(dir: File, df: DataFrame, fileCount: Int): Unit =
+    saveAsOrcTable( df.repartition(fileCount).write, dir.getCanonicalPath + 
"/orc")
+
+  private def saveAsOrcTable(df: DataFrameWriter[Row], dir: String): Unit = {
+    df.mode("overwrite").option("compression", "snappy").orc(dir)
+    spark.read.orc(dir).createOrReplaceTempView("orcTable")
+  }
+
+  def fileScanBenchmark(values: Int, width: Int, fileCount: Int): Unit = {
+    val benchmark = new Benchmark(
+      s"Scan from $width columns with $fileCount files",
+      values,
+      output = output)
+
+    withTempPath { dir =>
+      withTempTable("t1", "orcTable") {
+        import spark.implicits._
+        val middle = width / 2
+        val selectExpr = (1 to width).map(i => s"value as c$i")
+        val dataFrame = spark.range(values).map(_ => Random.nextLong).toDF()
+        dataFrame.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
+
+        prepareTable(dir, spark.sql("SELECT * FROM t1"), fileCount)
+
+        val filter = {
+          val rows =
+            spark.sql(s"SELECT c1, count(*) FROM orcTable group by c1")
+              .collect()
+              .sortBy(r => r.getLong(1))
+          rows.head.getLong(0)
+        }
+
+        benchmark.addCase("ORC Vectorized Full Scan: fileMetaCacheEnabled = 
false") { _ =>
+          spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop()
+        }
+
+        benchmark.addCase("ORC Vectorized Full Scan: fileMetaCacheEnabled = 
true") { _ =>
+          withSQLConf(SQLConf.FILE_META_CACHE_ENABLED.key -> "true") {
+            spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop()
+          }
+        }
+
+        benchmark.addCase("ORC Vectorized Filter Scan: fileMetaCacheEnabled = 
false") { _ =>
+          spark.sql(s"SELECT sum(c$middle) FROM orcTable where c1 = 
$filter").noop()
+        }
+
+        benchmark.addCase("ORC Vectorized Filter Scan: fileMetaCacheEnabled = 
true") { _ =>
+          withSQLConf(SQLConf.FILE_META_CACHE_ENABLED.key -> "true") {
+            spark.sql(s"SELECT sum(c$middle) FROM orcTable where c1 = 
$filter").noop()
+          }
+        }
+
+        benchmark.run()
+      }
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    for (fileCount <- List(100, 500, 1000)) {
+      runBenchmark(s"Scan From $fileCount files") {
+        for (columnWidth <- List(10, 50, 100)) {
+          fileScanBenchmark(1024 * 1024 * 5, columnWidth, fileCount)

Review comment:
       Interestingly, I found that the data from the server is not as good as 
Apple M1, haha
   
   

##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FileMetaCacheReadBenchmark.scala
##########
@@ -0,0 +1,129 @@
+/*
+ * 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.benchmark
+
+import java.io.File
+
+import scala.util.Random
+
+import org.apache.spark.SparkConf
+import org.apache.spark.benchmark.Benchmark
+import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * Benchmark to measure the performance of fileMetaCache in data source read.
+ * To run this benchmark:
+ * {{{
+ *   1. without sbt: bin/spark-submit --class <this class>
+ *        --jars <spark core test jar>,<spark catalyst test jar> <spark sql 
test jar>
+ *   2. build/sbt "sql/test:runMain <this class>"
+ *   3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt 
"sql/test:runMain <this class>"
+ *      Results will be written to 
"benchmarks/FileMetaCacheReadBenchmark-results.txt".
+ * }}}
+ */
+object FileMetaCacheReadBenchmark extends SqlBasedBenchmark {
+
+  override def getSparkSession: SparkSession = {
+    val conf = new SparkConf()
+      .setAppName("FileMetaCacheReadBenchmark")
+      // Since `spark.master` always exists, overrides this value
+      .set("spark.master", "local[1]")
+      .setIfMissing("spark.driver.memory", "4g")
+      .setIfMissing("spark.executor.memory", "4g")
+
+    val sparkSession = SparkSession.builder().config(conf).getOrCreate()
+
+    // Set default configs. Individual cases will change them if necessary.
+    sparkSession.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true")
+    sparkSession.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true")
+    sparkSession.conf.set(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS.key, 
"5")
+
+    sparkSession
+  }
+
+  def withTempTable(tableNames: String*)(f: => Unit): Unit = {
+    try f finally tableNames.foreach(spark.catalog.dropTempView)
+  }
+
+  private def prepareTable(dir: File, df: DataFrame, fileCount: Int): Unit =
+    saveAsOrcTable( df.repartition(fileCount).write, dir.getCanonicalPath + 
"/orc")
+
+  private def saveAsOrcTable(df: DataFrameWriter[Row], dir: String): Unit = {
+    df.mode("overwrite").option("compression", "snappy").orc(dir)
+    spark.read.orc(dir).createOrReplaceTempView("orcTable")
+  }
+
+  def fileScanBenchmark(values: Int, width: Int, fileCount: Int): Unit = {
+    val benchmark = new Benchmark(
+      s"Scan from $width columns with $fileCount files",
+      values,
+      output = output)
+
+    withTempPath { dir =>
+      withTempTable("t1", "orcTable") {
+        import spark.implicits._
+        val middle = width / 2
+        val selectExpr = (1 to width).map(i => s"value as c$i")
+        val dataFrame = spark.range(values).map(_ => Random.nextLong).toDF()
+        dataFrame.selectExpr(selectExpr: _*).createOrReplaceTempView("t1")
+
+        prepareTable(dir, spark.sql("SELECT * FROM t1"), fileCount)
+
+        val filter = {
+          val rows =
+            spark.sql(s"SELECT c1, count(*) FROM orcTable group by c1")
+              .collect()
+              .sortBy(r => r.getLong(1))
+          rows.head.getLong(0)
+        }
+
+        benchmark.addCase("ORC Vectorized Full Scan: fileMetaCacheEnabled = 
false") { _ =>
+          spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop()
+        }
+
+        benchmark.addCase("ORC Vectorized Full Scan: fileMetaCacheEnabled = 
true") { _ =>
+          withSQLConf(SQLConf.FILE_META_CACHE_ENABLED.key -> "true") {
+            spark.sql(s"SELECT sum(c$middle) FROM orcTable").noop()
+          }
+        }
+
+        benchmark.addCase("ORC Vectorized Filter Scan: fileMetaCacheEnabled = 
false") { _ =>
+          spark.sql(s"SELECT sum(c$middle) FROM orcTable where c1 = 
$filter").noop()
+        }
+
+        benchmark.addCase("ORC Vectorized Filter Scan: fileMetaCacheEnabled = 
true") { _ =>
+          withSQLConf(SQLConf.FILE_META_CACHE_ENABLED.key -> "true") {
+            spark.sql(s"SELECT sum(c$middle) FROM orcTable where c1 = 
$filter").noop()
+          }
+        }
+
+        benchmark.run()
+      }
+    }
+  }
+
+  override def runBenchmarkSuite(mainArgs: Array[String]): Unit = {
+    for (fileCount <- List(100, 500, 1000)) {
+      runBenchmark(s"Scan From $fileCount files") {
+        for (columnWidth <- List(10, 50, 100)) {
+          fileScanBenchmark(1024 * 1024 * 5, columnWidth, fileCount)

Review comment:
       Interestingly, I found that the benchmark result get from x86 server is 
not as good as Apple M1
   

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##########
@@ -60,6 +60,7 @@ case class OrcPartitionReaderFactory(
   private val capacity = sqlConf.orcVectorizedReaderBatchSize
   private val orcFilterPushDown = sqlConf.orcFilterPushDown
   private val ignoreCorruptFiles = sqlConf.ignoreCorruptFiles
+  private val metaCacheEnabled = sqlConf.fileMetaCacheEnabled

Review comment:
       7153d2a change to use a list config 




-- 
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.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to