Skip to content

Commit 0dc14f1

Browse files
ericlcloud-fan
authored andcommitted
[SPARK-18333][SQL] Revert hacks in parquet and orc reader to support case insensitive resolution
## What changes were proposed in this pull request? These are no longer needed after https://issues.apache.org/jira/browse/SPARK-17183 cc cloud-fan ## How was this patch tested? Existing parquet and orc tests. Author: Eric Liang <[email protected]> Closes #15799 from ericl/sc-4929. (cherry picked from commit 4afa39e) Signed-off-by: Wenchen Fan <[email protected]>
1 parent 98dd7ac commit 0dc14f1

File tree

3 files changed

+2
-44
lines changed

3 files changed

+2
-44
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala

Lines changed: 1 addition & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -269,15 +269,11 @@ private[parquet] object ParquetReadSupport {
269269
*/
270270
private def clipParquetGroupFields(
271271
parquetRecord: GroupType, structType: StructType): Seq[Type] = {
272-
val parquetFieldMap = parquetRecord.getFields.asScala
273-
.map(f => f.getName -> f).toMap
274-
val caseInsensitiveParquetFieldMap = parquetRecord.getFields.asScala
275-
.map(f => f.getName.toLowerCase -> f).toMap
272+
val parquetFieldMap = parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap
276273
val toParquet = new ParquetSchemaConverter(writeLegacyParquetFormat = false)
277274
structType.map { f =>
278275
parquetFieldMap
279276
.get(f.name)
280-
.orElse(caseInsensitiveParquetFieldMap.get(f.name.toLowerCase))
281277
.map(clipParquetType(_, f.dataType))
282278
.getOrElse(toParquet.convertField(f))
283279
}

sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala

Lines changed: 0 additions & 28 deletions
Original file line numberDiff line numberDiff line change
@@ -1080,34 +1080,6 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
10801080
}
10811081
}
10821082

1083-
testSchemaClipping(
1084-
"falls back to case insensitive resolution",
1085-
1086-
parquetSchema =
1087-
"""message root {
1088-
| required group A {
1089-
| optional int32 B;
1090-
| }
1091-
| optional int32 c;
1092-
|}
1093-
""".stripMargin,
1094-
1095-
catalystSchema = {
1096-
val nestedType = new StructType().add("b", IntegerType, nullable = true)
1097-
new StructType()
1098-
.add("a", nestedType, nullable = true)
1099-
.add("c", IntegerType, nullable = true)
1100-
},
1101-
1102-
expectedSchema =
1103-
"""message root {
1104-
| required group A {
1105-
| optional int32 B;
1106-
| }
1107-
| optional int32 c;
1108-
|}
1109-
""".stripMargin)
1110-
11111083
testSchemaClipping(
11121084
"simple nested struct",
11131085

sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala

Lines changed: 1 addition & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -305,17 +305,7 @@ private[orc] object OrcRelation extends HiveInspectors {
305305

306306
def setRequiredColumns(
307307
conf: Configuration, physicalSchema: StructType, requestedSchema: StructType): Unit = {
308-
val caseInsensitiveFieldMap: Map[String, Int] = physicalSchema.fieldNames
309-
.zipWithIndex
310-
.map(f => (f._1.toLowerCase, f._2))
311-
.toMap
312-
val ids = requestedSchema.map { a =>
313-
val exactMatch: Option[Int] = physicalSchema.getFieldIndex(a.name)
314-
val res = exactMatch.getOrElse(
315-
caseInsensitiveFieldMap.getOrElse(a.name,
316-
throw new IllegalArgumentException(s"""Field "$a.name" does not exist.""")))
317-
res: Integer
318-
}
308+
val ids = requestedSchema.map(a => physicalSchema.fieldIndex(a.name): Integer)
319309
val (sortedIDs, sortedNames) = ids.zip(requestedSchema.fieldNames).sorted.unzip
320310
HiveShim.appendReadColumns(conf, sortedIDs, sortedNames)
321311
}

0 commit comments

Comments
 (0)