@@ -22,37 +22,37 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
2222import org .apache .hadoop .conf .{Configurable , Configuration }
2323import org .apache .hadoop .io .Writable
2424import org .apache .hadoop .mapreduce .{JobContext , InputSplit , Job }
25+ import org .apache .spark .sql .catalyst .expressions .codegen .GeneratePredicate
26+
2527import parquet .hadoop .ParquetInputFormat
2628import parquet .hadoop .util .ContextUtil
2729
2830import org .apache .spark .annotation .DeveloperApi
2931import org .apache .spark .{Partition => SparkPartition , Logging }
3032import org .apache .spark .rdd .{NewHadoopPartition , RDD }
31- import org .apache .spark .sql .catalyst .expressions .codegen .GeneratePredicate
33+
34+ import org .apache .spark .sql .{SQLConf , Row , SQLContext }
3235import org .apache .spark .sql .catalyst .expressions ._
33- import org .apache .spark .sql .catalyst .types .{IntegerType , StructField , StructType }
36+ import org .apache .spark .sql .catalyst .types .{StringType , IntegerType , StructField , StructType }
3437import org .apache .spark .sql .sources ._
35- import org .apache .spark .sql .{SQLConf , SQLContext }
3638
3739import scala .collection .JavaConversions ._
3840
39-
4041/**
4142 * Allows creation of parquet based tables using the syntax
4243 * `CREATE TEMPORARY TABLE ... USING org.apache.spark.sql.parquet`. Currently the only option
4344 * required is `path`, which should be the location of a collection of, optionally partitioned,
4445 * parquet files.
4546 */
46- class DefaultSource extends SchemaRelationProvider {
47+ class DefaultSource extends RelationProvider {
4748 /** Returns a new base relation with the given parameters. */
4849 override def createRelation (
4950 sqlContext : SQLContext ,
50- parameters : Map [String , String ],
51- schema : Option [StructType ]): BaseRelation = {
51+ parameters : Map [String , String ]): BaseRelation = {
5252 val path =
5353 parameters.getOrElse(" path" , sys.error(" 'path' must be specified for parquet tables." ))
5454
55- ParquetRelation2 (path, schema )(sqlContext)
55+ ParquetRelation2 (path)(sqlContext)
5656 }
5757}
5858
@@ -82,9 +82,7 @@ private[parquet] case class Partition(partitionValues: Map[String, Any], files:
8282 * discovery.
8383 */
8484@ DeveloperApi
85- case class ParquetRelation2 (
86- path : String ,
87- userSpecifiedSchema : Option [StructType ])(@ transient val sqlContext : SQLContext )
85+ case class ParquetRelation2 (path : String )(@ transient val sqlContext : SQLContext )
8886 extends CatalystScan with Logging {
8987
9088 def sparkContext = sqlContext.sparkContext
@@ -135,13 +133,12 @@ case class ParquetRelation2(
135133
136134 override val sizeInBytes = partitions.flatMap(_.files).map(_.getLen).sum
137135
138- val dataSchema = userSpecifiedSchema.getOrElse(
139- StructType .fromAttributes( // TODO: Parquet code should not deal with attributes.
140- ParquetTypesConverter .readSchemaFromFile(
141- partitions.head.files.head.getPath,
142- Some (sparkContext.hadoopConfiguration),
143- sqlContext.isParquetBinaryAsString))
144- )
136+ val dataSchema = StructType .fromAttributes( // TODO: Parquet code should not deal with attributes.
137+ ParquetTypesConverter .readSchemaFromFile(
138+ partitions.head.files.head.getPath,
139+ Some (sparkContext.hadoopConfiguration),
140+ sqlContext.isParquetBinaryAsString))
141+
145142 val dataIncludesKey =
146143 partitionKeys.headOption.map(dataSchema.fieldNames.contains(_)).getOrElse(true )
147144
0 commit comments