You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "yabola (via GitHub)" <gi...@apache.org> on 2023/02/09 08:48:55 UTC

[GitHub] [spark] yabola opened a new pull request, #39950: SPARK-42388 Avoid unnecessary parquet footer reads when no filters in vectorized reader

yabola opened a new pull request, #39950:
URL: https://github.com/apache/spark/pull/39950

   … vectorized reader
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'core/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1142848071


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -205,11 +212,21 @@ class ParquetFileFormat
 
       val sharedConf = broadcastedHadoopConf.value.value
 
-      lazy val footerFileMetaData =
-        ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      val fileRange = HadoopReadOptions.builder(sharedConf, split.getPath)

Review Comment:
   Yes, before I created and passed `ParquetFileReader` because I wanted to create one less `file.newStream()` in it (if  there is no filter pushdown), but it doesn’t seem to make much sense. I have changed to pass footer here.
   Please take a look, thank you!



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1138487479


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -182,6 +186,9 @@ class ParquetFileFormat
     val parquetOptions = new ParquetOptions(options, sparkSession.sessionState.conf)
     val datetimeRebaseModeInRead = parquetOptions.datetimeRebaseModeInRead
     val int96RebaseModeInRead = parquetOptions.int96RebaseModeInRead
+    // whether read the complete row groups in advance (filter the row groups later if need)
+    // this can avoid reading the footer twice (currently only optimize for vectorized read)
+    val readRowGroupsEarly = enableVectorizedReader

Review Comment:
   now only optimize for vectorized read. 
   non-vectorized reader use api `org.apache.hadoop.mapreduce.RecordReader#initialize` , I can't pass the footer parameter.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153377375


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,
+    FileSplit split,

Review Comment:
   done, I have changed to pass `PartitionedFile`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao closed pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao closed pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader
URL: https://github.com/apache/spark/pull/39950


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid unnecessary parquet footer reads when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1426639409

   @MaxGekk could you help review my PR, thanks!


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1172104789


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,16 +90,27 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+      InputSplit inputSplit,
+      TaskAttemptContext taskAttemptContext,
+      Option<ParquetMetadata> fileFooter) throws IOException, InterruptedException {

Review Comment:
   do we need to use `Option`? The argument value is always `Some`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] dongjoon-hyun commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "dongjoon-hyun (via GitHub)" <gi...@apache.org>.
