Class DataFrameReader

Object
org.apache.spark.sql.DataFrameReader
All Implemented Interfaces:
org.apache.spark.internal.Logging

public class DataFrameReader extends Object implements org.apache.spark.internal.Logging
Interface used to load a Dataset from external storage systems (e.g. file systems, key-value stores, etc). Use SparkSession.read to access this.

Since:
1.4.0
  • Nested Class Summary

    Nested classes/interfaces inherited from interface org.apache.spark.internal.Logging

    org.apache.spark.internal.Logging.SparkShellLoggingFilter
  • Method Summary

    Modifier and Type
    Method
    Description
    csv(String path)
    Loads a CSV file and returns the result as a DataFrame.
    csv(String... paths)
    Loads CSV files and returns the result as a DataFrame.
    csv(Dataset<String> csvDataset)
    Loads an Dataset[String] storing CSV rows and returns the result as a DataFrame.
    csv(scala.collection.Seq<String> paths)
    Loads CSV files and returns the result as a DataFrame.
    format(String source)
    Specifies the input data source format.
    jdbc(String url, String table, String[] predicates, Properties connectionProperties)
    Construct a DataFrame representing the database table accessible via JDBC URL url named table using connection properties.
    jdbc(String url, String table, String columnName, long lowerBound, long upperBound, int numPartitions, Properties connectionProperties)
    Construct a DataFrame representing the database table accessible via JDBC URL url named table.
    jdbc(String url, String table, Properties properties)
    Construct a DataFrame representing the database table accessible via JDBC URL url named table and connection properties.
    json(String path)
    Loads a JSON file and returns the results as a DataFrame.
    json(String... paths)
    Loads JSON files and returns the results as a DataFrame.
    json(JavaRDD<String> jsonRDD)
    Deprecated.
    Use json(Dataset[String]) instead.
    json(RDD<String> jsonRDD)
    Deprecated.
    Use json(Dataset[String]) instead.
    json(Dataset<String> jsonDataset)
    Loads a Dataset[String] storing JSON objects (JSON Lines text format or newline-delimited JSON) and returns the result as a DataFrame.
    json(scala.collection.Seq<String> paths)
    Loads JSON files and returns the results as a DataFrame.
    Loads input in as a DataFrame, for data sources that don't require a path (e.g. external key-value stores).
    load(String path)
    Loads input in as a DataFrame, for data sources that require a path (e.g. data backed by a local or distributed file system).
    load(String... paths)
    Loads input in as a DataFrame, for data sources that support multiple paths.
    load(scala.collection.Seq<String> paths)
    Loads input in as a DataFrame, for data sources that support multiple paths.
    option(String key, boolean value)
    Adds an input option for the underlying data source.
    option(String key, double value)
    Adds an input option for the underlying data source.
    option(String key, long value)
    Adds an input option for the underlying data source.
    option(String key, String value)
    Adds an input option for the underlying data source.
    Adds input options for the underlying data source.
    options(scala.collection.Map<String,String> options)
    (Scala-specific) Adds input options for the underlying data source.
    orc(String path)
    Loads an ORC file and returns the result as a DataFrame.
    orc(String... paths)
    Loads ORC files and returns the result as a DataFrame.
    orc(scala.collection.Seq<String> paths)
    Loads ORC files and returns the result as a DataFrame.
    Loads a Parquet file, returning the result as a DataFrame.
    parquet(String... paths)
    Loads a Parquet file, returning the result as a DataFrame.
    parquet(scala.collection.Seq<String> paths)
    Loads a Parquet file, returning the result as a DataFrame.
    schema(String schemaString)
    Specifies the schema by using the input DDL-formatted string.
    Specifies the input schema.
    table(String tableName)
    Returns the specified table/view as a DataFrame.
    text(String path)
    Loads text files and returns a DataFrame whose schema starts with a string column named "value", and followed by partitioned columns if there are any.
    text(String... paths)
    Loads text files and returns a DataFrame whose schema starts with a string column named "value", and followed by partitioned columns if there are any.
    text(scala.collection.Seq<String> paths)
    Loads text files and returns a DataFrame whose schema starts with a string column named "value", and followed by partitioned columns if there are any.
    Loads text files and returns a Dataset of String.
    textFile(String... paths)
    Loads text files and returns a Dataset of String.
    textFile(scala.collection.Seq<String> paths)
    Loads text files and returns a Dataset of String.

    Methods inherited from class java.lang.Object

    equals, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait

    Methods inherited from interface org.apache.spark.internal.Logging

    initializeForcefully, initializeLogIfNecessary, initializeLogIfNecessary, initializeLogIfNecessary$default$2, isTraceEnabled, log, logDebug, logDebug, logError, logError, logInfo, logInfo, logName, logTrace, logTrace, logWarning, logWarning, org$apache$spark$internal$Logging$$log_, org$apache$spark$internal$Logging$$log__$eq
  • Method Details

    • csv

      public Dataset<Row> csv(String... paths)
      Loads CSV files and returns the result as a DataFrame.

      This function will go through the input once to determine the input schema if inferSchema is enabled. To avoid going through the entire data once, disable inferSchema option or specify the schema explicitly using schema.

      You can find the CSV-specific options for reading CSV files in Data Source Option in the version you use.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • csv

      public Dataset<Row> csv(String path)
      Loads a CSV file and returns the result as a DataFrame. See the documentation on the other overloaded csv() method for more details.

      Parameters:
      path - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • csv

      public Dataset<Row> csv(Dataset<String> csvDataset)
      Loads an Dataset[String] storing CSV rows and returns the result as a DataFrame.

      If the schema is not specified using schema function and inferSchema option is enabled, this function goes through the input once to determine the input schema.

      If the schema is not specified using schema function and inferSchema option is disabled, it determines the columns as string types and it reads only the first line to determine the names and the number of fields.

      If the enforceSchema is set to false, only the CSV header in the first line is checked to conform specified or inferred schema.

      Parameters:
      csvDataset - input Dataset with one CSV row per record
      Returns:
      (undocumented)
      Since:
      2.2.0
      Note:
      if header option is set to true when calling this API, all lines same with the header will be removed if exists.

    • csv

      public Dataset<Row> csv(scala.collection.Seq<String> paths)
      Loads CSV files and returns the result as a DataFrame.

      This function will go through the input once to determine the input schema if inferSchema is enabled. To avoid going through the entire data once, disable inferSchema option or specify the schema explicitly using schema.

      You can find the CSV-specific options for reading CSV files in Data Source Option in the version you use.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • format

      public DataFrameReader format(String source)
      Specifies the input data source format.

      Parameters:
      source - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • jdbc

      public Dataset<Row> jdbc(String url, String table, Properties properties)
      Construct a DataFrame representing the database table accessible via JDBC URL url named table and connection properties.

      You can find the JDBC-specific option and parameter documentation for reading tables via JDBC in Data Source Option in the version you use.

      Parameters:
      url - (undocumented)
      table - (undocumented)
      properties - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • jdbc

      public Dataset<Row> jdbc(String url, String table, String columnName, long lowerBound, long upperBound, int numPartitions, Properties connectionProperties)
      Construct a DataFrame representing the database table accessible via JDBC URL url named table. Partitions of the table will be retrieved in parallel based on the parameters passed to this function.

      Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash your external database systems.

      You can find the JDBC-specific option and parameter documentation for reading tables via JDBC in Data Source Option in the version you use.

      Parameters:
      table - Name of the table in the external database.
      columnName - Alias of partitionColumn option. Refer to partitionColumn in Data Source Option in the version you use.
      connectionProperties - JDBC database connection arguments, a list of arbitrary string tag/value. Normally at least a "user" and "password" property should be included. "fetchsize" can be used to control the number of rows per fetch and "queryTimeout" can be used to wait for a Statement object to execute to the given number of seconds.
      url - (undocumented)
      lowerBound - (undocumented)
      upperBound - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • jdbc

      public Dataset<Row> jdbc(String url, String table, String[] predicates, Properties connectionProperties)
      Construct a DataFrame representing the database table accessible via JDBC URL url named table using connection properties. The predicates parameter gives a list expressions suitable for inclusion in WHERE clauses; each one defines one partition of the DataFrame.

      Don't create too many partitions in parallel on a large cluster; otherwise Spark might crash your external database systems.

      You can find the JDBC-specific option and parameter documentation for reading tables via JDBC in Data Source Option in the version you use.

      Parameters:
      table - Name of the table in the external database.
      predicates - Condition in the where clause for each partition.
      connectionProperties - JDBC database connection arguments, a list of arbitrary string tag/value. Normally at least a "user" and "password" property should be included. "fetchsize" can be used to control the number of rows per fetch.
      url - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • json

      public Dataset<Row> json(String... paths)
      Loads JSON files and returns the results as a DataFrame.

      JSON Lines (newline-delimited JSON) is supported by default. For JSON (one record per file), set the multiLine option to true.

      This function goes through the input once to determine the input schema. If you know the schema in advance, use the version that specifies the schema to avoid the extra scan.

      You can find the JSON-specific options for reading JSON files in Data Source Option in the version you use.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • json

      public Dataset<Row> json(String path)
      Loads a JSON file and returns the results as a DataFrame.

      See the documentation on the overloaded json() method with varargs for more details.

      Parameters:
      path - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • json

      public Dataset<Row> json(scala.collection.Seq<String> paths)
      Loads JSON files and returns the results as a DataFrame.

      JSON Lines (newline-delimited JSON) is supported by default. For JSON (one record per file), set the multiLine option to true.

      This function goes through the input once to determine the input schema. If you know the schema in advance, use the version that specifies the schema to avoid the extra scan.

      You can find the JSON-specific options for reading JSON files in Data Source Option in the version you use.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • json

      public Dataset<Row> json(JavaRDD<String> jsonRDD)
      Deprecated.
      Use json(Dataset[String]) instead. Since 2.2.0.
      Loads a JavaRDD[String] storing JSON objects (JSON Lines text format or newline-delimited JSON) and returns the result as a DataFrame.

      Unless the schema is specified using schema function, this function goes through the input once to determine the input schema.

      Parameters:
      jsonRDD - input RDD with one JSON object per record
      Returns:
      (undocumented)
      Since:
      1.4.0
    • json

      public Dataset<Row> json(RDD<String> jsonRDD)
      Deprecated.
      Use json(Dataset[String]) instead. Since 2.2.0.
      Loads an RDD[String] storing JSON objects (JSON Lines text format or newline-delimited JSON) and returns the result as a DataFrame.

      Unless the schema is specified using schema function, this function goes through the input once to determine the input schema.

      Parameters:
      jsonRDD - input RDD with one JSON object per record
      Returns:
      (undocumented)
      Since:
      1.4.0
    • json

      public Dataset<Row> json(Dataset<String> jsonDataset)
      Loads a Dataset[String] storing JSON objects (JSON Lines text format or newline-delimited JSON) and returns the result as a DataFrame.

      Unless the schema is specified using schema function, this function goes through the input once to determine the input schema.

      Parameters:
      jsonDataset - input Dataset with one JSON object per record
      Returns:
      (undocumented)
      Since:
      2.2.0
    • load

      public Dataset<Row> load(String... paths)
      Loads input in as a DataFrame, for data sources that support multiple paths. Only works if the source is a HadoopFsRelationProvider.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • load

      public Dataset<Row> load()
      Loads input in as a DataFrame, for data sources that don't require a path (e.g. external key-value stores).

      Returns:
      (undocumented)
      Since:
      1.4.0
    • load

      public Dataset<Row> load(String path)
      Loads input in as a DataFrame, for data sources that require a path (e.g. data backed by a local or distributed file system).

      Parameters:
      path - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • load

      public Dataset<Row> load(scala.collection.Seq<String> paths)
      Loads input in as a DataFrame, for data sources that support multiple paths. Only works if the source is a HadoopFsRelationProvider.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • option

      public DataFrameReader option(String key, String value)
      Adds an input option for the underlying data source.

      All options are maintained in a case-insensitive way in terms of key names. If a new option has the same key case-insensitively, it will override the existing option.

      Parameters:
      key - (undocumented)
      value - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • option

      public DataFrameReader option(String key, boolean value)
      Adds an input option for the underlying data source.

      All options are maintained in a case-insensitive way in terms of key names. If a new option has the same key case-insensitively, it will override the existing option.

      Parameters:
      key - (undocumented)
      value - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • option

      public DataFrameReader option(String key, long value)
      Adds an input option for the underlying data source.

      All options are maintained in a case-insensitive way in terms of key names. If a new option has the same key case-insensitively, it will override the existing option.

      Parameters:
      key - (undocumented)
      value - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • option

      public DataFrameReader option(String key, double value)
      Adds an input option for the underlying data source.

      All options are maintained in a case-insensitive way in terms of key names. If a new option has the same key case-insensitively, it will override the existing option.

      Parameters:
      key - (undocumented)
      value - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • options

      public DataFrameReader options(scala.collection.Map<String,String> options)
      (Scala-specific) Adds input options for the underlying data source.

      All options are maintained in a case-insensitive way in terms of key names. If a new option has the same key case-insensitively, it will override the existing option.

      Parameters:
      options - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • options

      public DataFrameReader options(Map<String,String> options)
      Adds input options for the underlying data source.

      All options are maintained in a case-insensitive way in terms of key names. If a new option has the same key case-insensitively, it will override the existing option.

      Parameters:
      options - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • orc

      public Dataset<Row> orc(String... paths)
      Loads ORC files and returns the result as a DataFrame.

      ORC-specific option(s) for reading ORC files can be found in Data Source Option in the version you use.

      Parameters:
      paths - input paths
      Returns:
      (undocumented)
      Since:
      2.0.0
    • orc

      public Dataset<Row> orc(String path)
      Loads an ORC file and returns the result as a DataFrame.

      Parameters:
      path - input path
      Returns:
      (undocumented)
      Since:
      1.5.0
    • orc

      public Dataset<Row> orc(scala.collection.Seq<String> paths)
      Loads ORC files and returns the result as a DataFrame.

      ORC-specific option(s) for reading ORC files can be found in Data Source Option in the version you use.

      Parameters:
      paths - input paths
      Returns:
      (undocumented)
      Since:
      2.0.0
    • parquet

      public Dataset<Row> parquet(String... paths)
      Loads a Parquet file, returning the result as a DataFrame.

      Parquet-specific option(s) for reading Parquet files can be found in Data Source Option in the version you use.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • parquet

      public Dataset<Row> parquet(String path)
      Loads a Parquet file, returning the result as a DataFrame. See the documentation on the other overloaded parquet() method for more details.

      Parameters:
      path - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • parquet

      public Dataset<Row> parquet(scala.collection.Seq<String> paths)
      Loads a Parquet file, returning the result as a DataFrame.

      Parquet-specific option(s) for reading Parquet files can be found in Data Source Option in the version you use.

      Parameters:
      paths - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • schema

      public DataFrameReader schema(StructType schema)
      Specifies the input schema. Some data sources (e.g. JSON) can infer the input schema automatically from data. By specifying the schema here, the underlying data source can skip the schema inference step, and thus speed up data loading.

      Parameters:
      schema - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.4.0
    • schema

      public DataFrameReader schema(String schemaString)
      Specifies the schema by using the input DDL-formatted string. Some data sources (e.g. JSON) can infer the input schema automatically from data. By specifying the schema here, the underlying data source can skip the schema inference step, and thus speed up data loading.

      
         spark.read.schema("a INT, b STRING, c DOUBLE").csv("test.csv")
       

      Parameters:
      schemaString - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • table

      public Dataset<Row> table(String tableName)
      Returns the specified table/view as a DataFrame. If it's a table, it must support batch reading and the returned DataFrame is the batch scan query plan of this table. If it's a view, the returned DataFrame is simply the query plan of the view, which can either be a batch or streaming query plan.

      Parameters:
      tableName - is either a qualified or unqualified name that designates a table or view. If a database is specified, it identifies the table/view from the database. Otherwise, it first attempts to find a temporary view with the given name and then match the table/view from the current database. Note that, the global temporary view database is also valid here.
      Returns:
      (undocumented)
      Since:
      1.4.0
    • text

      public Dataset<Row> text(String... paths)
      Loads text files and returns a DataFrame whose schema starts with a string column named "value", and followed by partitioned columns if there are any. The text files must be encoded as UTF-8.

      By default, each line in the text files is a new row in the resulting DataFrame. For example:

      
         // Scala:
         spark.read.text("/path/to/spark/README.md")
      
         // Java:
         spark.read().text("/path/to/spark/README.md")
       

      You can find the text-specific options for reading text files in Data Source Option in the version you use.

      Parameters:
      paths - input paths
      Returns:
      (undocumented)
      Since:
      1.6.0
    • text

      public Dataset<Row> text(String path)
      Loads text files and returns a DataFrame whose schema starts with a string column named "value", and followed by partitioned columns if there are any. See the documentation on the other overloaded text() method for more details.

      Parameters:
      path - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • text

      public Dataset<Row> text(scala.collection.Seq<String> paths)
      Loads text files and returns a DataFrame whose schema starts with a string column named "value", and followed by partitioned columns if there are any. The text files must be encoded as UTF-8.

      By default, each line in the text files is a new row in the resulting DataFrame. For example:

      
         // Scala:
         spark.read.text("/path/to/spark/README.md")
      
         // Java:
         spark.read().text("/path/to/spark/README.md")
       

      You can find the text-specific options for reading text files in Data Source Option in the version you use.

      Parameters:
      paths - input paths
      Returns:
      (undocumented)
      Since:
      1.6.0
    • textFile

      public Dataset<String> textFile(String... paths)
      Loads text files and returns a Dataset of String. The underlying schema of the Dataset contains a single string column named "value". The text files must be encoded as UTF-8.

      If the directory structure of the text files contains partitioning information, those are ignored in the resulting Dataset. To include partitioning information as columns, use text.

      By default, each line in the text files is a new row in the resulting DataFrame. For example:

      
         // Scala:
         spark.read.textFile("/path/to/spark/README.md")
      
         // Java:
         spark.read().textFile("/path/to/spark/README.md")
       

      You can set the text-specific options as specified in DataFrameReader.text.

      Parameters:
      paths - input path
      Returns:
      (undocumented)
      Since:
      2.0.0
    • textFile

      public Dataset<String> textFile(String path)
      Loads text files and returns a Dataset of String. See the documentation on the other overloaded textFile() method for more details.
      Parameters:
      path - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • textFile

      public Dataset<String> textFile(scala.collection.Seq<String> paths)
      Loads text files and returns a Dataset of String. The underlying schema of the Dataset contains a single string column named "value". The text files must be encoded as UTF-8.

      If the directory structure of the text files contains partitioning information, those are ignored in the resulting Dataset. To include partitioning information as columns, use text.

      By default, each line in the text files is a new row in the resulting DataFrame. For example:

      
         // Scala:
         spark.read.textFile("/path/to/spark/README.md")
      
         // Java:
         spark.read().textFile("/path/to/spark/README.md")
       

      You can set the text-specific options as specified in DataFrameReader.text.

      Parameters:
      paths - input path
      Returns:
      (undocumented)
      Since:
      2.0.0