Class SparkSession

Object
org.apache.spark.sql.SparkSession
All Implemented Interfaces:
Closeable, Serializable, AutoCloseable, org.apache.spark.internal.Logging

public class SparkSession extends Object implements Serializable, Closeable, org.apache.spark.internal.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()
 

param: sparkContext The Spark context associated with this Spark session. param: existingSharedState If supplied, use the existing shared state instead of creating a new one. param: parentSessionState If supplied, inherit all session state (i.e. temporary views, SQL config, UDFs etc) from parent.

See Also:
  • Method Details

    • builder

      public static SparkSession.Builder builder()
      Creates a SparkSession.Builder for constructing a SparkSession.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • setActiveSession

      public static void setActiveSession(SparkSession session)
      Changes the SparkSession that will be returned in this thread and its children when SparkSession.getOrCreate() is called. This can be used to ensure that a given thread receives a SparkSession with an isolated session, instead of the global (first created) context.

      Parameters:
      session - (undocumented)
      Since:
      2.0.0
    • clearActiveSession

      public static void clearActiveSession()
      Clears the active SparkSession for current thread. Subsequent calls to getOrCreate will return the first created context instead of a thread-local override.

      Since:
      2.0.0
    • setDefaultSession

      public static void setDefaultSession(SparkSession session)
      Sets the default SparkSession that is returned by the builder.

      Parameters:
      session - (undocumented)
      Since:
      2.0.0
    • clearDefaultSession

      public static void clearDefaultSession()
      Clears the default SparkSession that is returned by the builder.

      Since:
      2.0.0
    • getActiveSession

      public static scala.Option<SparkSession> getActiveSession()
      Returns the active SparkSession for the current thread, returned by the builder.

      Returns:
      (undocumented)
      Since:
      2.2.0
      Note:
      Return None, when calling this function on executors

    • getDefaultSession

      public static scala.Option<SparkSession> getDefaultSession()
      Returns the default SparkSession that is returned by the builder.

      Returns:
      (undocumented)
      Since:
      2.2.0
      Note:
      Return None, when calling this function on executors

    • active

      public static SparkSession active()
      Returns the currently active SparkSession, otherwise the default one. If there is no default SparkSession, throws an exception.

      Returns:
      (undocumented)
      Since:
      2.4.0
    • org$apache$spark$internal$Logging$$log_

      public static org.slf4j.Logger org$apache$spark$internal$Logging$$log_()
    • org$apache$spark$internal$Logging$$log__$eq

      public static void org$apache$spark$internal$Logging$$log__$eq(org.slf4j.Logger x$1)
    • LogStringContext

      public static org.apache.spark.internal.Logging.LogStringContext LogStringContext(scala.StringContext sc)
    • implicits

      public SparkSession.implicits$ implicits()
      Accessor for nested Scala object
      Returns:
      (undocumented)
    • sparkContext

      public SparkContext sparkContext()
    • version

      public String version()
      The version of Spark on which this application is running.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • sharedState

      public org.apache.spark.sql.internal.SharedState sharedState()
    • sessionState

      public org.apache.spark.sql.internal.SessionState sessionState()
    • sqlContext

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

      Returns:
      (undocumented)
      Since:
      2.0.0
    • conf

      public RuntimeConfig conf()
    • listenerManager

      public ExecutionListenerManager listenerManager()
      An interface to register custom QueryExecutionListeners that listen for execution metrics.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • experimental

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

      Returns:
      (undocumented)
      Since:
      2.0.0
    • udf

      public UDFRegistration 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);
       

      Returns:
      (undocumented)
      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.

    • udtf

      public UDTFRegistration udtf()
    • dataSource

      public DataSourceRegistration dataSource()
      A collection of methods for registering user-defined data sources.

      Returns:
      (undocumented)
      Since:
      4.0.0
    • streams

      public StreamingQueryManager streams()
      Returns a StreamingQueryManager that allows managing all the StreamingQuerys active on this.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • newSession

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

      Returns:
      (undocumented)
      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.

    • emptyDataFrame

      public Dataset<Row> emptyDataFrame()
    • emptyDataset

      public <T> Dataset<T> emptyDataset(Encoder<T> evidence$1)
      Creates a new Dataset of type T containing zero elements.

      Parameters:
      evidence$1 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public <A extends scala.Product> Dataset<Row> createDataFrame(RDD<A> rdd, scala.reflect.api.TypeTags.TypeTag<A> evidence$2)
      Creates a DataFrame from an RDD of Product (e.g. case classes, tuples).

      Parameters:
      rdd - (undocumented)
      evidence$2 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public <A extends scala.Product> Dataset<Row> createDataFrame(scala.collection.immutable.Seq<A> data, scala.reflect.api.TypeTags.TypeTag<A> evidence$3)
      Creates a DataFrame from a local Seq of Product.

      Parameters:
      data - (undocumented)
      evidence$3 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public Dataset<Row> createDataFrame(RDD<Row> rowRDD, StructType 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)
       

      Parameters:
      rowRDD - (undocumented)
      schema - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public Dataset<Row> createDataFrame(JavaRDD<Row> rowRDD, StructType 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.

      Parameters:
      rowRDD - (undocumented)
      schema - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public Dataset<Row> createDataFrame(List<Row> rows, StructType 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.

      Parameters:
      rows - (undocumented)
      schema - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public Dataset<Row> createDataFrame(RDD<?> rdd, Class<?> beanClass)
      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.

      Parameters:
      rdd - (undocumented)
      beanClass - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public Dataset<Row> createDataFrame(JavaRDD<?> rdd, Class<?> beanClass)
      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.

      Parameters:
      rdd - (undocumented)
      beanClass - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataFrame

      public Dataset<Row> createDataFrame(List<?> data, Class<?> beanClass)
      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.

      Parameters:
      data - (undocumented)
      beanClass - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • baseRelationToDataFrame

      public Dataset<Row> baseRelationToDataFrame(BaseRelation baseRelation)
      Convert a BaseRelation created for external data sources into a DataFrame.

      Parameters:
      baseRelation - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataset

      public <T> Dataset<T> createDataset(scala.collection.immutable.Seq<T> data, Encoder<T> evidence$4)
      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|
         // +-------+---+
       

      Parameters:
      data - (undocumented)
      evidence$4 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataset

      public <T> Dataset<T> createDataset(RDD<T> data, Encoder<T> evidence$5)
      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.

      Parameters:
      data - (undocumented)
      evidence$5 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • createDataset

      public <T> Dataset<T> createDataset(List<T> data, Encoder<T> evidence$6)
      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());
       

      Parameters:
      data - (undocumented)
      evidence$6 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • range

      public Dataset<Long> range(long end)
      Creates a Dataset with a single LongType column named id, containing elements in a range from 0 to end (exclusive) with step value 1.

      Parameters:
      end - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • range

      public Dataset<Long> range(long start, long end)
      Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with step value 1.

      Parameters:
      start - (undocumented)
      end - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • range

      public Dataset<Long> range(long start, long end, long step)
      Creates a Dataset with a single LongType column named id, containing elements in a range from start to end (exclusive) with a step value.

      Parameters:
      start - (undocumented)
      end - (undocumented)
      step - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • range

      public Dataset<Long> range(long start, long end, long step, int numPartitions)
      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.

      Parameters:
      start - (undocumented)
      end - (undocumented)
      step - (undocumented)
      numPartitions - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • catalog

      public Catalog catalog()
    • 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:
      2.0.0
    • sql

      public Dataset<Row> sql(String sqlText, Object args)
      Executes a SQL query substituting positional parameters by the given arguments, returning the result as a DataFrame. This API eagerly runs DDL/DML commands, but not for SELECT queries.

      Parameters:
      sqlText - A SQL statement with positional parameters to execute.
      args - An array of Java/Scala objects that can be converted to SQL literal expressions. See Supported Data Types for supported value types in Scala/Java. For example, 1, "Steven", LocalDate.of(2023, 4, 2). A value can be also a Column of a literal or collection constructor functions such as map(), array(), struct(), in that case it is taken as is.

      Returns:
      (undocumented)
      Since:
      3.5.0
    • sql

      public Dataset<Row> sql(String sqlText, scala.collection.immutable.Map<String,Object> args)
      Executes a SQL query substituting named parameters by the given arguments, returning the result as a DataFrame. This API eagerly runs DDL/DML commands, but not for SELECT queries.

      Parameters:
      sqlText - A SQL statement with named parameters to execute.
      args - A map of parameter names to Java/Scala objects that can be converted to SQL literal expressions. See Supported Data Types for supported value types in Scala/Java. For example, map keys: "rank", "name", "birthdate"; map values: 1, "Steven", LocalDate.of(2023, 4, 2). Map value can be also a Column of a literal or collection constructor functions such as map(), array(), struct(), in that case it is taken as is.

      Returns:
      (undocumented)
      Since:
      3.4.0
    • sql

      public Dataset<Row> sql(String sqlText, Map<String,Object> args)
      Executes a SQL query substituting named parameters by the given arguments, returning the result as a DataFrame. This API eagerly runs DDL/DML commands, but not for SELECT queries.

      Parameters:
      sqlText - A SQL statement with named parameters to execute.
      args - A map of parameter names to Java/Scala objects that can be converted to SQL literal expressions. See Supported Data Types for supported value types in Scala/Java. For example, map keys: "rank", "name", "birthdate"; map values: 1, "Steven", LocalDate.of(2023, 4, 2). Map value can be also a Column of a literal or collection constructor functions such as map(), array(), struct(), in that case it is taken as is.

      Returns:
      (undocumented)
      Since:
      3.4.0
    • sql

      public Dataset<Row> sql(String sqlText)
      Executes a SQL query using Spark, returning the result as a DataFrame. This API eagerly runs DDL/DML commands, but not for SELECT queries.

      Parameters:
      sqlText - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • executeCommand

      public Dataset<Row> executeCommand(String runner, String command, scala.collection.immutable.Map<String,String> options)
      Execute an arbitrary string command inside an external execution engine rather than Spark. This could be useful when user wants to execute some commands out of Spark. For example, executing custom DDL/DML command for JDBC, creating index for ElasticSearch, creating cores for Solr and so on.

      The command will be eagerly executed after this method is called and the returned DataFrame will contain the output of the command(if any).

      Parameters:
      runner - The class name of the runner that implements ExternalCommandRunner.
      command - The target command to be executed
      options - The options for the runner.

      Returns:
      (undocumented)
      Since:
      3.0.0
    • read

      public DataFrameReader read()
      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")
       

      Returns:
      (undocumented)
      Since:
      2.0.0
    • readStream

      public DataStreamReader readStream()
      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")
       

      Returns:
      (undocumented)
      Since:
      2.0.0
    • time

      public <T> T time(scala.Function0<T> f)
      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.

      Parameters:
      f - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.1.0
    • stop

      public void stop()
      Stop the underlying SparkContext.

      Since:
      2.0.0
    • close

      public void close()
      Synonym for stop().

      Specified by:
      close in interface AutoCloseable
      Specified by:
      close in interface Closeable
      Since:
      2.1.0