Skip to content
Closed
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
Expand Up @@ -163,8 +163,7 @@ case class DataSource(
// maintain old behavior before SPARK-18510. If userSpecifiedSchema is empty used inferred
// partitioning
if (userSpecifiedSchema.isEmpty) {
val inferredPartitions = tempFileIndex.partitionSchema
inferredPartitions
tempFileIndex.partitionSchema
} else {
val partitionFields = partitionColumns.map { partitionColumn =>
userSpecifiedSchema.flatMap(_.find(c => equality(c.name, partitionColumn))).orElse {
Expand Down Expand Up @@ -357,7 +356,11 @@ case class DataSource(
} else {
tempFileCatalog
}
val dataSchema = userSpecifiedSchema.orElse {

val partitionSchema = fileCatalog.partitionSchema
val dataSchema = userSpecifiedSchema.map { schema =>
StructType(schema.filterNot(f => partitionSchema.exists(p => equality(p.name, f.name))))
}.orElse {
format.inferSchema(
sparkSession,
caseInsensitiveOptions,
Expand All @@ -370,7 +373,7 @@ case class DataSource(

HadoopFsRelation(
fileCatalog,
partitionSchema = fileCatalog.partitionSchema,
partitionSchema = partitionSchema,
dataSchema = dataSchema,
bucketSpec = None,
format,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,8 +59,8 @@ case class HadoopFsRelation(
overlappedPartCols += getColName(partitionField) -> partitionField
}
}
StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)) ++
partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f))))
StructType(dataSchema.filterNot(f => overlappedPartCols.contains(getColName(f))) ++
partitionSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)))
}

def partitionSchemaOption: Option[StructType] =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -519,7 +519,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha
i <- 1 to 10
pi <- Seq(1, 2)
ps <- Seq("foo", "bar")
} yield Row(i, pi, i.toString, ps))
} yield Row(i, i.toString, pi, ps))

checkAnswer(
sql("SELECT intField, pi FROM t"),
Expand All @@ -534,14 +534,14 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha
for {
i <- 1 to 10
ps <- Seq("foo", "bar")
} yield Row(i, 1, i.toString, ps))
} yield Row(i, i.toString, 1, ps))

checkAnswer(
sql("SELECT * FROM t WHERE ps = 'foo'"),
for {
i <- 1 to 10
pi <- Seq(1, 2)
} yield Row(i, pi, i.toString, "foo"))
} yield Row(i, i.toString, pi, "foo"))
}
}
}
Expand Down Expand Up @@ -608,14 +608,14 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha
i <- 1 to 10
pi <- Seq(1, 2)
ps <- Seq("foo", null.asInstanceOf[String])
} yield Row(i, pi, i.toString, ps))
} yield Row(i, i.toString, pi, ps))

checkAnswer(
sql("SELECT * FROM t WHERE ps IS NULL"),
for {
i <- 1 to 10
pi <- Seq(1, 2)
} yield Row(i, pi, i.toString, null))
} yield Row(i, i.toString, pi, null))
}
}
}
Expand Down Expand Up @@ -1019,7 +1019,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha
val path = dir.getCanonicalPath
val df = Seq((1L, 2.0)).toDF("a", "b")
df.write.parquet(s"$path/a=1")
checkAnswer(spark.read.parquet(s"$path"), Seq(Row(1, 2.0)))
// partition columns are always at the end of the schema.
assert(spark.read.parquet(s"$path").columns === Array("b", "a"))
checkAnswer(spark.read.parquet(s"$path"), Seq(Row(2.0, 1)))
}
}
}
Expand Down Expand Up @@ -1048,7 +1050,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha

checkAnswer(
readBack,
Seq(Row("2017-01-01-00", 1), Row("2017-01-01-01", 2))
Seq(Row(1, "2017-01-01-00"), Row(2, "2017-01-01-01"))
)
} finally {
spark.streams.active.foreach(_.stop())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,13 +164,12 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
}
}

val (dataSchema, updatedTable) =
inferIfNeeded(relation, options, fileFormat, Option(fileIndex))
val updatedTable = inferIfNeeded(relation, options, fileFormat, Option(fileIndex))

