[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-15 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileMeta.scala
##
@@ -0,0 +1,49 @@
+/*
+ * 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.orc
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.orc.OrcFile
+import org.apache.orc.impl.{OrcTail, ReaderImpl}
+
+import org.apache.spark.sql.execution.datasources.{FileMeta, 
FileMetaCacheManager, FileMetaKey}
+import org.apache.spark.util.Utils
+
+private[sql] case class OrcFileMetaKey(path: Path, configuration: 
Configuration)
+  extends FileMetaKey {
+  override def getFileMeta: OrcFileMeta = OrcFileMeta(path, configuration)
+}
+
+private[sql] case class OrcFileMeta(tail: OrcTail) extends FileMeta
+
+private[sql] object OrcFileMeta {
+  def apply(path: Path, conf: Configuration): OrcFileMeta = {
+val fs = path.getFileSystem(conf)
+val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+Utils.tryWithResource(new ReaderImpl(path, readerOptions)) { fileReader =>
+  new OrcFileMeta(new OrcTail(fileReader.getFileTail, 
fileReader.getSerializedFileFooter))

Review comment:
   a74c793 deleted the definition of `ForTailCacheReader` and use `new 
OrcTail(fileReader.getFileTail, fileReader.getSerializedFileFooter)` to 
construct cached orc tail, Is this correct @dongjoon-hyun 




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+private[sql] object FileMetaCacheManager extends Logging {

Review comment:
   30df269 fix this

##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##
@@ -0,0 +1,83 @@
+/*
+ * 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.
+ */
+private[sql] 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)
+
+  def get(dataFile: FileMetaKey): FileMeta = cache.get(dataFile)

Review comment:
   30df269 fix this




-- 
This is an automated message from the Apache Git Service.
To respond to the mess