dongjoon-hyun commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1434095965

   cc @sunchao , too.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1140745689


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -279,7 +301,7 @@ class ParquetFileFormat
         // Instead, we use FileScanRDD's task completion listener to close this iterator.
         val iter = new RecordReaderIterator(vectorizedReader)
         try {
-          vectorizedReader.initialize(split, hadoopAttemptContext)
+          vectorizedReader.initialize(split, hadoopAttemptContext, fileReader)

Review Comment:
   can we pass footer to `initialize` instead?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -205,11 +212,21 @@ class ParquetFileFormat
 
       val sharedConf = broadcastedHadoopConf.value.value
 
-      lazy val footerFileMetaData =
-        ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      val fileRange = HadoopReadOptions.builder(sharedConf, split.getPath)

Review Comment:
   can we add these in `ParquetFooterReader`? we may need to use `try-with-resources` clause to make sure resources are properly closed.
   
   we can just obtain the footer here and use it later for `footerFileMetaData` and pass it to `VectorizedParquetRecordReader`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid unnecessary parquet footer reads when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1425253282

   Please ignore my PR. There is something wrong..


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1471754271

   @sunchao please take a look, thank you


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1510120727

   @sunchao Thank you for your detailed review!


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1167616335


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,57 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
+import org.apache.spark.sql.execution.datasources.PartitionedFile;
 
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * Reads footer for the input Parquet file 'split'. If 'skipRowGroup' is true,
+   * this will skip reading the Parquet row group metadata.
+   *
+   * @param partitionedFile a part (i.e. "block") of a single file that should be read
+   * @param configuration hadoop configuration of file
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file split range
+   */
+  public static ParquetMetadata readFooter(
+      Configuration configuration,
+      PartitionedFile partitionedFile,
+      boolean skipRowGroup) throws IOException {
+    FileSplit split = new FileSplit(partitionedFile.toPath(), partitionedFile.start(),

Review Comment:
   nit: I think there is no need to use `FileSplit` and hence depend on `org.apache.hadoop.mapred` here. Instead we can do:
   ```java
       long start = file.start();
       long length = file.length();
       Path filePath = new Path(new URI(file.filePath().toString()));
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1101155361


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -207,11 +207,11 @@ class ParquetFileFormat
 
       lazy val footerFileMetaData =
         ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
-      val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
+      lazy val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
         footerFileMetaData.getKeyValueMetaData.get,

Review Comment:
   `footerFileMetaData` is lazy, but `datetimeRebaseSpec` causes the footer to be read immediately. 
   Actually we can avoid this unnecessary footer reads and use footer metadata in `VectorizedParquetRecordReader`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1121979988


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -296,41 +309,45 @@ class ParquetFileFormat
             throw e
         }
       } else {
-        logDebug(s"Falling back to parquet-mr")
-        // ParquetRecordReader returns InternalRow
-        val readSupport = new ParquetReadSupport(
-          convertTz,
-          enableVectorizedReader = false,
-          datetimeRebaseSpec,
-          int96RebaseSpec)
-        val reader = if (pushed.isDefined && enableRecordFilter) {
-          val parquetFilter = FilterCompat.get(pushed.get, null)
-          new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
-        } else {
-          new ParquetRecordReader[InternalRow](readSupport)
-        }
-        val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader,
-            requiredSchema)
-        val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes)
         try {
-          readerWithRowIndexes.initialize(split, hadoopAttemptContext)
-
-          val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
-          val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
-
-          if (partitionSchema.length == 0) {
-            // There is no partition columns
-            iter.map(unsafeProjection)
+          logDebug(s"Falling back to parquet-mr")
+          // ParquetRecordReader returns InternalRow
+          val readSupport = new ParquetReadSupport(
+            convertTz,
+            enableVectorizedReader = false,
+            datetimeRebaseSpec,
+            int96RebaseSpec)
+          val reader = if (pushed.isDefined && enableRecordFilter) {
+            val parquetFilter = FilterCompat.get(pushed.get, null)
+            new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
           } else {
-            val joinedRow = new JoinedRow()
-            iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
+            new ParquetRecordReader[InternalRow](readSupport)
           }
-        } catch {
-          case e: Throwable =>
-            // SPARK-23457: In case there is an exception in initialization, close the iterator to
-            // avoid leaking resources.
-            iter.close()
-            throw e
+          val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader,
+            requiredSchema)
+          val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes)
+          try {
+            readerWithRowIndexes.initialize(split, hadoopAttemptContext)
+
+            val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
+            val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+
+            if (partitionSchema.length == 0) {
+              // There is no partition columns
+              iter.map(unsafeProjection)
+            } else {
+              val joinedRow = new JoinedRow()
+              iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
+            }
+          } catch {
+            case e: Throwable =>
+              // SPARK-23457: In case there is an exception in initialization, close the iterator to
+              // avoid leaking resources.
+              iter.close()
+              throw e
+          }
+        } finally {
+          parquetReader.close()

Review Comment:
   I haven't change the code logic, just add to close parquetReader.
   Before, if the non-vectorization code path will close the reader after reading `footerFileMetaData` , it is consistent here. Please see
   [the original implementation code](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java#L53)
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1465215240

   @sunchao Hi~ Could you take a look at this PR?  I think it will be useful when there are joined tables and filter conditions are on few tables.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153492439


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,17 +90,28 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+      InputSplit inputSplit,
+      TaskAttemptContext taskAttemptContext,
+      Option<ParquetMetadata> fileFooter) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     FileSplit split = (FileSplit) inputSplit;
     this.file = split.getPath();
-
-    ParquetReadOptions options = HadoopReadOptions
-      .builder(configuration, file)
-      .withRange(split.getStart(), split.getStart() + split.getLength())
-      .withCodecFactory(new ParquetCodecFactory(configuration, 0))
-      .build();
-    ParquetFileReader fileReader = new ParquetFileReader(
-        HadoopInputFile.fromPath(file, configuration), options);
+    ParquetFileReader fileReader;
+    if (fileFooter.isDefined()) {
+      fileReader = new ParquetFileReader(configuration, file, fileFooter.get());

Review Comment:
   when we pass footer here, then we can't pass `CodecFactory` any more. Parquet will create in default `HadoopCodecs.newFactory(conf, 0)`.
   
   This may need to be solved by upgrading parquet in the future, and I will also create a pr to remove `@Deprecated`  tag in parquet



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153375489


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,
+    FileSplit split,
+    boolean skipRowGroup) throws IOException {
+    ParquetMetadataConverter.MetadataFilter filter;
+    if (skipRowGroup) {
+      filter = ParquetMetadataConverter.SKIP_ROW_GROUPS;
+    } else {
+      filter = HadoopReadOptions.builder(configuration, split.getPath())
+          .withRange(split.getStart(), split.getStart() + split.getLength())
+          .withCodecFactory(new ParquetCodecFactory(configuration, 0))

Review Comment:
   Thank you for review. Yes, if just read footer, there is no need to bring



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1550990489

   @sadikovi I have tested the scenario as you said. The smaller the row group size (footer size will be larger), the higher the PR benefits.
   Environment:  `parquet.block.size=10240` (10kb),  file size is 253.6 MB( one file has  about 25000 row groups), and I wrote 24 files.
   Before this PR:
   ![image](https://github.com/apache/spark/assets/31469905/89db510e-5622-4cbf-874f-c19c999a19d8)
   
   After this PR:
   ![image](https://github.com/apache/spark/assets/31469905/06d36af1-95da-4c0e-aa2d-35bfbea56215)
   
   
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1121979988


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -296,41 +309,45 @@ class ParquetFileFormat
             throw e
         }
       } else {
-        logDebug(s"Falling back to parquet-mr")
-        // ParquetRecordReader returns InternalRow
-        val readSupport = new ParquetReadSupport(
-          convertTz,
-          enableVectorizedReader = false,
-          datetimeRebaseSpec,
-          int96RebaseSpec)
-        val reader = if (pushed.isDefined && enableRecordFilter) {
-          val parquetFilter = FilterCompat.get(pushed.get, null)
-          new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
-        } else {
-          new ParquetRecordReader[InternalRow](readSupport)
-        }
-        val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader,
-            requiredSchema)
-        val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes)
         try {
-          readerWithRowIndexes.initialize(split, hadoopAttemptContext)
-
-          val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
-          val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
-
-          if (partitionSchema.length == 0) {
-            // There is no partition columns
-            iter.map(unsafeProjection)
+          logDebug(s"Falling back to parquet-mr")
+          // ParquetRecordReader returns InternalRow
+          val readSupport = new ParquetReadSupport(
+            convertTz,
+            enableVectorizedReader = false,
+            datetimeRebaseSpec,
+            int96RebaseSpec)
+          val reader = if (pushed.isDefined && enableRecordFilter) {
+            val parquetFilter = FilterCompat.get(pushed.get, null)
+            new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
           } else {
-            val joinedRow = new JoinedRow()
-            iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
+            new ParquetRecordReader[InternalRow](readSupport)
           }
-        } catch {
-          case e: Throwable =>
-            // SPARK-23457: In case there is an exception in initialization, close the iterator to
-            // avoid leaking resources.
-            iter.close()
-            throw e
+          val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader,
+            requiredSchema)
+          val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes)
+          try {
+            readerWithRowIndexes.initialize(split, hadoopAttemptContext)
+
+            val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
+            val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+
+            if (partitionSchema.length == 0) {
+              // There is no partition columns
+              iter.map(unsafeProjection)
+            } else {
+              val joinedRow = new JoinedRow()
+              iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
+            }
+          } catch {
+            case e: Throwable =>
+              // SPARK-23457: In case there is an exception in initialization, close the iterator to
+              // avoid leaking resources.
+              iter.close()
+              throw e
+          }
+        } finally {
+          parquetReader.close()

Review Comment:
   I haven't change the code logic, just add to close parquetReader.
   Before, if the non-vectorization code path will close the reader, it is consistent here. Please see
   [the original implementation code](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java#L53)
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1167676059


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,57 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
+import org.apache.spark.sql.execution.datasources.PartitionedFile;
 
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * Reads footer for the input Parquet file 'split'. If 'skipRowGroup' is true,
+   * this will skip reading the Parquet row group metadata.
+   *
+   * @param partitionedFile a part (i.e. "block") of a single file that should be read
+   * @param configuration hadoop configuration of file
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file split range
+   */
+  public static ParquetMetadata readFooter(
+      Configuration configuration,
+      PartitionedFile partitionedFile,
+      boolean skipRowGroup) throws IOException {
+    FileSplit split = new FileSplit(partitionedFile.toPath(), partitionedFile.start(),

Review Comment:
   Good idea, done~



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1167676059


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,57 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
+import org.apache.spark.sql.execution.datasources.PartitionedFile;
 
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * Reads footer for the input Parquet file 'split'. If 'skipRowGroup' is true,
+   * this will skip reading the Parquet row group metadata.
+   *
+   * @param partitionedFile a part (i.e. "block") of a single file that should be read
+   * @param configuration hadoop configuration of file
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file split range
+   */
+  public static ParquetMetadata readFooter(
+      Configuration configuration,
+      PartitionedFile partitionedFile,
+      boolean skipRowGroup) throws IOException {
+    FileSplit split = new FileSplit(partitionedFile.toPath(), partitionedFile.start(),

Review Comment:
   Good idea, done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1465508828

   > Sorry, it might be necessary to read footer twice if having filters. We should read schema in footer meta first to get which filters need to be pushed down. After that we set pushdown info ((https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L261) and read filtered RowGroups with the filter configuration.
   
   Hmm why? when we read the footer for the first time, the result already contain all the row groups. We just need to pass these to `ParquetFileReader`, which will apply the filters we pushed down on these row groups and return a list of filtered ones. See [here](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L769).
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1437794387

   @sunchao Thank you for your reply! Yes, I also noticed this, just before is for minimal changes.
   In the original implementation:
   The first `footerFileMetaData` use `SKIP_ROW_GROUPS` option (`SkipMetadataFilter`, return meta without rowGroup);
   The second `footerFileMetaData` use `RangeMetadataFilter`(return meta with rowGroup info).
   Actually the second `footerFileMetaData` contains all information used in the first `footerFileMetaData`(the detail implementation difference can see `ParquetMetadataConverter#readParquetMetadata`)
   
   So when in case that we need filter pushdown and also `enableVectorizedReader`, we can only create one `ParquetFileReader` and read parquet footer only once. Other situations can also be optimized to read footer.
   
   This needs to modify some more codes, do you think it is suitable?
   
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153375489


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,
+    FileSplit split,
+    boolean skipRowGroup) throws IOException {
+    ParquetMetadataConverter.MetadataFilter filter;
+    if (skipRowGroup) {
+      filter = ParquetMetadataConverter.SKIP_ROW_GROUPS;
+    } else {
+      filter = HadoopReadOptions.builder(configuration, split.getPath())
+          .withRange(split.getStart(), split.getStart() + split.getLength())
+          .withCodecFactory(new ParquetCodecFactory(configuration, 0))

Review Comment:
   Thank you, yes, if just read footer, there is no need to bring



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1155105787


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -205,11 +205,22 @@ class ParquetFileFormat
 
       val sharedConf = broadcastedHadoopConf.value.value
 
-      lazy val footerFileMetaData =
-        ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      val fileFooter = if (enableVectorizedReader) {
+        // This can avoid reading the footer twice(currently only optimize for vectorized read).

Review Comment:
   I updated comments, thank you



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1519373732

   @sadikovi Thanks for your advice. I would do an experiment in the scenario you describe. But my understanding is that reading RowGroup meta (not data) should be indispensable, otherwise RowGroup cannot be filtered? I will follow up to confirm your concerns.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1427202368

   @MaxGekk @gengliangwang If you have time, please take a look, thanks


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1152231701


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer

Review Comment:
   nit: How about:
   ```
   Reads footer for the input Parquet file 'split'.  If 'skipRowGroup' is true, this will skip reading the Parquet row group metadata.
   ```
   
   



##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,

Review Comment:
   nit: 4 space indent 



##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,
+    FileSplit split,
+    boolean skipRowGroup) throws IOException {
+    ParquetMetadataConverter.MetadataFilter filter;
+    if (skipRowGroup) {
+      filter = ParquetMetadataConverter.SKIP_ROW_GROUPS;
+    } else {
+      filter = HadoopReadOptions.builder(configuration, split.getPath())
+          .withRange(split.getStart(), split.getStart() + split.getLength())
+          .withCodecFactory(new ParquetCodecFactory(configuration, 0))

Review Comment:
   hmm is this required? we don't need the codec factory when reading footer?



##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedParquetRecordReader.java:
##########
@@ -181,6 +184,16 @@ public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptCont
     initializeInternal();
   }
 
+  @Override
+  public void initialize(
+    InputSplit inputSplit,

Review Comment:
   nit: 4 space indent



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala:
##########
@@ -88,12 +88,18 @@ case class ParquetPartitionReaderFactory(
   private def getFooter(file: PartitionedFile): ParquetMetadata = {
     val conf = broadcastedConf.value.value
     val filePath = file.toPath
+    val split = new FileSplit(file.toPath, file.start, file.length, Array.empty[String])
 
-    if (aggregation.isEmpty) {
-      ParquetFooterReader.readFooter(conf, filePath, SKIP_ROW_GROUPS)
-    } else {
+    if (aggregation.isDefined) {
       // For aggregate push down, we will get max/min/count from footer statistics.
       ParquetFooterReader.readFooter(conf, filePath, NO_FILTER)
+    } else if (enableVectorizedReader) {
+      // This can avoid reading the footer twice(currently only optimize for vectorized read).
+      // Read all the row groups in advance and filter the row groups later if there are
+      // filters that need push down.
+      ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS)
+    } else {

Review Comment:
   it looks we can, since when aggregation push down is enabled, `ParquetScan.isSplitable` returns false, and we'll always read all the row groups in the file, so `NO_FILTER` is the same as `WITH_ROW_GROUPS`.



##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,17 +90,29 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+    InputSplit inputSplit,

Review Comment:
   nit: 4 space indent



##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,
+    FileSplit split,

Review Comment:
   instead of using `FileSplit`, can we just pass `PartitionedFile` here? `FileSplit` is from `org.apache.hadoop.mapred` which is really out-dated.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -205,11 +205,22 @@ class ParquetFileFormat
 
       val sharedConf = broadcastedHadoopConf.value.value
 
-      lazy val footerFileMetaData =
-        ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      val fileFooter = if (enableVectorizedReader) {
+        // This can avoid reading the footer twice(currently only optimize for vectorized read).

Review Comment:
   nit: space after "twice"



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1440499024

   @yabola yes, we'll need to use `RangeMetadataFilter` (i.e.: `HadoopReadOptions.builder().withRange()`) when we initially read the footer. This is possible since in places like `ParquetFileFormat` we already have a `PartitionedFile` which is just a segment in a Parquet file with a `start` and `length`.
   
   The only problem is we need new non-deprecated API from `parquet-mr` to support this use case. Personally I think we can just use the deprecated [API](https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java#L632) for now, and replace it after a new Parquet version is released.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1143531801


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala:
##########
@@ -88,12 +88,18 @@ case class ParquetPartitionReaderFactory(
   private def getFooter(file: PartitionedFile): ParquetMetadata = {
     val conf = broadcastedConf.value.value
     val filePath = file.toPath
+    val split = new FileSplit(file.toPath, file.start, file.length, Array.empty[String])
 
-    if (aggregation.isEmpty) {
-      ParquetFooterReader.readFooter(conf, filePath, SKIP_ROW_GROUPS)
-    } else {
+    if (aggregation.isDefined) {
       // For aggregate push down, we will get max/min/count from footer statistics.
       ParquetFooterReader.readFooter(conf, filePath, NO_FILTER)
+    } else if (enableVectorizedReader) {
+      // This can avoid reading the footer twice(currently only optimize for vectorized read).
+      // Read all the row groups in advance and filter the row groups later if there are
+      // filters that need push down.
+      ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS)
+    } else {

Review Comment:
   I think we can use 
   ```java
   if (aggregation.isDefined || enableVectorizedReader) {
       ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS)
   }
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1143531801


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala:
##########
@@ -88,12 +88,18 @@ case class ParquetPartitionReaderFactory(
   private def getFooter(file: PartitionedFile): ParquetMetadata = {
     val conf = broadcastedConf.value.value
     val filePath = file.toPath
+    val split = new FileSplit(file.toPath, file.start, file.length, Array.empty[String])
 
-    if (aggregation.isEmpty) {
-      ParquetFooterReader.readFooter(conf, filePath, SKIP_ROW_GROUPS)
-    } else {
+    if (aggregation.isDefined) {
       // For aggregate push down, we will get max/min/count from footer statistics.
       ParquetFooterReader.readFooter(conf, filePath, NO_FILTER)
+    } else if (enableVectorizedReader) {
+      // This can avoid reading the footer twice(currently only optimize for vectorized read).
+      // Read all the row groups in advance and filter the row groups later if there are
+      // filters that need push down.
+      ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS)
+    } else {

Review Comment:
   I think we can use 
   ```java
   if (aggregation.isDefined || enableVectorizedReader) {
   ParquetFooterReader.readFooter(conf, split, ParquetFooterReader.WITH_ROW_GROUPS)
   }
   ```



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153492439


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,17 +90,28 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+      InputSplit inputSplit,
+      TaskAttemptContext taskAttemptContext,
+      Option<ParquetMetadata> fileFooter) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     FileSplit split = (FileSplit) inputSplit;
     this.file = split.getPath();
-
-    ParquetReadOptions options = HadoopReadOptions
-      .builder(configuration, file)
-      .withRange(split.getStart(), split.getStart() + split.getLength())
-      .withCodecFactory(new ParquetCodecFactory(configuration, 0))
-      .build();
-    ParquetFileReader fileReader = new ParquetFileReader(
-        HadoopInputFile.fromPath(file, configuration), options);
+    ParquetFileReader fileReader;
+    if (fileFooter.isDefined()) {
+      fileReader = new ParquetFileReader(configuration, file, fileFooter.get());

Review Comment:
   when we pass footer here, then we can't pass `CodecFactory` any more. Parquet will create in default `HadoopCodecs.newFactory(conf, 0)`.
   
   This may need to be solved by upgrading parquet in the future, and I will also create a pr to remove @Deprecated tag.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153492439


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,17 +90,28 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+      InputSplit inputSplit,
+      TaskAttemptContext taskAttemptContext,
+      Option<ParquetMetadata> fileFooter) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     FileSplit split = (FileSplit) inputSplit;
     this.file = split.getPath();
-
-    ParquetReadOptions options = HadoopReadOptions
-      .builder(configuration, file)
-      .withRange(split.getStart(), split.getStart() + split.getLength())
-      .withCodecFactory(new ParquetCodecFactory(configuration, 0))
-      .build();
-    ParquetFileReader fileReader = new ParquetFileReader(
-        HadoopInputFile.fromPath(file, configuration), options);
+    ParquetFileReader fileReader;
+    if (fileFooter.isDefined()) {
+      fileReader = new ParquetFileReader(configuration, file, fileFooter.get());

Review Comment:
   when we pass footer here, then we can't pass `CodecFactory` any more. Parquet will create in default `HadoopCodecs.newFactory(conf, 0)`.
   
   This may need to be solved by upgrading parquet in the future, and I will also create a pr to remove `@Deprecated` in parquet tag.



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] cloud-fan commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1172104789


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,16 +90,27 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+      InputSplit inputSplit,
+      TaskAttemptContext taskAttemptContext,
+      Option<ParquetMetadata> fileFooter) throws IOException, InterruptedException {

Review Comment:
   do we need to use `Option`? The argument value is always `Some`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1520563772

   Yea @yabola is correct, if we have 100 row groups in a file and there are 100 tasks to read them, each task will only be assigned a range (e.g., a single row group) in the file to read, so it won't read metadata for all the row groups in the file.
   
   


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sadikovi commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "sadikovi (via GitHub)" <gi...@apache.org>.
sadikovi commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1519349584

   @yabola @sunchao  Could you share any benchmark numbers for the second optimisation of reading all row groups for each task? My concern is that it could be suboptimal in performance when you have, let's say, 100 row groups in a file, you create 100 tasks for each row group but then you read the full footer with all of the row groups for every task just to process one row group.


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1154849296


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java:
##########
@@ -89,17 +90,28 @@
   @Override
   public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
       throws IOException, InterruptedException {
+    initialize(inputSplit, taskAttemptContext, Option.empty());
+  }
+
+  public void initialize(
+      InputSplit inputSplit,
+      TaskAttemptContext taskAttemptContext,
+      Option<ParquetMetadata> fileFooter) throws IOException, InterruptedException {
     Configuration configuration = taskAttemptContext.getConfiguration();
     FileSplit split = (FileSplit) inputSplit;
     this.file = split.getPath();
-
-    ParquetReadOptions options = HadoopReadOptions
-      .builder(configuration, file)
-      .withRange(split.getStart(), split.getStart() + split.getLength())
-      .withCodecFactory(new ParquetCodecFactory(configuration, 0))
-      .build();
-    ParquetFileReader fileReader = new ParquetFileReader(
-        HadoopInputFile.fromPath(file, configuration), options);
+    ParquetFileReader fileReader;
+    if (fileFooter.isDefined()) {
+      fileReader = new ParquetFileReader(configuration, file, fileFooter.get());

Review Comment:
   yes.. this will be an issue unless Spark is upgraded to Parquet 1.12.4



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -205,11 +205,22 @@ class ParquetFileFormat
 
       val sharedConf = broadcastedHadoopConf.value.value
 
-      lazy val footerFileMetaData =
-        ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
+      val fileFooter = if (enableVectorizedReader) {
+        // This can avoid reading the footer twice(currently only optimize for vectorized read).

Review Comment:
   not addressed yet



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] sunchao commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice in vectorized reader

Posted by "sunchao (via GitHub)" <gi...@apache.org>.
sunchao commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1510042955

   Merged to master, thanks!


-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153376539


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer
+   *
+   * @param configuration hadoop configuration of file
+   * @param split file spit
+   * @param skipRowGroup If true, skip reading row groups;
+   *                     if false, read row groups according to the file range
+   */
+  public static ParquetMetadata readFooter(
+    Configuration configuration,

Review Comment:
   done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1153376111


##########
sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java:
##########
@@ -17,23 +17,53 @@
 
 package org.apache.spark.sql.execution.datasources.parquet;
 
+import java.io.IOException;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.mapred.FileSplit;
 import org.apache.parquet.HadoopReadOptions;
 import org.apache.parquet.ParquetReadOptions;
 import org.apache.parquet.format.converter.ParquetMetadataConverter;
 import org.apache.parquet.hadoop.ParquetFileReader;
 import org.apache.parquet.hadoop.metadata.ParquetMetadata;
 import org.apache.parquet.hadoop.util.HadoopInputFile;
 
-import java.io.IOException;
-
 /**
  * `ParquetFooterReader` is a util class which encapsulates the helper
  * methods of reading parquet file footer
  */
 public class ParquetFooterReader {
+
+  public static final boolean SKIP_ROW_GROUPS = true;
+  public static final boolean WITH_ROW_GROUPS = false;
+
+  /**
+   * method to read parquet file footer

Review Comment:
   done



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid unnecessary parquet footer reads when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1101155361


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -207,11 +207,11 @@ class ParquetFileFormat
 
       lazy val footerFileMetaData =
         ParquetFooterReader.readFooter(sharedConf, filePath, SKIP_ROW_GROUPS).getFileMetaData
-      val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
+      lazy val datetimeRebaseSpec = DataSourceUtils.datetimeRebaseSpec(
         footerFileMetaData.getKeyValueMetaData.get,

Review Comment:
   `footerFileMetaData` is lazy, but this method causes the footer to be read immediately. 
   Actually we can avoid this unnecessary footer reads and use footer metadata in `VectorizedParquetRecordReader`



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on a diff in pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on code in PR #39950:
URL: https://github.com/apache/spark/pull/39950#discussion_r1121979988


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala:
##########
@@ -296,41 +309,45 @@ class ParquetFileFormat
             throw e
         }
       } else {
-        logDebug(s"Falling back to parquet-mr")
-        // ParquetRecordReader returns InternalRow
-        val readSupport = new ParquetReadSupport(
-          convertTz,
-          enableVectorizedReader = false,
-          datetimeRebaseSpec,
-          int96RebaseSpec)
-        val reader = if (pushed.isDefined && enableRecordFilter) {
-          val parquetFilter = FilterCompat.get(pushed.get, null)
-          new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
-        } else {
-          new ParquetRecordReader[InternalRow](readSupport)
-        }
-        val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader,
-            requiredSchema)
-        val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes)
         try {
-          readerWithRowIndexes.initialize(split, hadoopAttemptContext)
-
-          val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
-          val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
-
-          if (partitionSchema.length == 0) {
-            // There is no partition columns
-            iter.map(unsafeProjection)
+          logDebug(s"Falling back to parquet-mr")
+          // ParquetRecordReader returns InternalRow
+          val readSupport = new ParquetReadSupport(
+            convertTz,
+            enableVectorizedReader = false,
+            datetimeRebaseSpec,
+            int96RebaseSpec)
+          val reader = if (pushed.isDefined && enableRecordFilter) {
+            val parquetFilter = FilterCompat.get(pushed.get, null)
+            new ParquetRecordReader[InternalRow](readSupport, parquetFilter)
           } else {
-            val joinedRow = new JoinedRow()
-            iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
+            new ParquetRecordReader[InternalRow](readSupport)
           }
-        } catch {
-          case e: Throwable =>
-            // SPARK-23457: In case there is an exception in initialization, close the iterator to
-            // avoid leaking resources.
-            iter.close()
-            throw e
+          val readerWithRowIndexes = ParquetRowIndexUtil.addRowIndexToRecordReaderIfNeeded(reader,
+            requiredSchema)
+          val iter = new RecordReaderIterator[InternalRow](readerWithRowIndexes)
+          try {
+            readerWithRowIndexes.initialize(split, hadoopAttemptContext)
+
+            val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes
+            val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema)
+
+            if (partitionSchema.length == 0) {
+              // There is no partition columns
+              iter.map(unsafeProjection)
+            } else {
+              val joinedRow = new JoinedRow()
+              iter.map(d => unsafeProjection(joinedRow(d, file.partitionValues)))
+            }
+          } catch {
+            case e: Throwable =>
+              // SPARK-23457: In case there is an exception in initialization, close the iterator to
+              // avoid leaking resources.
+              iter.close()
+              throw e
+          }
+        } finally {
+          parquetReader.close()

Review Comment:
   I haven't change the code logic, just add to close parquetReader.
   Before, if the non-vectorization code path will close the reader after reading `footerFileMetaData` , it is consistent here. Please see
   [the original implementation code](https://github.com/apache/spark/blob/master/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/ParquetFooterReader.java#L53)
   



-- 
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-unsubscribe@spark.apache.org

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


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


[GitHub] [spark] yabola commented on pull request #39950: [SPARK-42388][SQL] Avoid parquet footer reads twice when no filters in vectorized reader

Posted by "yabola (via GitHub)" <gi...@apache.org>.
yabola commented on PR #39950:
URL: https://github.com/apache/spark/pull/39950#issuecomment-1451913404

   @sunchao Sorry, it might be a mistake. We should read schema in footer meta first to get which filters need to be pushed down. After that we set pushdown info [codes](https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L261)  and read filtered RowGroups.
   
   So I think it might have to read footer twice if having filters. But we can read footer once when no filter is needed pushdown.
   
   I think It will be useful when scanning joined tables (filter condition is only on the other side of the join table and filter is none in the joined table).  It can avoid reading much footer when there are many  joined tables.


-- 
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-unsubscribe@spark.apache.org

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


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