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

Disable write/read Parquet when Parquet field IDs are used #4882

Merged
merged 5 commits into from
Mar 7, 2022
Merged
Show file tree
Hide file tree
Changes from 2 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
Original file line number Diff line number Diff line change
Expand Up @@ -16,13 +16,24 @@

package com.nvidia.spark.rapids.shims.v2

import com.nvidia.spark.rapids.RapidsMeta
import org.apache.hadoop.conf.Configuration

import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType

object ParquetFieldIdShims {
/** Updates the Hadoop configuration with the Parquet field ID write setting from SQLConf */
def setupParquetFieldIdWriteConfig(conf: Configuration, sqlConf: SQLConf): Unit = {
// Parquet field ID support configs are not supported until Spark 3.3
}

def tagGpuSupportWriteForFieldId(meta: RapidsMeta[_, _, _], schema: StructType): Unit = {
// Parquet field ID support configs are not supported until Spark 3.3
}

def tagGpuSupportReadForFieldId(meta: RapidsMeta[_, _, _], conf: RuntimeConfig): Unit = {
// Parquet field ID support configs are not supported until Spark 3.3
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -16,9 +16,13 @@

package com.nvidia.spark.rapids.shims.v2

import com.nvidia.spark.rapids.RapidsMeta
import org.apache.hadoop.conf.Configuration

import org.apache.spark.sql.RuntimeConfig
import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType

object ParquetFieldIdShims {
/** Updates the Hadoop configuration with the Parquet field ID write setting from SQLConf */
Expand All @@ -27,4 +31,18 @@ object ParquetFieldIdShims {
SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED.key,
sqlConf.parquetFieldIdWriteEnabled.toString)
}

def tagGpuSupportWriteForFieldId(meta: RapidsMeta[_, _, _], schema: StructType): Unit = {
if (ParquetUtils.hasFieldIds(schema)) {
jlowe marked this conversation as resolved.
Show resolved Hide resolved
meta.willNotWorkOnGpu(
"Currently not support 'parquet.field.id' in parquet writer, schema is " + schema.json)
jlowe marked this conversation as resolved.
Show resolved Hide resolved
}
}

def tagGpuSupportReadForFieldId(meta: RapidsMeta[_, _, _], conf: RuntimeConfig): Unit = {
if(conf.get(SQLConf.PARQUET_FIELD_ID_READ_ENABLED.key, "false").toBoolean) {
meta.willNotWorkOnGpu("Currently not support reading field ids, " +
"please set spark.sql.parquet.fieldId.read.enabled as false")
jlowe marked this conversation as resolved.
Show resolved Hide resolved
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,8 @@ object GpuParquetFileFormat {
options: Map[String, String],
schema: StructType): Option[GpuParquetFileFormat] = {

ParquetFieldIdShims.tagGpuSupportWriteForFieldId(meta, schema)

val sqlConf = spark.sessionState.conf
val parquetOptions = new ParquetOptions(options, sqlConf)

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import com.nvidia.spark.RebaseHelper
import com.nvidia.spark.rapids.GpuMetric._
import com.nvidia.spark.rapids.ParquetPartitionReader.CopyRange
import com.nvidia.spark.rapids.RapidsPluginImplicits._
import com.nvidia.spark.rapids.shims.v2.ParquetFieldIdShims
import org.apache.commons.io.output.{CountingOutputStream, NullOutputStream}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FSDataInputStream, Path}
Expand Down Expand Up @@ -143,6 +144,8 @@ object GpuParquetScanBase {
meta: RapidsMeta[_, _, _]): Unit = {
val sqlConf = sparkSession.conf

ParquetFieldIdShims.tagGpuSupportReadForFieldId(meta, sqlConf)

if (!meta.conf.isParquetEnabled) {
meta.willNotWorkOnGpu("Parquet input and output has been disabled. To enable set" +
s"${RapidsConf.ENABLE_PARQUET} to true")
Expand Down
30 changes: 30 additions & 0 deletions tests/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -238,6 +238,36 @@
</dependency>
</dependencies>
</profile>
<profile>
<id>release330</id>
<activation>
<property>
<name>buildver</name>
<value>330</value>
</property>
</activation>

<build>
<plugins>
<plugin>
<groupId>org.codehaus.mojo</groupId>
<artifactId>build-helper-maven-plugin</artifactId>
<executions>
<execution>
<id>add-profile-test-src</id>
<goals><goal>add-test-source</goal></goals>
<configuration>
<sources>
<!-- some test cases that can't be compiled before Spark 330 -->
<source>${project.basedir}/src/test/330+/scala</source>
</sources>
</configuration>
</execution>
</executions>
</plugin>
</plugins>
</build>
</profile>
</profiles>

<build>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* Copyright (c) 2022, NVIDIA CORPORATION.
*
* Licensed under the Apache License, Version 2.0 (the "License");
* you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package com.nvidia.spark.rapids

import java.io.File

import org.apache.spark.SparkConf
import org.apache.spark.sql.Row
import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils
import org.apache.spark.sql.types.{IntegerType, MetadataBuilder, StructType}

/**
* TODO should update after cuDF supports field id
* See https://github.com/NVIDIA/spark-rapids/issues/4846
*/
class ParquetFieldIdSuite extends SparkQueryCompareTestSuite {

// this should failed
test("try to write field id") {
val tmpFile = File.createTempFile("field-id", ".parquet")
try {
def withId(id: Int) =
new MetadataBuilder().putLong(ParquetUtils.FIELD_ID_METADATA_KEY, id).build()
// not support writing field id
val schema = new StructType().add("c1", IntegerType, nullable = true, withId(1))
val data = (1 to 4).map(i => Row(i))

assertThrows[IllegalArgumentException] {
jlowe marked this conversation as resolved.
Show resolved Hide resolved
withGpuSparkSession(
spark => spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
.write.mode("overwrite").parquet(tmpFile.getAbsolutePath)
)
}
} finally {
tmpFile.delete()
}
}

// this should failed
test("try to read field id") {
val tmpFile = File.createTempFile("field-id", ".parquet")
try {
def withId(id: Int) =
new MetadataBuilder().putLong(ParquetUtils.FIELD_ID_METADATA_KEY, id).build()
val schema = new StructType().add("c1", IntegerType, nullable = true, withId(1))
val data = (1 to 4).map(i => Row(i))

withCpuSparkSession(
spark => spark.createDataFrame(spark.sparkContext.parallelize(data), schema)
.write.mode("overwrite").parquet(tmpFile.getAbsolutePath)
)

assertThrows[IllegalArgumentException] {
withGpuSparkSession(
spark => spark.read.parquet(tmpFile.getAbsolutePath).collect(),
// not support read field id
new SparkConf().set("spark.sql.parquet.fieldId.read.enabled", "true")
)
}
} finally {
tmpFile.delete()
}
}
}