Skip to content

Commit 65e9c73

Browse files
committed
Revert all changes since applying a given schema has not been testd.
1 parent a852b10 commit 65e9c73

File tree

1 file changed

+15
-18
lines changed

1 file changed

+15
-18
lines changed

sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala

Lines changed: 15 additions & 18 deletions
Original file line numberDiff line numberDiff line change
@@ -22,37 +22,37 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path}
2222
import org.apache.hadoop.conf.{Configurable, Configuration}
2323
import org.apache.hadoop.io.Writable
2424
import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job}
25+
import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate
26+
2527
import parquet.hadoop.ParquetInputFormat
2628
import parquet.hadoop.util.ContextUtil
2729

2830
import org.apache.spark.annotation.DeveloperApi
2931
import org.apache.spark.{Partition => SparkPartition, Logging}
3032
import 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}
3235
import 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}
3437
import org.apache.spark.sql.sources._
35-
import org.apache.spark.sql.{SQLConf, SQLContext}
3638

3739
import 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

Comments
 (0)