1818package org .apache .spark .sql .execution .datasources
1919
2020import java .io .IOException
21- import java .util .{Date , UUID }
22-
23- import scala .collection .JavaConversions .asScalaIterator
2421
2522import org .apache .hadoop .fs .Path
2623import org .apache .hadoop .mapreduce ._
27- import org .apache .hadoop .mapreduce .lib .output .{ FileOutputCommitter => MapReduceFileOutputCommitter , FileOutputFormat }
24+ import org .apache .hadoop .mapreduce .lib .output .FileOutputFormat
2825import org .apache .spark ._
29- import org .apache .spark .mapred .SparkHadoopMapRedUtil
30- import org .apache .spark .mapreduce .SparkHadoopMapReduceUtil
3126import org .apache .spark .sql ._
3227import org .apache .spark .sql .catalyst .analysis .UnresolvedAttribute
33- import org .apache .spark .sql .catalyst .expressions ._
34- import org .apache .spark .sql .catalyst .expressions .codegen .GenerateProjection
3528import org .apache .spark .sql .catalyst .plans .logical .{LogicalPlan , Project }
36- import org .apache .spark .sql .catalyst .{ CatalystTypeConverters , InternalRow }
29+ import org .apache .spark .sql .catalyst .InternalRow
3730import org .apache .spark .sql .execution .{RunnableCommand , SQLExecution }
3831import org .apache .spark .sql .sources ._
39- import org .apache .spark .sql .types .StringType
40- import org .apache .spark .util .{Utils , SerializableConfiguration }
32+ import org .apache .spark .util .Utils
4133
4234
4335/**
@@ -109,14 +101,11 @@ private[sql] case class InsertIntoHadoopFsRelation(
109101 // We create a DataFrame by applying the schema of relation to the data to make sure.
110102 // We are writing data based on the expected schema,
111103
112- // For partitioned relation r, r.schema's column ordering can be different from the column
113- // ordering of data.logicalPlan (partition columns are all moved after data column). We
114- // need a Project to adjust the ordering, so that inside InsertIntoHadoopFsRelation, we can
115- // safely apply the schema of r.schema to the data.
116-
104+ // A partitioned relation schema's can be different from the input logicalPlan, since
105+ // partition columns are all moved after data column. We Project to adjust the ordering.
117106 // TODO: this belongs in the analyzer.
118107 val project = Project (
119- relation.schema.map(field => new UnresolvedAttribute ( Seq ( field.name) )), query)
108+ relation.schema.map(field => UnresolvedAttribute .quoted( field.name)), query)
120109 val queryExecution = DataFrame (sqlContext, project).queryExecution
121110
122111 SQLExecution .withNewExecutionId(sqlContext, queryExecution) {
@@ -128,14 +117,14 @@ private[sql] case class InsertIntoHadoopFsRelation(
128117 df.schema == relation.schema,
129118 s """ DataFrame must have the same schema as the relation to which is inserted.
130119 |DataFrame schema: ${df.schema}
131- |Relation schema: ${relation.schema}
120+ |Relation schema: ${relation.schema}
132121 """ .stripMargin)
133122 val partitionColumnsInSpec = relation.partitionColumns.fieldNames
134123 require(
135124 partitionColumnsInSpec.sameElements(partitionColumns),
136125 s """ Partition columns mismatch.
137126 |Expected: ${partitionColumnsInSpec.mkString(" , " )}
138- |Actual: ${partitionColumns.mkString(" , " )}
127+ |Actual: ${partitionColumns.mkString(" , " )}
139128 """ .stripMargin)
140129
141130 val writerContainer = if (partitionColumns.isEmpty) {
0 commit comments