sunchao commented on a change in pull request #30483:
URL: https://github.com/apache/spark/pull/30483#discussion_r691697807



##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -967,6 +967,20 @@ object SQLConf {
     .booleanConf
     .createWithDefault(false)
 
+  val FILE_META_CACHE_PARQUET_ENABLED = 
buildConf("spark.sql.fileMetaCache.parquet.enabled")
+    .doc("To indicate if enable parquet file meta cache, it is recommended to 
enabled " +

Review comment:
       hmm curious whether this can help if your Spark queries is running as 
separate Spark jobs, where each of them may use different executors.

##########
File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
##########
@@ -967,6 +967,20 @@ object SQLConf {
     .booleanConf
     .createWithDefault(false)
 
+  val FILE_META_CACHE_PARQUET_ENABLED = 
buildConf("spark.sql.fileMetaCache.parquet.enabled")
+    .doc("To indicate if enable parquet 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 =

Review comment:
       nit: maybe `FILE_META_CACHE_TTL_SINCE_LAST_ACCESS_SEC` and 
`spark.sql.fileMetaCache.ttlSinceLastAccessSec` so it's easier to know that the 
unit is second?

##########
File path: 
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
##########
@@ -77,28 +82,31 @@
 
   protected ParquetFileReader reader;
 
+  protected ParquetMetadata cachedFooter;
+
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext 
taskAttemptContext)
       throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     FileSplit split = (FileSplit) inputSplit;
     this.file = split.getPath();
 
-    ParquetReadOptions options = HadoopReadOptions
-      .builder(configuration)
-      .withRange(split.getStart(), split.getStart() + split.getLength())
-      .build();
-    this.reader = new ParquetFileReader(HadoopInputFile.fromPath(file, 
configuration), options);
-    this.fileSchema = reader.getFileMetaData().getSchema();
-    Map<String, String> fileMetadata = 
reader.getFileMetaData().getKeyValueMetaData();
+    ParquetMetadata footer =
+      readFooterByRange(configuration, split.getStart(), split.getStart() + 
split.getLength());
+    this.fileSchema = footer.getFileMetaData().getSchema();
+    FilterCompat.Filter filter = ParquetInputFormat.getFilter(configuration);
+    List<BlockMetaData> blocks =
+      RowGroupFilter.filterRowGroups(filter, footer.getBlocks(), fileSchema);

Review comment:
       does this apply all the filter levels? e.g., stats, dictionary, and 
bloom filter.

##########
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##########
@@ -0,0 +1,87 @@
+/*
+ * 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(entry: FileMetaKey): FileMeta = {
+      logDebug(s"Loading Data File Meta ${entry.path}")
+      entry.getFileMeta
+    }
+  }
+
+  private lazy val ttlTime =
+    SparkEnv.get.conf.get(SQLConf.FILE_META_CACHE_TTL_SINCE_LAST_ACCESS)
+
+  private lazy val cache = Caffeine
+    .newBuilder()
+    .expireAfterAccess(ttlTime, TimeUnit.SECONDS)
+    .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(dataFile: FileMetaKey): FileMeta = cache.get(dataFile)
+
+  /**
+   * This is visible for testing.
+   */
+  def cacheStats: CacheStats = cache.stats()
+
+  /**
+   * This is visible for testing.
+   */
+  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)

Review comment:
       What if the same file gets replaced? how do we invalidate the cache? 
this is very common from my experience, e.g., Hive overwrite a partition.




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