Packages

class SparkSession extends Serializable with Closeable with Logging

The entry point to programming Spark with the Dataset and DataFrame API.

In environments that this has been created upfront (e.g. REPL, notebooks), use the builder to get an existing session:

SparkSession.builder().getOrCreate()

The builder can also be used to create a new session:

SparkSession.builder
  .master("local")
  .appName("Word Count")
  .config("spark.some.config.option", "some-value")
  .getOrCreate()
Self Type
SparkSession
Annotations
@Stable()
Linear Supertypes
Logging, Closeable, AutoCloseable, Serializable, Serializable, AnyRef, Any
Ordering
  1. Alphabetic
  2. By Inheritance
Inherited
  1. SparkSession
  2. Logging
  3. Closeable
  4. AutoCloseable
  5. Serializable
  6. Serializable
  7. AnyRef
  8. Any
  1. Hide All
  2. Show All
Visibility
  1. Public
  2. All

Value Members

  1. final def !=(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  2. final def ##(): Int
    Definition Classes
    AnyRef → Any
  3. final def ==(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  4. final def asInstanceOf[T0]: T0
    Definition Classes
    Any
  5. def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame

    Convert a BaseRelation created for external data sources into a DataFrame.

    Convert a BaseRelation created for external data sources into a DataFrame.

    Since

    2.0.0

  6. lazy val catalog: Catalog

    Interface through which the user may create, drop, alter or query underlying databases, tables, functions etc.

    Interface through which the user may create, drop, alter or query underlying databases, tables, functions etc.

    Annotations
    @transient()
    Since

    2.0.0

  7. def clone(): AnyRef
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @native() @throws( ... )
  8. def close(): Unit

    Synonym for stop().

    Synonym for stop().

    Definition Classes
    SparkSession → Closeable → AutoCloseable
    Since

    2.1.0

  9. lazy val conf: RuntimeConfig

    Runtime configuration interface for Spark.

    Runtime configuration interface for Spark.

    This is the interface through which the user can get and set all Spark and Hadoop configurations that are relevant to Spark SQL. When getting the value of a config, this defaults to the value set in the underlying SparkContext, if any.

    Annotations
    @transient()
    Since

    2.0.0

  10. def createDataFrame(data: List[_], beanClass: Class[_]): DataFrame

    Applies a schema to a List of Java Beans.

    Applies a schema to a List of Java Beans.

    WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries will return the columns in an undefined order.

    Since

    1.6.0

  11. def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame

    Applies a schema to an RDD of Java Beans.

    Applies a schema to an RDD of Java Beans.

    WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries will return the columns in an undefined order.

    Since

    2.0.0

  12. def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame

    Applies a schema to an RDD of Java Beans.

    Applies a schema to an RDD of Java Beans.

    WARNING: Since there is no guaranteed ordering for fields in a Java Bean, SELECT * queries will return the columns in an undefined order.

    Since

    2.0.0

  13. def createDataFrame(rows: List[Row], schema: StructType): DataFrame

    :: DeveloperApi :: Creates a DataFrame from a java.util.List containing Rows using the given schema.

    :: DeveloperApi :: Creates a DataFrame from a java.util.List containing Rows using the given schema. It is important to make sure that the structure of every Row of the provided List matches the provided schema. Otherwise, there will be runtime exception.

    Annotations
    @DeveloperApi() @Evolving()
    Since

    2.0.0

  14. def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame

    :: DeveloperApi :: Creates a DataFrame from a JavaRDD containing Rows using the given schema.

    :: DeveloperApi :: Creates a DataFrame from a JavaRDD containing Rows using the given schema. It is important to make sure that the structure of every Row of the provided RDD matches the provided schema. Otherwise, there will be runtime exception.

    Annotations
    @DeveloperApi() @Evolving()
    Since

    2.0.0

  15. def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame

    :: DeveloperApi :: Creates a DataFrame from an RDD containing Rows using the given schema.

    :: DeveloperApi :: Creates a DataFrame from an RDD containing Rows using the given schema. It is important to make sure that the structure of every Row of the provided RDD matches the provided schema. Otherwise, there will be runtime exception. Example:

    import org.apache.spark.sql._
    import org.apache.spark.sql.types._
    val sparkSession = new org.apache.spark.sql.SparkSession(sc)
    
    val schema =
      StructType(
        StructField("name", StringType, false) ::
        StructField("age", IntegerType, true) :: Nil)
    
    val people =
      sc.textFile("examples/src/main/resources/people.txt").map(
        _.split(",")).map(p => Row(p(0), p(1).trim.toInt))
    val dataFrame = sparkSession.createDataFrame(people, schema)
    dataFrame.printSchema
    // root
    // |-- name: string (nullable = false)
    // |-- age: integer (nullable = true)
    
    dataFrame.createOrReplaceTempView("people")
    sparkSession.sql("select name from people").collect.foreach(println)
    Annotations
    @DeveloperApi() @Evolving()
    Since

    2.0.0

  16. def createDataFrame[A <: Product](data: Seq[A])(implicit arg0: scala.reflect.api.JavaUniverse.TypeTag[A]): DataFrame

    :: Experimental :: Creates a DataFrame from a local Seq of Product.

    :: Experimental :: Creates a DataFrame from a local Seq of Product.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  17. def createDataFrame[A <: Product](rdd: RDD[A])(implicit arg0: scala.reflect.api.JavaUniverse.TypeTag[A]): DataFrame

    :: Experimental :: Creates a DataFrame from an RDD of Product (e.g.

    :: Experimental :: Creates a DataFrame from an RDD of Product (e.g. case classes, tuples).

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  18. def createDataset[T](data: List[T])(implicit arg0: Encoder[T]): Dataset[T]

    :: Experimental :: Creates a Dataset from a java.util.List of a given type.

    :: Experimental :: Creates a Dataset from a java.util.List of a given type. This method requires an encoder (to convert a JVM object of type T to and from the internal Spark SQL representation) that is generally created automatically through implicits from a SparkSession, or can be created explicitly by calling static methods on Encoders.

    Java Example

    List<String> data = Arrays.asList("hello", "world");
    Dataset<String> ds = spark.createDataset(data, Encoders.STRING());
    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  19. def createDataset[T](data: RDD[T])(implicit arg0: Encoder[T]): Dataset[T]

    :: Experimental :: Creates a Dataset from an RDD of a given type.

    :: Experimental :: Creates a Dataset from an RDD of a given type. This method requires an encoder (to convert a JVM object of type T to and from the internal Spark SQL representation) that is generally created automatically through implicits from a SparkSession, or can be created explicitly by calling static methods on Encoders.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  20. def createDataset[T](data: Seq[T])(implicit arg0: Encoder[T]): Dataset[T]

    :: Experimental :: Creates a Dataset from a local Seq of data of a given type.

    :: Experimental :: Creates a Dataset from a local Seq of data of a given type. This method requires an encoder (to convert a JVM object of type T to and from the internal Spark SQL representation) that is generally created automatically through implicits from a SparkSession, or can be created explicitly by calling static methods on Encoders.

    Example

    import spark.implicits._
    case class Person(name: String, age: Long)
    val data = Seq(Person("Michael", 29), Person("Andy", 30), Person("Justin", 19))
    val ds = spark.createDataset(data)
    
    ds.show()
    // +-------+---+
    // |   name|age|
    // +-------+---+
    // |Michael| 29|
    // |   Andy| 30|
    // | Justin| 19|
    // +-------+---+
    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  21. lazy val emptyDataFrame: DataFrame

    Returns a DataFrame with no rows or columns.

    Returns a DataFrame with no rows or columns.

    Annotations
    @transient()
    Since

    2.0.0

  22. def emptyDataset[T](implicit arg0: Encoder[T]): Dataset[T]

    :: Experimental :: Creates a new Dataset of type T containing zero elements.

    :: Experimental :: Creates a new Dataset of type T containing zero elements.

    returns

    2.0.0

    Annotations
    @Experimental() @Evolving()
  23. final def eq(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  24. def equals(arg0: Any): Boolean
    Definition Classes
    AnyRef → Any
  25. def experimental: ExperimentalMethods

    :: Experimental :: A collection of methods that are considered experimental, but can be used to hook into the query planner for advanced functionality.

    :: Experimental :: A collection of methods that are considered experimental, but can be used to hook into the query planner for advanced functionality.

    Annotations
    @Experimental() @Unstable()
    Since

    2.0.0

  26. def finalize(): Unit
    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  27. final def getClass(): Class[_]
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  28. def hashCode(): Int
    Definition Classes
    AnyRef → Any
    Annotations
    @native()
  29. def initializeLogIfNecessary(isInterpreter: Boolean, silent: Boolean = false): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  30. def initializeLogIfNecessary(isInterpreter: Boolean): Unit
    Attributes
    protected
    Definition Classes
    Logging
  31. final def isInstanceOf[T0]: Boolean
    Definition Classes
    Any
  32. def isTraceEnabled(): Boolean
    Attributes
    protected
    Definition Classes
    Logging
  33. def listenerManager: ExecutionListenerManager

    :: Experimental :: An interface to register custom org.apache.spark.sql.util.QueryExecutionListeners that listen for execution metrics.

    :: Experimental :: An interface to register custom org.apache.spark.sql.util.QueryExecutionListeners that listen for execution metrics.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  34. def log: Logger
    Attributes
    protected
    Definition Classes
    Logging
  35. def logDebug(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  36. def logDebug(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  37. def logError(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  38. def logError(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  39. def logInfo(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  40. def logInfo(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  41. def logName: String
    Attributes
    protected
    Definition Classes
    Logging
  42. def logTrace(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  43. def logTrace(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  44. def logWarning(msg: ⇒ String, throwable: Throwable): Unit
    Attributes
    protected
    Definition Classes
    Logging
  45. def logWarning(msg: ⇒ String): Unit
    Attributes
    protected
    Definition Classes
    Logging
  46. final def ne(arg0: AnyRef): Boolean
    Definition Classes
    AnyRef
  47. def newSession(): SparkSession

    Start a new session with isolated SQL configurations, temporary tables, registered functions are isolated, but sharing the underlying SparkContext and cached data.

    Start a new session with isolated SQL configurations, temporary tables, registered functions are isolated, but sharing the underlying SparkContext and cached data.

    Since

    2.0.0

    Note

    Other than the SparkContext, all shared state is initialized lazily. This method will force the initialization of the shared state to ensure that parent and child sessions are set up with the same shared state. If the underlying catalog implementation is Hive, this will initialize the metastore, which may take some time.

  48. final def notify(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  49. final def notifyAll(): Unit
    Definition Classes
    AnyRef
    Annotations
    @native()
  50. def parseDataType(dataTypeString: String): DataType

    Parses the data type in our internal string representation.

    Parses the data type in our internal string representation. The data type string should have the same format as the one generated by toString in scala. It is only used by PySpark.

    Attributes
    protected[org.apache.spark.sql]
  51. def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[Long]

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with a step value, with partition number specified.

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with a step value, with partition number specified.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  52. def range(start: Long, end: Long, step: Long): Dataset[Long]

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with a step value.

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with a step value.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  53. def range(start: Long, end: Long): Dataset[Long]

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with step value 1.

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with step value 1.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  54. def range(end: Long): Dataset[Long]

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from 0 to end (exclusive) with step value 1.

    :: Experimental :: Creates a Dataset with a single LongType column named id, containing elements in a range from 0 to end (exclusive) with step value 1.

    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

  55. def read: DataFrameReader

    Returns a DataFrameReader that can be used to read non-streaming data in as a DataFrame.

    Returns a DataFrameReader that can be used to read non-streaming data in as a DataFrame.

    sparkSession.read.parquet("/path/to/file.parquet")
    sparkSession.read.schema(schema).json("/path/to/file.json")
    Since

    2.0.0

  56. def readStream: DataStreamReader

    Returns a DataStreamReader that can be used to read streaming data in as a DataFrame.

    Returns a DataStreamReader that can be used to read streaming data in as a DataFrame.

    sparkSession.readStream.parquet("/path/to/directory/of/parquet/files")
    sparkSession.readStream.schema(schema).json("/path/to/directory/of/json/files")
    Annotations
    @Evolving()
    Since

    2.0.0

  57. lazy val sessionState: SessionState

    State isolated across sessions, including SQL configurations, temporary tables, registered functions, and everything else that accepts a org.apache.spark.sql.internal.SQLConf.

    State isolated across sessions, including SQL configurations, temporary tables, registered functions, and everything else that accepts a org.apache.spark.sql.internal.SQLConf. If parentSessionState is not null, the SessionState will be a copy of the parent.

    This is internal to Spark and there is no guarantee on interface stability.

    Annotations
    @Unstable() @transient()
    Since

    2.2.0

  58. lazy val sharedState: SharedState

    State shared across sessions, including the SparkContext, cached data, listener, and a catalog that interacts with external systems.

    State shared across sessions, including the SparkContext, cached data, listener, and a catalog that interacts with external systems.

    This is internal to Spark and there is no guarantee on interface stability.

    Annotations
    @Unstable() @transient()
    Since

    2.2.0

  59. val sparkContext: SparkContext
  60. def sql(sqlText: String): DataFrame

    Executes a SQL query using Spark, returning the result as a DataFrame.

    Executes a SQL query using Spark, returning the result as a DataFrame. The dialect that is used for SQL parsing can be configured with 'spark.sql.dialect'.

    Since

    2.0.0

  61. val sqlContext: SQLContext

    A wrapped version of this session in the form of a SQLContext, for backward compatibility.

    A wrapped version of this session in the form of a SQLContext, for backward compatibility.

    Since

    2.0.0

  62. def stop(): Unit

    Stop the underlying SparkContext.

    Stop the underlying SparkContext.

    Since

    2.0.0

  63. def streams: StreamingQueryManager

    :: Experimental :: Returns a StreamingQueryManager that allows managing all the StreamingQuerys active on this.

    :: Experimental :: Returns a StreamingQueryManager that allows managing all the StreamingQuerys active on this.

    Annotations
    @Experimental() @Unstable()
    Since

    2.0.0

  64. final def synchronized[T0](arg0: ⇒ T0): T0
    Definition Classes
    AnyRef
  65. def table(tableName: String): DataFrame

    Returns the specified table/view as a DataFrame.

    Returns the specified table/view as a DataFrame.

    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.

    Since

    2.0.0

  66. def time[T](f: ⇒ T): T

    Executes some code block and prints to stdout the time taken to execute the block.

    Executes some code block and prints to stdout the time taken to execute the block. This is available in Scala only and is used primarily for interactive testing and debugging.

    Since

    2.1.0

  67. def toString(): String
    Definition Classes
    AnyRef → Any
  68. def udf: UDFRegistration

    A collection of methods for registering user-defined functions (UDF).

    A collection of methods for registering user-defined functions (UDF).

    The following example registers a Scala closure as UDF:

    sparkSession.udf.register("myUDF", (arg1: Int, arg2: String) => arg2 + arg1)

    The following example registers a UDF in Java:

    sparkSession.udf().register("myUDF",
        (Integer arg1, String arg2) -> arg2 + arg1,
        DataTypes.StringType);
    Since

    2.0.0

    Note

    The user-defined functions must be deterministic. Due to optimization, duplicate invocations may be eliminated or the function may even be invoked more times than it is present in the query.

  69. def version: String

    The version of Spark on which this application is running.

    The version of Spark on which this application is running.

    Since

    2.0.0

  70. final def wait(): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  71. final def wait(arg0: Long, arg1: Int): Unit
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  72. final def wait(arg0: Long): Unit
    Definition Classes
    AnyRef
    Annotations
    @native() @throws( ... )
  73. object implicits extends SQLImplicits with Serializable

    :: Experimental :: (Scala-specific) Implicit methods available in Scala for converting common Scala objects into DataFrames.

    :: Experimental :: (Scala-specific) Implicit methods available in Scala for converting common Scala objects into DataFrames.

    val sparkSession = SparkSession.builder.getOrCreate()
    import sparkSession.implicits._
    Annotations
    @Experimental() @Evolving()
    Since

    2.0.0

Inherited from Logging

Inherited from Closeable

Inherited from AutoCloseable

Inherited from Serializable

Inherited from Serializable

Inherited from AnyRef

Inherited from Any

Ungrouped