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
3 changes: 3 additions & 0 deletions project/MimaExcludes.scala
Original file line number Diff line number Diff line change
Expand Up @@ -64,6 +64,9 @@ object MimaExcludes {
excludePackage("org.apache.spark.sql.execution"),
// Parquet support is considered private.
excludePackage("org.apache.spark.sql.parquet"),
// The old JSON RDD is removed in favor of streaming Jackson
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.json.JsonRDD$"),
ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.json.JsonRDD"),
// local function inside a method
ProblemFilters.exclude[MissingMethodProblem](
"org.apache.spark.sql.SQLContext.org$apache$spark$sql$SQLContext$$needsConversion$1")
Expand Down
15 changes: 3 additions & 12 deletions sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ import org.apache.spark.api.java.JavaRDD
import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation}
import org.apache.spark.sql.json.{JsonRDD, JSONRelation}
import org.apache.spark.sql.json.JSONRelation
import org.apache.spark.sql.parquet.ParquetRelation2
import org.apache.spark.sql.sources.{LogicalRelation, ResolvedDataSource}
import org.apache.spark.sql.types.StructType
Expand Down Expand Up @@ -236,17 +236,8 @@ class DataFrameReader private[sql](sqlContext: SQLContext) {
*/
def json(jsonRDD: RDD[String]): DataFrame = {
val samplingRatio = extraOptions.getOrElse("samplingRatio", "1.0").toDouble
if (sqlContext.conf.useJacksonStreamingAPI) {
sqlContext.baseRelationToDataFrame(
new JSONRelation(() => jsonRDD, None, samplingRatio, userSpecifiedSchema)(sqlContext))
} else {
val columnNameOfCorruptJsonRecord = sqlContext.conf.columnNameOfCorruptRecord
val appliedSchema = userSpecifiedSchema.getOrElse(
JsonRDD.nullTypeToStringType(
JsonRDD.inferSchema(jsonRDD, 1.0, columnNameOfCorruptJsonRecord)))
val rowRDD = JsonRDD.jsonStringToRow(jsonRDD, appliedSchema, columnNameOfCorruptJsonRecord)
sqlContext.internalCreateDataFrame(rowRDD, appliedSchema)
}
sqlContext.baseRelationToDataFrame(
new JSONRelation(() => jsonRDD, None, samplingRatio, userSpecifiedSchema)(sqlContext))
}

/**
Expand Down
5 changes: 0 additions & 5 deletions sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala
Original file line number Diff line number Diff line change
Expand Up @@ -401,9 +401,6 @@ private[spark] object SQLConf {
"spark.sql.useSerializer2",
defaultValue = Some(true), isPublic = false)

val USE_JACKSON_STREAMING_API = booleanConf("spark.sql.json.useJacksonStreamingAPI",
defaultValue = Some(true), doc = "<TODO>")

object Deprecated {
val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks"
}
Expand Down Expand Up @@ -473,8 +470,6 @@ private[sql] class SQLConf extends Serializable with CatalystConf {

private[spark] def useSqlSerializer2: Boolean = getConf(USE_SQL_SERIALIZER2)

private[spark] def useJacksonStreamingAPI: Boolean = getConf(USE_JACKSON_STREAMING_API)

private[spark] def autoBroadcastJoinThreshold: Int = getConf(AUTO_BROADCASTJOIN_THRESHOLD)

private[spark] def defaultSizeInBytes: Long =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,51 +157,27 @@ private[sql] class JSONRelation(
}
}

private val useJacksonStreamingAPI: Boolean = sqlContext.conf.useJacksonStreamingAPI

override val needConversion: Boolean = false

override lazy val schema = userSpecifiedSchema.getOrElse {
if (useJacksonStreamingAPI) {
InferSchema(
baseRDD(),
samplingRatio,
sqlContext.conf.columnNameOfCorruptRecord)
} else {
JsonRDD.nullTypeToStringType(
JsonRDD.inferSchema(
baseRDD(),
samplingRatio,
sqlContext.conf.columnNameOfCorruptRecord))
}
InferSchema(
baseRDD(),
samplingRatio,
sqlContext.conf.columnNameOfCorruptRecord)
}

override def buildScan(): RDD[Row] = {
if (useJacksonStreamingAPI) {
JacksonParser(
baseRDD(),
schema,
sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
} else {
JsonRDD.jsonStringToRow(
baseRDD(),
schema,
sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
}
JacksonParser(
baseRDD(),
schema,
sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
}

override def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] = {
if (useJacksonStreamingAPI) {
JacksonParser(
baseRDD(),
StructType.fromAttributes(requiredColumns),
sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
} else {
JsonRDD.jsonStringToRow(
baseRDD(),
StructType.fromAttributes(requiredColumns),
sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
}
JacksonParser(
baseRDD(),
StructType.fromAttributes(requiredColumns),
sqlContext.conf.columnNameOfCorruptRecord).map(_.asInstanceOf[Row])
}

override def insert(data: DataFrame, overwrite: Boolean): Unit = {
Expand Down
Loading