[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileMeta.scala
##
@@ -0,0 +1,49 @@
+/*
+ * 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.orc
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.orc.OrcFile
+import org.apache.orc.impl.{OrcTail, ReaderImpl}
+
+import org.apache.spark.sql.execution.datasources.{FileMeta, 
FileMetaCacheManager, FileMetaKey}
+import org.apache.spark.util.Utils
+
+case class OrcFileMetaKey(path: Path, configuration: Configuration)
+  extends FileMetaKey {
+  override def getFileMeta: OrcFileMeta = OrcFileMeta(path, configuration)
+}
+
+case class OrcFileMeta(tail: OrcTail) extends FileMeta
+
+object OrcFileMeta {
+  def apply(path: Path, conf: Configuration): OrcFileMeta = {
+val fs = path.getFileSystem(conf)
+val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+Utils.tryWithResource(new ReaderImpl(path, readerOptions)) { fileReader =>
+  new OrcFileMeta(new OrcTail(fileReader.getFileTail, 
fileReader.getSerializedFileFooter))
+}
+  }
+
+  def readTailFromCache(path: Path, conf: Configuration): OrcTail =
+readTailFromCache(OrcFileMetaKey(path, conf))

Review comment:
   No, read files only occurs when cache missing
   
   
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileMeta.scala
##
@@ -0,0 +1,49 @@
+/*
+ * 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.orc
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.orc.OrcFile
+import org.apache.orc.impl.{OrcTail, ReaderImpl}
+
+import org.apache.spark.sql.execution.datasources.{FileMeta, 
FileMetaCacheManager, FileMetaKey}
+import org.apache.spark.util.Utils
+
+case class OrcFileMetaKey(path: Path, configuration: Configuration)
+  extends FileMetaKey {
+  override def getFileMeta: OrcFileMeta = OrcFileMeta(path, configuration)
+}
+
+case class OrcFileMeta(tail: OrcTail) extends FileMeta
+
+object OrcFileMeta {
+  def apply(path: Path, conf: Configuration): OrcFileMeta = {
+val fs = path.getFileSystem(conf)
+val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+Utils.tryWithResource(new ReaderImpl(path, readerOptions)) { fileReader =>
+  new OrcFileMeta(new OrcTail(fileReader.getFileTail, 
fileReader.getSerializedFileFooter))
+}
+  }
+
+  def readTailFromCache(path: Path, conf: Configuration): OrcTail =
+readTailFromCache(OrcFileMetaKey(path, conf))

Review comment:
   Let me check this




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -158,6 +169,10 @@ case class OrcPartitionReaderFactory(
   val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId)
 
   val batchReader = new OrcColumnarBatchReader(capacity)
+  if (orcMetaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)

Review comment:
   No, 
   ```
   def readTailFromCache(path: Path, conf: Configuration): OrcTail =
readTailFromCache(OrcFileMetaKey(path, conf))
   ```
   
   equals 
   
   ```
   def readTailFromCache(path: Path, conf: Configuration): OrcTail =
FileMetaCacheManager.get(OrcFileMetaKey(path, 
conf)).asInstanceOf[OrcFileMeta].tail
   ```
   
   And `OrcFileMeta.apply` will trigger file read but `OrcFileMetaKey.apply`  
does not trigger file read




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -158,6 +169,10 @@ case class OrcPartitionReaderFactory(
   val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId)
 
   val batchReader = new OrcColumnarBatchReader(capacity)
+  if (orcMetaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)

Review comment:
   42d2bfd change 
   
   ```
   def readTailFromCache(path: Path, conf: Configuration): OrcTail =
readTailFromCache(OrcFileMetaKey(path, conf))
   ```
   to
   
   ```
   def readTailFromCache(path: Path, conf: Configuration): OrcTail =
FileMetaCacheManager.get(OrcFileMetaKey(path, 
conf)).asInstanceOf[OrcFileMeta].tail
   ```
   
   and remove a redundant method
   
   
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileMetaCacheManager.scala
##
@@ -0,0 +1,90 @@
+/*
+ * 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 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(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()

Review comment:
   need use `private` ? @dongjoon-hyun 




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
##
@@ -742,6 +742,43 @@ abstract class OrcQuerySuite extends OrcQueryTest with 
SharedSparkSession {
   }
 }
   }
+
+  test("SPARK-36516: simple select queries with file meta cache") {
+withSQLConf(SQLConf.FILE_META_CACHE_ORC_ENABLED.key -> "true") {
+  import org.scalatest.PrivateMethodTester._
+  import com.github.benmanes.caffeine.cache.stats.CacheStats

Review comment:
   Import only here, because only this case is using them




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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(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 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(fileMeteKey: FileMetaKey): FileMeta = cache.get(fileMeteKey)
+
+  /**
+   * This is visible for testing.
+   */
+  def cacheStats: CacheStats = cache.stats()
+
+  /**
+   * This is visible for testing.
+   */
+  def cleanUp(): Unit = cache.cleanUp()

Review comment:
   0e6c52b change to use `PrivateMethodTester `




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala
##
@@ -158,6 +169,10 @@ case class OrcPartitionReaderFactory(
   val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId)
 
   val batchReader = new OrcColumnarBatchReader(capacity)
+  if (orcMetaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)

Review comment:
   Are there any other better naming suggestions?
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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:
   This is a full path, such as `hdfs://nn/path0/path1/xxx.orc` It is an 
orc file. Do we need to re-check its type under the same path?




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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_ORC_ENABLED = 
buildConf("spark.sql.fileMetaCache.orc.enabled")
+.doc("To indicate if enable orc 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)

Review comment:
   ok




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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:
   Add `key.getClass.equals(other.getClass)`?  This is a full path, such as 
`hdfs://nn/path0/path1/xxx.orc` It is an orc file. Do we need to re-check its 
type under the same path?




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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:
   Add `key.getClass.equals(other.getClass)` or other comparison 
conditions?  This is a full path, such as `hdfs://nn/path0/path1/xxx.orc` It is 
an orc file. Do we need to re-check its type under the same path?




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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:
   Add `key.getClass.equals(other.getClass)` or other comparison 
conditions?  This is a full path, such as `hdfs://nn/path0/path1/xxx.orc`, do 
we need to re-check its type under the same path?




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   This is a very good problem. If we want to handle it well, we need a 
relatively more complex design, such as adding a command to trigger the 
relevant cleaning of all executors when the file changes.
   
   However, if the file changes are not perceived by spark, I think wrong data 
will be read here




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   This is a very good question. If we want to handle it well, we need a 
relatively more complex design, such as adding a command to trigger the 
relevant cleaning of all executors when the file changes.
   
   However, if the file changes are not perceived by spark, I think wrong data 
will be read here




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   This is a very good question!!! If we want to handle it well, we need a 
relatively more complex design, such as adding a command to trigger the 
relevant cleaning of all executors when the file changes.
   
   However, if the file changes are not perceived by spark, I think wrong data 
will be read here




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala
##
@@ -742,6 +742,43 @@ abstract class OrcQuerySuite extends OrcQueryTest with 
SharedSparkSession {
   }
 }
   }
