aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorCheng Lian <lian@databricks.com>2016-06-16 10:24:29 -0700
committerWenchen Fan <wenchen@databricks.com>2016-06-16 10:24:29 -0700
commit9ea0d5e326e08b914aa46f1eec8795688a61bf74 (patch)
tree41b7df75ad7d557e09297cacbf4b48f000fa9706 /sql
parent6451cf9270b55465d8ecea4c4031329a1058561a (diff)
downloadspark-9ea0d5e326e08b914aa46f1eec8795688a61bf74.tar.gz
spark-9ea0d5e326e08b914aa46f1eec8795688a61bf74.tar.bz2
spark-9ea0d5e326e08b914aa46f1eec8795688a61bf74.zip
[SPARK-15983][SQL] Removes FileFormat.prepareRead
## What changes were proposed in this pull request? Interface method `FileFormat.prepareRead()` was added in #12088 to handle a special case in the LibSVM data source. However, the semantics of this interface method isn't intuitive: it returns a modified version of the data source options map. Considering that the LibSVM case can be easily handled using schema metadata inside `inferSchema`, we can remove this interface method to keep the `FileFormat` interface clean. ## How was this patch tested? Existing tests. Author: Cheng Lian <lian@databricks.com> Closes #13698 from liancheng/remove-prepare-read.
Diffstat (limited to 'sql')
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala5
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala11
2 files changed, 1 insertions, 15 deletions
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
index d3273025b6..7f3683fc98 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
@@ -386,9 +386,6 @@ case class DataSource(
"It must be specified manually")
}
- val enrichedOptions =
- format.prepareRead(sparkSession, caseInsensitiveOptions, fileCatalog.allFiles())
-
HadoopFsRelation(
sparkSession,
fileCatalog,
@@ -396,7 +393,7 @@ case class DataSource(
dataSchema = dataSchema.asNullable,
bucketSpec = bucketSpec,
format,
- enrichedOptions)
+ caseInsensitiveOptions)
case _ =>
throw new AnalysisException(
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
index 641c5cb02b..4ac555be7f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
@@ -26,7 +26,6 @@ import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompres
import org.apache.hadoop.mapred.{FileInputFormat, JobConf}
import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext}
-import org.apache.spark.SparkContext
import org.apache.spark.annotation.Experimental
import org.apache.spark.internal.Logging
import org.apache.spark.sql._
@@ -187,15 +186,6 @@ trait FileFormat {
files: Seq[FileStatus]): Option[StructType]
/**
- * Prepares a read job and returns a potentially updated data source option [[Map]]. This method
- * can be useful for collecting necessary global information for scanning input data.
- */
- def prepareRead(
- sparkSession: SparkSession,
- options: Map[String, String],
- files: Seq[FileStatus]): Map[String, String] = options
-
- /**
* Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can
* be put here. For example, user defined output committer can be configured here
* by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass.
@@ -454,7 +444,6 @@ private[sql] object HadoopFsRelation extends Logging {
logInfo(s"Listing leaf files and directories in parallel under: ${paths.mkString(", ")}")
val sparkContext = sparkSession.sparkContext
- val sqlConf = sparkSession.sessionState.conf
val serializableConfiguration = new SerializableConfiguration(hadoopConf)
val serializedPaths = paths.map(_.toString)