forked from delta-io/delta
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
Add a column name mapping mode in Delta, which allows Delta to use different names in the table schema and in the underlying Parquet files. This is a building block for issues delta-io#957 and delta-io#958. New unit tests. Closes delta-io#962 GitOrigin-RevId: 7a64a33fd60781c17236bff6168044e702c8413a
- Loading branch information
1 parent
85b35c2
commit 5fa760f
Showing
8 changed files
with
1,322 additions
and
18 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
73 changes: 73 additions & 0 deletions
73
core/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,73 @@ | ||
/* | ||
* Copyright (2021) The Delta Lake Project Authors. | ||
* | ||
* 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 org.apache.spark.sql.delta | ||
|
||
import org.apache.hadoop.conf.Configuration | ||
|
||
import org.apache.spark.sql.SparkSession | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.execution.datasources.PartitionedFile | ||
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat | ||
import org.apache.spark.sql.sources.Filter | ||
import org.apache.spark.sql.types.StructType | ||
|
||
/** A thin wrapper over the Parquet file format to support columns names without restrictions. */ | ||
class DeltaParquetFileFormat( | ||
val columnMappingMode: DeltaColumnMappingMode, | ||
val referenceSchema: StructType) | ||
extends ParquetFileFormat { | ||
|
||
private def prepareSchema(inputSchema: StructType): StructType = { | ||
DeltaColumnMapping.createPhysicalSchema(inputSchema, referenceSchema, columnMappingMode) | ||
} | ||
|
||
/** | ||
* We sometimes need to replace FileFormat within LogicalPlans, so we have to override | ||
* `equals` to ensure file format changes are captured | ||
*/ | ||
override def equals(other: Any): Boolean = { | ||
other match { | ||
case ff: DeltaParquetFileFormat | ||
=> ff.columnMappingMode == columnMappingMode && ff.referenceSchema == referenceSchema | ||
case _ => false | ||
} | ||
} | ||
|
||
override def hashCode(): Int = getClass.hashCode() | ||
|
||
override def buildReaderWithPartitionValues( | ||
sparkSession: SparkSession, | ||
dataSchema: StructType, | ||
partitionSchema: StructType, | ||
requiredSchema: StructType, | ||
filters: Seq[Filter], | ||
options: Map[String, String], | ||
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { | ||
super.buildReaderWithPartitionValues( | ||
sparkSession, | ||
prepareSchema(dataSchema), | ||
prepareSchema(partitionSchema), | ||
prepareSchema(requiredSchema), | ||
filters, | ||
options, | ||
hadoopConf) | ||
} | ||
|
||
override def supportFieldName(name: String): Boolean = { | ||
if (columnMappingMode != NoMapping) true else super.supportFieldName(name) | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.