val fsRelation = HadoopFsRelation(
location = fileIndex,
partitionSchema = partitionSchema,
dataSchema = dataSchema,
dataSchema = updatedTable.dataSchema,
bucketSpec = None,
fileFormat = fileFormat,
options = options)(sparkSession = sparkSession)
Expand All @@ -191,13 +190,13 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
fileFormatClass,
None)
val logicalRelation = cached.getOrElse {
val (dataSchema, updatedTable) = inferIfNeeded(relation, options, fileFormat)
val updatedTable = inferIfNeeded(relation, options, fileFormat)
val created =
LogicalRelation(
DataSource(
sparkSession = sparkSession,
paths = rootPath.toString :: Nil,
userSpecifiedSchema = Option(dataSchema),
userSpecifiedSchema = Option(updatedTable.schema),
bucketSpec = None,
options = options,
className = fileType).resolveRelation(),
Expand All @@ -220,11 +219,15 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
result.copy(output = newOutput)
}

/**
* Infer the data schema from files if needed, and return a `CatalogTable` with the corrected
* table schema.
*/
private def inferIfNeeded(
relation: HiveTableRelation,
options: Map[String, String],
fileFormat: FileFormat,
fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
fileIndexOpt: Option[FileIndex] = None): CatalogTable = {
val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
val tableName = relation.tableMeta.identifier.unquotedString
Expand All @@ -241,21 +244,22 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
sparkSession,
options,
fileIndex.listFiles(Nil, Nil).flatMap(_.files))
.map(mergeWithMetastoreSchema(relation.tableMeta.schema, _))
.map(mergeWithMetastoreSchema(relation.tableMeta.dataSchema, _))

inferredSchema match {
case Some(schema) =>
case Some(dataSchema) =>
val actualSchema = StructType(dataSchema ++ relation.tableMeta.partitionSchema)
if (inferenceMode == INFER_AND_SAVE) {
updateCatalogSchema(relation.tableMeta.identifier, schema)
updateCatalogSchema(relation.tableMeta.identifier, actualSchema)
}
(schema, relation.tableMeta.copy(schema = schema))
relation.tableMeta.copy(schema = actualSchema)
case None =>
logWarning(s"Unable to infer schema for table $tableName from file format " +
s"$fileFormat (inference mode: $inferenceMode). Using metastore schema.")
(relation.tableMeta.schema, relation.tableMeta)
relation.tableMeta
}
} else {
(relation.tableMeta.schema, relation.tableMeta)
relation.tableMeta
}
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,6 @@ class HiveMetadataCacheSuite extends QueryTest with SQLTestUtils with TestHiveSi
|location "${dir.toURI}"""".stripMargin)
spark.sql("msck repair table test")

val df = spark.sql("select * from test")
assert(sql("select * from test").count() == 5)

def deleteRandomFile(): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B
i <- 1 to 10
pi <- Seq(1, 2)
ps <- Seq("foo", "bar")
} yield Row(i, pi, i.toString, ps))
} yield Row(i, i.toString, pi, ps))

checkAnswer(
sql("SELECT intField, pi FROM t"),
Expand All @@ -161,14 +161,14 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B
for {
i <- 1 to 10
ps <- Seq("foo", "bar")
} yield Row(i, 1, i.toString, ps))
} yield Row(i, i.toString, 1, ps))

checkAnswer(
sql("SELECT * FROM t WHERE ps = 'foo'"),
for {
i <- 1 to 10
pi <- Seq(1, 2)
} yield Row(i, pi, i.toString, "foo"))
} yield Row(i, i.toString, pi, "foo"))
}
}
}
Expand Down Expand Up @@ -240,14 +240,14 @@ class OrcPartitionDiscoverySuite extends QueryTest with TestHiveSingleton with B
i <- 1 to 10
pi <- Seq(1, 2)
ps <- Seq("foo", null.asInstanceOf[String])
} yield Row(i, pi, i.toString, ps))
} yield Row(i, i.toString, pi, ps))

checkAnswer(
sql("SELECT * FROM t WHERE ps IS NULL"),
for {
i <- 1 to 10
pi <- Seq(1, 2)
} yield Row(i, pi, i.toString, null))
} yield Row(i, i.toString, pi, null))
}
}
}
Expand Down