You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2016/11/09 07:01:13 UTC
spark git commit: [SPARK-18333][SQL] Revert hacks in parquet and orc
reader to support case insensitive resolution
Repository: spark
Updated Branches:
refs/heads/master 55964c15a -> 4afa39e22
[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 <ek...@databricks.com>
Closes #15799 from ericl/sc-4929.
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4afa39e2
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4afa39e2
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4afa39e2
Branch: refs/heads/master
Commit: 4afa39e223c70e91b6ee19e9ea76fa9115203d74
Parents: 55964c1
Author: Eric Liang <ek...@databricks.com>
Authored: Wed Nov 9 15:00:46 2016 +0800
Committer: Wenchen Fan <we...@databricks.com>
Committed: Wed Nov 9 15:00:46 2016 +0800
----------------------------------------------------------------------
.../parquet/ParquetReadSupport.scala | 6 +----
.../parquet/ParquetSchemaSuite.scala | 28 --------------------
.../spark/sql/hive/orc/OrcFileFormat.scala | 12 +--------
3 files changed, 2 insertions(+), 44 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/4afa39e2/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala
----------------------------------------------------------------------
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 4dea8cf..f1a35dd 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))
}
http://git-wip-us.apache.org/repos/asf/spark/blob/4afa39e2/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
----------------------------------------------------------------------
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 c3d202c..8a980a7 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
@@ -1081,34 +1081,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",
parquetSchema =
http://git-wip-us.apache.org/repos/asf/spark/blob/4afa39e2/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala
----------------------------------------------------------------------
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 7c519a0..42c92ed 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)
}
---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org