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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.datasources.BasicWriteJobStatsTracker
import org.apache.spark.sql.execution.datasources.FileFormatWriter
import org.apache.spark.sql.execution.metric.SQLMetric
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.SerializableConfiguration

/**
Expand Down Expand Up @@ -75,18 +74,4 @@ object DataWritingCommand {
attr.withName(outputName)
}
}

/**
* Returns schema of logical plan with provided names.
* The length of provided names should be the same of the length of [[LogicalPlan.schema]].
*/
def logicalPlanSchemaWithNames(
query: LogicalPlan,
names: Seq[String]): StructType = {
assert(query.schema.length == names.length,
"The length of provided names doesn't match the length of query schema.")
StructType(query.schema.zip(names).map { case (structField, outputName) =>
structField.copy(name = outputName)
})
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -69,8 +69,8 @@ case class CreateHiveTableAsSelectCommand(
// add the relation into catalog, just in case of failure occurs while data
// processing.
assert(tableDesc.schema.isEmpty)
val schema = DataWritingCommand.logicalPlanSchemaWithNames(query, outputColumnNames)
catalog.createTable(tableDesc.copy(schema = schema), ignoreIfExists = false)
catalog.createTable(
tableDesc.copy(schema = outputColumns.toStructType), ignoreIfExists = false)

try {
// Read back the metadata of the table which was created just now.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -66,7 +66,7 @@ case class InsertIntoHiveDirCommand(
identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")),
tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW,
storage = storage,
schema = query.schema
schema = outputColumns.toStructType
))
hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB,
storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
Expand Down Expand Up @@ -104,8 +104,7 @@ case class InsertIntoHiveDirCommand(
plan = child,
hadoopConf = hadoopConf,
fileSinkConf = fileSinkConf,
outputLocation = tmpPath.toString,
allColumns = outputColumns)
outputLocation = tmpPath.toString)

val fs = writeToPath.getFileSystem(hadoopConf)
if (overwrite && fs.exists(writeToPath)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -198,7 +198,6 @@ case class InsertIntoHiveTable(
hadoopConf = hadoopConf,
fileSinkConf = fileSinkConf,
outputLocation = tmpLocation.toString,
allColumns = outputColumns,
partitionAttributes = partitionAttributes)

if (partition.nonEmpty) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,6 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand {
hadoopConf: Configuration,
fileSinkConf: FileSinkDesc,
outputLocation: String,
allColumns: Seq[Attribute],
customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty,
partitionAttributes: Seq[Attribute] = Nil): Set[String] = {

Expand Down Expand Up @@ -90,7 +89,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand {
fileFormat = new HiveFileFormat(fileSinkConf),
committer = committer,
outputSpec =
FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, allColumns),
FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, outputColumns),
hadoopConf = hadoopConf,
partitionColumns = partitionAttributes,
bucketSpec = None,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -803,6 +803,25 @@ class HiveDDLSuite
}
}

test("SPARK-25313 Insert overwrite directory should output correct schema") {
withSQLConf(CONVERT_METASTORE_PARQUET.key -> "false") {
withTable("tbl") {
withView("view1") {
spark.sql("CREATE TABLE tbl(id long)")
spark.sql("INSERT OVERWRITE TABLE tbl VALUES 4")
spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl")
withTempPath { path =>
spark.sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.getCanonicalPath}' " +
"STORED AS PARQUET SELECT ID FROM view1")
val expectedSchema = StructType(Seq(StructField("ID", LongType, true)))
assert(spark.read.parquet(path.toString).schema == expectedSchema)
checkAnswer(spark.read.parquet(path.toString), Seq(Row(4)))
}
}
}
}
}

test("alter table partition - storage information") {
sql("CREATE TABLE boxes (height INT, length INT) PARTITIONED BY (width INT)")
sql("INSERT OVERWRITE TABLE boxes PARTITION (width=4) SELECT 4, 4")
Expand Down