diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
index 4dea8cf29ec58b407d1768d571b33f54247e46e8..f1a35dd8a6200fb4c14069dd75e629d3707d08a9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
@@ -269,15 +269,11 @@ private[parquet] object ParquetReadSupport {
    */
   private def clipParquetGroupFields(
       parquetRecord: GroupType, structType: StructType): Seq[Type] = {
-    val parquetFieldMap = parquetRecord.getFields.asScala
-      .map(f => f.getName -> f).toMap
-    val caseInsensitiveParquetFieldMap = parquetRecord.getFields.asScala
-      .map(f => f.getName.toLowerCase -> f).toMap
+    val parquetFieldMap = parquetRecord.getFields.asScala.map(f => f.getName -> f).toMap
     val toParquet = new ParquetSchemaConverter(writeLegacyParquetFormat = false)
     structType.map { f =>
       parquetFieldMap
         .get(f.name)
-        .orElse(caseInsensitiveParquetFieldMap.get(f.name.toLowerCase))
         .map(clipParquetType(_, f.dataType))
         .getOrElse(toParquet.convertField(f))
     }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
index c3d202ced24c802ed51a48e00b8a767e5730b196..8a980a7eb538f5dcda85a3861104f336966924b3 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
@@ -1080,34 +1080,6 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
     }
   }
 
-  testSchemaClipping(
-    "falls back to case insensitive resolution",
-
-    parquetSchema =
-      """message root {
-        |  required group A {
-        |    optional int32 B;
-        |  }
-        |  optional int32 c;
-        |}
-      """.stripMargin,
-
-    catalystSchema = {
-      val nestedType = new StructType().add("b", IntegerType, nullable = true)
-      new StructType()
-        .add("a", nestedType, nullable = true)
-        .add("c", IntegerType, nullable = true)
-    },
-
-    expectedSchema =
-      """message root {
-        |  required group A {
-        |    optional int32 B;
-        |  }
-        |  optional int32 c;
-        |}
-      """.stripMargin)
-
   testSchemaClipping(
     "simple nested struct",
 
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
index 7c519a074317ae07d705e5e1eafc9de1e66f9b94..42c92ed5cae267d167c7d55ce23f385d8755067f 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
@@ -305,17 +305,7 @@ private[orc] object OrcRelation extends HiveInspectors {
 
   def setRequiredColumns(
       conf: Configuration, physicalSchema: StructType, requestedSchema: StructType): Unit = {
-    val caseInsensitiveFieldMap: Map[String, Int] = physicalSchema.fieldNames
-      .zipWithIndex
-      .map(f => (f._1.toLowerCase, f._2))
-      .toMap
-    val ids = requestedSchema.map { a =>
-      val exactMatch: Option[Int] = physicalSchema.getFieldIndex(a.name)
-      val res = exactMatch.getOrElse(
-        caseInsensitiveFieldMap.getOrElse(a.name,
-          throw new IllegalArgumentException(s"""Field "$a.name" does not exist.""")))
-      res: Integer
-    }
+    val ids = requestedSchema.map(a => physicalSchema.fieldIndex(a.name): Integer)
     val (sortedIDs, sortedNames) = ids.zip(requestedSchema.fieldNames).sorted.unzip
     HiveShim.appendReadColumns(conf, sortedIDs, sortedNames)
   }