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 of failing to read parquet files after writing the hidden file metadata in #4954

Merged
merged 1 commit into from
Mar 16, 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
Original file line number Diff line number Diff line change
@@ -0,0 +1,24 @@
/*
* 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.shims

import org.apache.spark.sql.catalyst.expressions.Attribute

object RapidsFileSourceMetaUtils {
/** Do nothing before Spark 3.3.0 */
def cleanupFileSourceMetadataInformation(attr: Attribute): Attribute = attr
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,29 @@
/*
* 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.shims

import org.apache.spark.sql.catalyst.expressions.{Attribute, FileSourceMetadataAttribute}

object RapidsFileSourceMetaUtils {
/**
* Cleanup the internal metadata information of an attribute if it is
* a 'FileSourceMetadataAttribute', it will remove both 'METADATA_COL_ATTR_KEY' and
* 'FILE_SOURCE_METADATA_COL_ATTR_KEY' from the attribute 'Metadata'
*/
def cleanupFileSourceMetadataInformation(attr: Attribute): Attribute =
FileSourceMetadataAttribute.cleanupFileSourceMetadataInformation(attr)
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,7 @@ import java.util.{Date, UUID}
import ai.rapids.cudf.ColumnVector
import com.nvidia.spark.TimingUtils
import com.nvidia.spark.rapids._
import com.nvidia.spark.rapids.shims.SparkShimImpl
import com.nvidia.spark.rapids.shims.{RapidsFileSourceMetaUtils, SparkShimImpl}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce._
Expand Down Expand Up @@ -117,7 +117,11 @@ object GpuFileFormatWriter extends Logging {
FileOutputFormat.setOutputPath(job, new Path(outputSpec.outputPath))

val partitionSet = AttributeSet(partitionColumns)
val dataColumns = outputSpec.outputColumns.filterNot(partitionSet.contains)
// cleanup the internal metadata information of
// the file source metadata attribute if any before write out when needed.
val finalOutputSpec = outputSpec.copy(outputColumns = outputSpec.outputColumns
.map(RapidsFileSourceMetaUtils.cleanupFileSourceMetadataInformation))
val dataColumns = finalOutputSpec.outputColumns.filterNot(partitionSet.contains)

var needConvert = false
val projectList: List[NamedExpression] = plan.output.map {
Expand Down Expand Up @@ -153,12 +157,12 @@ object GpuFileFormatWriter extends Logging {
uuid = UUID.randomUUID.toString,
serializableHadoopConf = new SerializableConfiguration(job.getConfiguration),
outputWriterFactory = outputWriterFactory,
allColumns = outputSpec.outputColumns,
allColumns = finalOutputSpec.outputColumns,
dataColumns = dataColumns,
partitionColumns = partitionColumns,
bucketSpec = writerBucketSpec,
path = outputSpec.outputPath,
customPartitionLocations = outputSpec.customPartitionLocations,
path = finalOutputSpec.outputPath,
customPartitionLocations = finalOutputSpec.customPartitionLocations,
maxRecordsPerFile = caseInsensitiveOptions.get("maxRecordsPerFile").map(_.toLong)
.getOrElse(sparkSession.sessionState.conf.maxRecordsPerFile),
timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION)
Expand Down Expand Up @@ -199,7 +203,7 @@ object GpuFileFormatWriter extends Logging {
// aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch.
val orderingExpr = GpuBindReferences.bindReferences(
requiredOrdering
.map(attr => SparkShimImpl.sortOrder(attr, Ascending)), outputSpec.outputColumns)
.map(attr => SparkShimImpl.sortOrder(attr, Ascending)), finalOutputSpec.outputColumns)
val sortType = if (RapidsConf.STABLE_SORT.get(plan.conf)) {
FullSortSingleBatch
} else {
Expand Down