Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Fix merge of branch-22.04 to branch-22.06 #5027

Merged
merged 5 commits into from
Mar 23, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion docs/additional-functionality/rapids-shuffle.md
Original file line number Diff line number Diff line change
Expand Up @@ -60,7 +60,7 @@ The minimum UCX requirement for the RAPIDS Shuffle Manager is
Other considerations:

- Please refer to [Mellanox documentation](
https://community.mellanox.com/s/article/recommended-network-configuration-examples-for-roce-deployment)
https://support.mellanox.com/s/article/recommended-network-configuration-examples-for-roce-deployment)
on how to configure RoCE networks (lossless/lossy, QoS, and more)

- We recommend that the `--without-ucx` option is passed when installing MLNX_OFED
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@ import org.apache.parquet.filter2.predicate.FilterApi
import org.apache.parquet.format.converter.ParquetMetadataConverter
import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat}
import org.apache.parquet.hadoop.metadata._
import org.apache.parquet.hadoop.util.HadoopInputFile
import org.apache.parquet.schema.{GroupType, MessageType, OriginalType, PrimitiveType, Type, Types}
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName

Expand Down Expand Up @@ -369,8 +370,8 @@ private case class GpuParquetFileFilterHandler(@transient sqlConf: SQLConf) exte

val filePath = new Path(new URI(file.filePath))
//noinspection ScalaDeprecation
val footer = ParquetFileReader.readFooter(conf, filePath,
ParquetMetadataConverter.range(file.start, file.start + file.length))
val inputFile = HadoopInputFile.fromPath(filePath, conf)
val footer = withResource(ParquetFileReader.open(inputFile))(_.getFooter)
val fileSchema = footer.getFileMetaData.getSchema
val pushedFilters = if (enableParquetFilterPushDown) {
val parquetFilters = SparkShimImpl.getParquetFilters(fileSchema, pushDownDate,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ package com.nvidia.spark.rapids

import scala.annotation.tailrec

import com.nvidia.spark.rapids.shims.{GpuOrcScan, GpuParquetScan, SparkShimImpl}
import com.nvidia.spark.rapids.shims.SparkShimImpl

import org.apache.spark.sql.catalyst.expressions.{Ascending, Attribute, AttributeReference, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, SortOrder}
import org.apache.spark.sql.catalyst.rules.Rule
Expand Down
6 changes: 6 additions & 0 deletions tests/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -224,6 +224,12 @@
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-common</artifactId>
<version>${spark.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.parquet</groupId>
<artifactId>parquet-column</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,15 @@

package com.nvidia.spark.rapids

import java.io.File
import java.io.{File, FilenameFilter}
import java.nio.charset.StandardCharsets

import com.nvidia.spark.rapids.shims.SparkShimImpl
import org.apache.commons.io.filefilter.WildcardFileFilter
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext}
import org.apache.parquet.hadoop.ParquetFileReader
import org.apache.parquet.hadoop.util.HadoopInputFile

import org.apache.spark.{SparkConf, SparkException}
import org.apache.spark.internal.io.FileCommitProtocol
Expand All @@ -32,22 +34,20 @@ import org.apache.spark.sql.rapids.BasicColumnarWriteJobStatsTracker
/**
* Tests for writing Parquet files with the GPU.
*/
@scala.annotation.nowarn(
"msg=method readFooters in class ParquetFileReader is deprecated"
)
class ParquetWriterSuite extends SparkQueryCompareTestSuite {
test("file metadata") {
val tempFile = File.createTempFile("stats", ".parquet")
try {
withGpuSparkSession(spark => {
val df = mixedDfWithNulls(spark)
df.write.mode("overwrite").parquet(tempFile.getAbsolutePath)
val filter: FilenameFilter = new WildcardFileFilter("*.parquet")
val inputFile = HadoopInputFile.fromPath(
new Path(tempFile.listFiles(filter)(0).getAbsolutePath),
spark.sparkContext.hadoopConfiguration)
val parquetMeta = withResource(ParquetFileReader.open(inputFile))(_.getFooter)

val footer = ParquetFileReader.readFooters(spark.sparkContext.hadoopConfiguration,
new Path(tempFile.getAbsolutePath)).get(0)

val parquetMeta = footer.getParquetMetadata
val fileMeta = footer.getParquetMetadata.getFileMetaData
val fileMeta = parquetMeta.getFileMetaData
val extra = fileMeta.getKeyValueMetaData
assert(extra.containsKey("org.apache.spark.version"))
assert(extra.containsKey("org.apache.spark.sql.parquet.row.metadata"))
Expand Down