+
+  test("SPARK-36516: simple select queries with file meta cache") {
+withSQLConf(SQLConf.FILE_META_CACHE_ORC_ENABLED.key -> "true") {
+  import org.scalatest.PrivateMethodTester._
+  import com.github.benmanes.caffeine.cache.stats.CacheStats

Review comment:
   OK




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   If the file name has the timestamp, I think we don't have to worry too 
much. The names of the new file and the old file are different and  they can 
ensure that they don't read the wrong data.
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   If it is manually replaced and the file has the same name and the 
corresponding file meta exists in the cache, an incorrect file meta will be 
used to read the data. If the data reading fails, the job will fail. But if the 
data reading happens to be successful, the job will read the wrong data.
   
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   If it is manually file replaced and the file has the same name and the 
corresponding file meta exists in the cache, an incorrect file meta will be 
used to read the data. If the data reading fails, the job will fail. But if the 
data reading happens to be successful, the job will read the wrong data.
   
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   In fact, even if there is no `FileMetaCache`, there is a similar risk in 
manually replace files with same name, because the offset and length  of 
`PartitionedFile` maybe don't match after manually replace.




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   This is a very good question!!! ~~If we want to handle it well, we need 
a relatively more complex design, such as adding a command to trigger the 
relevant cleaning of all executors when the file changes.
   
   However, if the file changes are not perceived by spark, I think wrong data 
will be read here~~
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   This is a very good question!!! ~~If we want to handle it well, we need 
a relatively more complex design, such as adding a command to trigger the 
relevant cleaning of all executors when the file changes.~~
   
   However, if the file changes are not perceived by spark, I think wrong data 
will be read here~~
   
   

##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   This is a very good question!!! ~~If we want to handle it well, we need 
a relatively more complex design, such as adding a command to trigger the 
relevant cleaning of all executors when the file changes.~~
   
   ~~However, if the file changes are not perceived by spark, I think wrong 
data will be read here~~
   
   




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala
##
@@ -154,11 +155,18 @@ class OrcFileFormat
 
 (file: PartitionedFile) => {
   val conf = broadcastedConf.value.value
+  val metaCacheEnabled =
+conf.getBoolean(SQLConf.FILE_META_CACHE_ORC_ENABLED.key, false)
 
   val filePath = new Path(new URI(file.filePath))
 
   val fs = filePath.getFileSystem(conf)
-  val readerOptions = OrcFile.readerOptions(conf).filesystem(fs)
+  val readerOptions = if (metaCacheEnabled) {
+val tail = OrcFileMeta.readTailFromCache(filePath, conf)
+OrcFile.readerOptions(conf).filesystem(fs).orcTail(tail)

Review comment:
   In fact, even if there is no `FileMetaCache`, there is a similar risk in 
manually replace files with same name, because the offset and length  of 
`PartitionedFile` maybe don't match after manually replace for a running job




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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_ORC_ENABLED = 
buildConf("spark.sql.fileMetaCache.orc.enabled")
+.doc("To indicate if enable orc 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)

Review comment:
   ebb7e0b rename config name




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] [spark] LuciferYang commented on a change in pull request #33748: [SPARK-36516][SQL] Add File Metadata cache support for Orc

2021-08-16 Thread GitBox


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



##
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_ORC_ENABLED = 
buildConf("spark.sql.fileMetaCache.orc.enabled")
+.doc("To indicate if enable orc 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)

Review comment:
   ebb7e0b rename this config entry




-- 
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: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org