@@ -24,17 +24,17 @@ import scala.collection.JavaConverters._
2424import org .apache .hadoop .fs .Path
2525import org .apache .hadoop .util .StringUtils
2626
27+ import org .apache .spark .{Logging , Partition }
2728import org .apache .spark .annotation .Experimental
2829import org .apache .spark .api .java .JavaRDD
2930import org .apache .spark .deploy .SparkHadoopUtil
3031import org .apache .spark .rdd .RDD
32+ import org .apache .spark .sql .catalyst .SqlParser
3133import org .apache .spark .sql .execution .datasources .jdbc .{JDBCPartition , JDBCPartitioningInfo , JDBCRelation }
32- import org .apache .spark .sql .execution .datasources .json .{ JSONOptions , JSONRelation }
34+ import org .apache .spark .sql .execution .datasources .json .JSONRelation
3335import org .apache .spark .sql .execution .datasources .parquet .ParquetRelation
3436import org .apache .spark .sql .execution .datasources .{LogicalRelation , ResolvedDataSource }
3537import org .apache .spark .sql .types .StructType
36- import org .apache .spark .{Logging , Partition }
37- import org .apache .spark .sql .catalyst .{SqlParser , TableIdentifier }
3838
3939/**
4040 * :: Experimental ::
@@ -104,6 +104,7 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging {
104104 *
105105 * @since 1.4.0
106106 */
107+ // TODO: Remove this one in Spark 2.0.
107108 def load (path : String ): DataFrame = {
108109 option(" path" , path).load()
109110 }
@@ -130,7 +131,8 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging {
130131 *
131132 * @since 1.6.0
132133 */
133- def load (paths : Array [String ]): DataFrame = {
134+ @ scala.annotation.varargs
135+ def load (paths : String * ): DataFrame = {
134136 option(" paths" , paths.map(StringUtils .escapeString(_, '\\ ' , ',' )).mkString(" ," )).load()
135137 }
136138
@@ -236,11 +238,30 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging {
236238 * <li>`allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers
237239 * (e.g. 00012)</li>
238240 *
239- * @param path input path
240241 * @since 1.4.0
241242 */
243+ // TODO: Remove this one in Spark 2.0.
242244 def json (path : String ): DataFrame = format(" json" ).load(path)
243245
246+ /**
247+ * Loads a JSON file (one object per line) and returns the result as a [[DataFrame ]].
248+ *
249+ * This function goes through the input once to determine the input schema. If you know the
250+ * schema in advance, use the version that specifies the schema to avoid the extra scan.
251+ *
252+ * You can set the following JSON-specific options to deal with non-standard JSON files:
253+ * <li>`primitivesAsString` (default `false`): infers all primitive values as a string type</li>
254+ * <li>`allowComments` (default `false`): ignores Java/C++ style comment in JSON records</li>
255+ * <li>`allowUnquotedFieldNames` (default `false`): allows unquoted JSON field names</li>
256+ * <li>`allowSingleQuotes` (default `true`): allows single quotes in addition to double quotes
257+ * </li>
258+ * <li>`allowNumericLeadingZeros` (default `false`): allows leading zeros in numbers
259+ * (e.g. 00012)</li>
260+ *
261+ * @since 1.6.0
262+ */
263+ def json (paths : String * ): DataFrame = format(" json" ).load(paths : _* )
264+
244265 /**
245266 * Loads an `JavaRDD[String]` storing JSON objects (one object per record) and
246267 * returns the result as a [[DataFrame ]].
@@ -328,10 +349,11 @@ class DataFrameReader private[sql](sqlContext: SQLContext) extends Logging {
328349 * sqlContext.read().text("/path/to/spark/README.md")
329350 * }}}
330351 *
331- * @param path input path
352+ * @param paths input path
332353 * @since 1.6.0
333354 */
334- def text (path : String ): DataFrame = format(" text" ).load(path)
355+ @ scala.annotation.varargs
356+ def text (paths : String * ): DataFrame = format(" text" ).load(paths : _* )
335357
336358 // /////////////////////////////////////////////////////////////////////////////////////
337359 // Builder pattern config options
0 commit comments