/
SparkSession.scala
1206 lines (1092 loc) · 40.8 KB
/
SparkSession.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.sql
import java.io.Closeable
import java.util.UUID
import java.util.concurrent.TimeUnit._
import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference}
import scala.collection.JavaConverters._
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal
import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext}
import org.apache.spark.annotation.{DeveloperApi, Experimental, Stable, Unstable}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.internal.Logging
import org.apache.spark.internal.config.{ConfigEntry, EXECUTOR_ALLOW_SPARK_CONTEXT}
import org.apache.spark.rdd.RDD
import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
import org.apache.spark.sql.catalog.Catalog
import org.apache.spark.sql.catalyst._
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders._
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Range}
import org.apache.spark.sql.catalyst.util.CharVarcharUtils
import org.apache.spark.sql.connector.ExternalCommandRunner
import org.apache.spark.sql.execution._
import org.apache.spark.sql.execution.command.ExternalCommandExecutor
import org.apache.spark.sql.execution.datasources.{DataSource, LogicalRelation}
import org.apache.spark.sql.internal._
import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.sql.streaming._
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.sql.util.ExecutionListenerManager
import org.apache.spark.util.{CallSite, Utils}
/**
* 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.
*/
@Stable
class SparkSession private(
@transient val sparkContext: SparkContext,
@transient private val existingSharedState: Option[SharedState],
@transient private val parentSessionState: Option[SessionState],
@transient private[sql] val extensions: SparkSessionExtensions,
@transient private[sql] val initialSessionOptions: Map[String, String])
extends Serializable with Closeable with Logging { self =>
// The call site where this SparkSession was constructed.
private val creationSite: CallSite = Utils.getCallSite()
/**
* Constructor used in Pyspark. Contains explicit application of Spark Session Extensions
* which otherwise only occurs during getOrCreate. We cannot add this to the default constructor
* since that would cause every new session to reinvoke Spark Session Extensions on the currently
* running extensions.
*/
private[sql] def this(sc: SparkContext) = {
this(sc, None, None,
SparkSession.applyExtensions(
sc.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty),
new SparkSessionExtensions), Map.empty)
}
private[sql] val sessionUUID: String = UUID.randomUUID.toString
sparkContext.assertNotStopped()
// If there is no active SparkSession, uses the default SQL conf. Otherwise, use the session's.
SQLConf.setSQLConfGetter(() => {
SparkSession.getActiveSession.filterNot(_.sparkContext.isStopped).map(_.sessionState.conf)
.getOrElse(SQLConf.getFallbackConf)
})
/**
* The version of Spark on which this application is running.
*
* @since 2.0.0
*/
def version: String = SPARK_VERSION
/* ----------------------- *
| Session-related state |
* ----------------------- */
/**
* 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.
*
* @since 2.2.0
*/
@Unstable
@transient
lazy val sharedState: SharedState = {
existingSharedState.getOrElse(new SharedState(sparkContext, initialSessionOptions))
}
/**
* 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.
*
* @since 2.2.0
*/
@Unstable
@transient
lazy val sessionState: SessionState = {
parentSessionState
.map(_.clone(this))
.getOrElse {
val state = SparkSession.instantiateSessionState(
SparkSession.sessionStateClassName(sparkContext.conf),
self)
state
}
}
/**
* A wrapped version of this session in the form of a [[SQLContext]], for backward compatibility.
*
* @since 2.0.0
*/
@transient
val sqlContext: SQLContext = new SQLContext(this)
/**
* 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.
*
* @since 2.0.0
*/
@transient lazy val conf: RuntimeConfig = new RuntimeConfig(sessionState.conf)
/**
* An interface to register custom [[org.apache.spark.sql.util.QueryExecutionListener]]s
* that listen for execution metrics.
*
* @since 2.0.0
*/
def listenerManager: ExecutionListenerManager = sessionState.listenerManager
/**
* :: Experimental ::
* A collection of methods that are considered experimental, but can be used to hook into
* the query planner for advanced functionality.
*
* @since 2.0.0
*/
@Experimental
@Unstable
def experimental: ExperimentalMethods = sessionState.experimentalMethods
/**
* 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);
* }}}
*
* @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.
*
* @since 2.0.0
*/
def udf: UDFRegistration = sessionState.udfRegistration
/**
* Returns a `StreamingQueryManager` that allows managing all the
* `StreamingQuery`s active on `this`.
*
* @since 2.0.0
*/
@Unstable
def streams: StreamingQueryManager = sessionState.streamingQueryManager
/**
* Start a new session with isolated SQL configurations, temporary tables, registered
* functions are isolated, but sharing the underlying `SparkContext` and cached data.
*
* @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.
*
* @since 2.0.0
*/
def newSession(): SparkSession = {
new SparkSession(
sparkContext,
Some(sharedState),
parentSessionState = None,
extensions,
initialSessionOptions)
}
/**
* Create an identical copy of this `SparkSession`, sharing the underlying `SparkContext`
* and shared state. All the state of this session (i.e. SQL configurations, temporary tables,
* registered functions) is copied over, and the cloned session is set up with the same shared
* state as this session. The cloned session is independent of this session, that is, any
* non-global change in either session is not reflected in the other.
*
* @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.
*/
private[sql] def cloneSession(): SparkSession = {
val result = new SparkSession(
sparkContext,
Some(sharedState),
Some(sessionState),
extensions,
Map.empty)
result.sessionState // force copy of SessionState
result
}
/* --------------------------------- *
| Methods for creating DataFrames |
* --------------------------------- */
/**
* Returns a `DataFrame` with no rows or columns.
*
* @since 2.0.0
*/
@transient
lazy val emptyDataFrame: DataFrame = Dataset.ofRows(self, LocalRelation())
/**
* Creates a new [[Dataset]] of type T containing zero elements.
*
* @return 2.0.0
*/
def emptyDataset[T: Encoder]: Dataset[T] = {
val encoder = implicitly[Encoder[T]]
new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder)
}
/**
* Creates a `DataFrame` from an RDD of Product (e.g. case classes, tuples).
*
* @since 2.0.0
*/
def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = withActive {
val encoder = Encoders.product[A]
Dataset.ofRows(self, ExternalRDD(rdd, self)(encoder))
}
/**
* Creates a `DataFrame` from a local Seq of Product.
*
* @since 2.0.0
*/
def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = withActive {
val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType]
val attributeSeq = schema.toAttributes
Dataset.ofRows(self, LocalRelation.fromProduct(attributeSeq, data))
}
/**
* :: DeveloperApi ::
* Creates a `DataFrame` from an `RDD` containing [[Row]]s 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)
* }}}
*
* @since 2.0.0
*/
@DeveloperApi
def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = withActive {
val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType]
// TODO: use MutableProjection when rowRDD is another DataFrame and the applied
// schema differs from the existing schema on any field data type.
val encoder = RowEncoder(replaced)
val toRow = encoder.createSerializer()
val catalystRows = rowRDD.map(toRow)
internalCreateDataFrame(catalystRows.setName(rowRDD.name), schema)
}
/**
* :: DeveloperApi ::
* Creates a `DataFrame` from a `JavaRDD` containing [[Row]]s 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.
*
* @since 2.0.0
*/
@DeveloperApi
def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = {
val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType]
createDataFrame(rowRDD.rdd, replaced)
}
/**
* :: DeveloperApi ::
* Creates a `DataFrame` from a `java.util.List` containing [[Row]]s 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.
*
* @since 2.0.0
*/
@DeveloperApi
def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = withActive {
val replaced = CharVarcharUtils.failIfHasCharVarchar(schema).asInstanceOf[StructType]
Dataset.ofRows(self, LocalRelation.fromExternalRows(replaced.toAttributes, rows.asScala.toSeq))
}
/**
* 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
*/
def createDataFrame(rdd: RDD[_], beanClass: Class[_]): DataFrame = withActive {
val attributeSeq: Seq[AttributeReference] = getSchema(beanClass)
val className = beanClass.getName
val rowRdd = rdd.mapPartitions { iter =>
// BeanInfo is not serializable so we must rediscover it remotely for each partition.
SQLContext.beansToRows(iter, Utils.classForName(className), attributeSeq)
}
Dataset.ofRows(self, LogicalRDD(attributeSeq, rowRdd.setName(rdd.name))(self))
}
/**
* 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
*/
def createDataFrame(rdd: JavaRDD[_], beanClass: Class[_]): DataFrame = {
createDataFrame(rdd.rdd, 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.
* @since 1.6.0
*/
def createDataFrame(data: java.util.List[_], beanClass: Class[_]): DataFrame = withActive {
val attrSeq = getSchema(beanClass)
val rows = SQLContext.beansToRows(data.asScala.iterator, beanClass, attrSeq)
Dataset.ofRows(self, LocalRelation(attrSeq, rows.toSeq))
}
/**
* Convert a `BaseRelation` created for external data sources into a `DataFrame`.
*
* @since 2.0.0
*/
def baseRelationToDataFrame(baseRelation: BaseRelation): DataFrame = {
Dataset.ofRows(self, LogicalRelation(baseRelation))
}
/* ------------------------------- *
| Methods for creating DataSets |
* ------------------------------- */
/**
* 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|
* // +-------+---+
* }}}
*
* @since 2.0.0
*/
def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = {
val enc = encoderFor[T]
val toRow = enc.createSerializer()
val attributes = enc.schema.toAttributes
val encoded = data.map(d => toRow(d).copy())
val plan = new LocalRelation(attributes, encoded)
Dataset[T](self, plan)
}
/**
* 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]].
*
* @since 2.0.0
*/
def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = {
Dataset[T](self, ExternalRDD(data, self))
}
/**
* 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());
* }}}
*
* @since 2.0.0
*/
def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = {
createDataset(data.asScala.toSeq)
}
/**
* Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements
* in a range from 0 to `end` (exclusive) with step value 1.
*
* @since 2.0.0
*/
def range(end: Long): Dataset[java.lang.Long] = range(0, 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.
*
* @since 2.0.0
*/
def range(start: Long, end: Long): Dataset[java.lang.Long] = {
range(start, end, step = 1, numPartitions = leafNodeDefaultParallelism)
}
/**
* Creates a [[Dataset]] with a single `LongType` column named `id`, containing elements
* in a range from `start` to `end` (exclusive) with a step value.
*
* @since 2.0.0
*/
def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = {
range(start, end, step, numPartitions = leafNodeDefaultParallelism)
}
/**
* 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.
*
* @since 2.0.0
*/
def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = {
new Dataset(self, Range(start, end, step, numPartitions), Encoders.LONG)
}
/**
* Creates a `DataFrame` from an `RDD[InternalRow]`.
*/
private[sql] def internalCreateDataFrame(
catalystRows: RDD[InternalRow],
schema: StructType,
isStreaming: Boolean = false): DataFrame = {
// TODO: use MutableProjection when rowRDD is another DataFrame and the applied
// schema differs from the existing schema on any field data type.
val logicalPlan = LogicalRDD(
schema.toAttributes,
catalystRows,
isStreaming = isStreaming)(self)
Dataset.ofRows(self, logicalPlan)
}
/* ------------------------- *
| Catalog-related methods |
* ------------------------- */
/**
* Interface through which the user may create, drop, alter or query underlying
* databases, tables, functions etc.
*
* @since 2.0.0
*/
@transient lazy val catalog: Catalog = new CatalogImpl(self)
/**
* 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.
*
* @param 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
*/
def table(tableName: String): DataFrame = {
read.table(tableName)
}
private[sql] def table(tableIdent: TableIdentifier): DataFrame = {
Dataset.ofRows(self, UnresolvedRelation(tableIdent))
}
/* ----------------- *
| Everything else |
* ----------------- */
/**
* Executes a SQL query using Spark, returning the result as a `DataFrame`.
* This API eagerly runs DDL/DML commands, but not for SELECT queries.
*
* @since 2.0.0
*/
def sql(sqlText: String): DataFrame = withActive {
val tracker = new QueryPlanningTracker
val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) {
sessionState.sqlParser.parsePlan(sqlText)
}
Dataset.ofRows(self, plan, tracker)
}
/**
* 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).
*
* @param runner The class name of the runner that implements `ExternalCommandRunner`.
* @param command The target command to be executed
* @param options The options for the runner.
*
* @since 3.0.0
*/
@Unstable
def executeCommand(runner: String, command: String, options: Map[String, String]): DataFrame = {
DataSource.lookupDataSource(runner, sessionState.conf) match {
case source if classOf[ExternalCommandRunner].isAssignableFrom(source) =>
Dataset.ofRows(self, ExternalCommandExecutor(
source.newInstance().asInstanceOf[ExternalCommandRunner], command, options))
case _ =>
throw new AnalysisException(s"Command execution is not supported in runner $runner")
}
}
/**
* 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
*/
def read: DataFrameReader = new DataFrameReader(self)
/**
* 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")
* }}}
*
* @since 2.0.0
*/
def readStream: DataStreamReader = new DataStreamReader(self)
/**
* 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
*/
def time[T](f: => T): T = {
val start = System.nanoTime()
val ret = f
val end = System.nanoTime()
// scalastyle:off println
println(s"Time taken: ${NANOSECONDS.toMillis(end - start)} ms")
// scalastyle:on println
ret
}
// scalastyle:off
// Disable style checker so "implicits" object can start with lowercase i
/**
* (Scala-specific) Implicit methods available in Scala for converting
* common Scala objects into `DataFrame`s.
*
* {{{
* val sparkSession = SparkSession.builder.getOrCreate()
* import sparkSession.implicits._
* }}}
*
* @since 2.0.0
*/
object implicits extends SQLImplicits with Serializable {
protected override def _sqlContext: SQLContext = SparkSession.this.sqlContext
}
// scalastyle:on
/**
* Stop the underlying `SparkContext`.
*
* @since 2.0.0
*/
def stop(): Unit = {
sparkContext.stop()
}
/**
* Synonym for `stop()`.
*
* @since 2.1.0
*/
override def close(): Unit = stop()
/**
* 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.
*/
protected[sql] def parseDataType(dataTypeString: String): DataType = {
DataType.fromJson(dataTypeString)
}
/**
* Apply a schema defined by the schemaString to an RDD. It is only used by PySpark.
*/
private[sql] def applySchemaToPythonRDD(
rdd: RDD[Array[Any]],
schemaString: String): DataFrame = {
val schema = DataType.fromJson(schemaString).asInstanceOf[StructType]
applySchemaToPythonRDD(rdd, schema)
}
/**
* Apply `schema` to an RDD.
*
* @note Used by PySpark only
*/
private[sql] def applySchemaToPythonRDD(
rdd: RDD[Array[Any]],
schema: StructType): DataFrame = {
val rowRdd = rdd.mapPartitions { iter =>
val fromJava = python.EvaluatePython.makeFromJava(schema)
iter.map(r => fromJava(r).asInstanceOf[InternalRow])
}
internalCreateDataFrame(rowRdd, schema)
}
/**
* Returns a Catalyst Schema for the given java bean class.
*/
private def getSchema(beanClass: Class[_]): Seq[AttributeReference] = {
val (dataType, _) = JavaTypeInference.inferDataType(beanClass)
dataType.asInstanceOf[StructType].fields.map { f =>
AttributeReference(f.name, f.dataType, f.nullable)()
}
}
/**
* Execute a block of code with the this session set as the active session, and restore the
* previous session on completion.
*/
private[sql] def withActive[T](block: => T): T = {
// Use the active session thread local directly to make sure we get the session that is actually
// set and not the default session. This to prevent that we promote the default session to the
// active session once we are done.
val old = SparkSession.activeThreadSession.get()
SparkSession.setActiveSession(this)
try block finally {
SparkSession.setActiveSession(old)
}
}
private[sql] def leafNodeDefaultParallelism: Int = {
conf.get(SQLConf.LEAF_NODE_DEFAULT_PARALLELISM).getOrElse(sparkContext.defaultParallelism)
}
}
@Stable
object SparkSession extends Logging {
/**
* Builder for [[SparkSession]].
*/
@Stable
class Builder extends Logging {
private[this] val options = new scala.collection.mutable.HashMap[String, String]
private[this] val extensions = new SparkSessionExtensions
private[this] var userSuppliedContext: Option[SparkContext] = None
private[spark] def sparkContext(sparkContext: SparkContext): Builder = synchronized {
userSuppliedContext = Option(sparkContext)
this
}
/**
* Sets a name for the application, which will be shown in the Spark web UI.
* If no application name is set, a randomly generated name will be used.
*
* @since 2.0.0
*/
def appName(name: String): Builder = config("spark.app.name", name)
/**
* Sets a config option. Options set using this method are automatically propagated to
* both `SparkConf` and SparkSession's own configuration.
*
* @since 2.0.0
*/
def config(key: String, value: String): Builder = synchronized {
options += key -> value
this
}
/**
* Sets a config option. Options set using this method are automatically propagated to
* both `SparkConf` and SparkSession's own configuration.
*
* @since 2.0.0
*/
def config(key: String, value: Long): Builder = synchronized {
options += key -> value.toString
this
}
/**
* Sets a config option. Options set using this method are automatically propagated to
* both `SparkConf` and SparkSession's own configuration.
*
* @since 2.0.0
*/
def config(key: String, value: Double): Builder = synchronized {
options += key -> value.toString
this
}
/**
* Sets a config option. Options set using this method are automatically propagated to
* both `SparkConf` and SparkSession's own configuration.
*
* @since 2.0.0
*/
def config(key: String, value: Boolean): Builder = synchronized {
options += key -> value.toString
this
}
/**
* Sets a list of config options based on the given `SparkConf`.
*
* @since 2.0.0
*/
def config(conf: SparkConf): Builder = synchronized {
conf.getAll.foreach { case (k, v) => options += k -> v }
this
}
/**
* Sets the Spark master URL to connect to, such as "local" to run locally, "local[4]" to
* run locally with 4 cores, or "spark://master:7077" to run on a Spark standalone cluster.
*
* @since 2.0.0
*/
def master(master: String): Builder = config("spark.master", master)
/**
* Enables Hive support, including connectivity to a persistent Hive metastore, support for
* Hive serdes, and Hive user-defined functions.
*
* @since 2.0.0
*/
def enableHiveSupport(): Builder = synchronized {
if (hiveClassesArePresent) {
config(CATALOG_IMPLEMENTATION.key, "hive")
} else {
throw new IllegalArgumentException(
"Unable to instantiate SparkSession with Hive support because " +
"Hive classes are not found.")
}
}
/**
* Inject extensions into the [[SparkSession]]. This allows a user to add Analyzer rules,
* Optimizer rules, Planning Strategies or a customized parser.
*
* @since 2.2.0
*/
def withExtensions(f: SparkSessionExtensions => Unit): Builder = synchronized {
f(extensions)
this
}
/**
* Gets an existing [[SparkSession]] or, if there is no existing one, creates a new
* one based on the options set in this builder.
*
* This method first checks whether there is a valid thread-local SparkSession,
* and if yes, return that one. It then checks whether there is a valid global
* default SparkSession, and if yes, return that one. If no valid global default
* SparkSession exists, the method creates a new SparkSession and assigns the
* newly created SparkSession as the global default.
*
* In case an existing SparkSession is returned, the non-static config options specified in
* this builder will be applied to the existing SparkSession.
*
* @since 2.0.0
*/
def getOrCreate(): SparkSession = synchronized {
val sparkConf = new SparkConf()
options.foreach { case (k, v) => sparkConf.set(k, v) }
if (!sparkConf.get(EXECUTOR_ALLOW_SPARK_CONTEXT)) {
assertOnDriver()
}
// Get the session from current thread's active session.
var session = activeThreadSession.get()
if ((session ne null) && !session.sparkContext.isStopped) {
applyModifiableSettings(session)
return session
}
// Global synchronization so we will only set the default session once.
SparkSession.synchronized {
// If the current thread does not have an active session, get it from the global session.
session = defaultSession.get()
if ((session ne null) && !session.sparkContext.isStopped) {
applyModifiableSettings(session)
return session
}
// No active nor global default session. Create a new one.
val sparkContext = userSuppliedContext.getOrElse {
// set a random app name if not given.
if (!sparkConf.contains("spark.app.name")) {
sparkConf.setAppName(java.util.UUID.randomUUID().toString)
}
SparkContext.getOrCreate(sparkConf)
// Do not update `SparkConf` for existing `SparkContext`, as it's shared by all sessions.
}
applyExtensions(
sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty),
extensions)
session = new SparkSession(sparkContext, None, None, extensions, options.toMap)
setDefaultSession(session)
setActiveSession(session)
registerContextListener(sparkContext)
}
return session
}
private def applyModifiableSettings(session: SparkSession): Unit = {
val (staticConfs, otherConfs) =
options.partition(kv => SQLConf.staticConfKeys.contains(kv._1))
otherConfs.foreach { case (k, v) => session.sessionState.conf.setConfString(k, v) }
if (staticConfs.nonEmpty) {
logWarning("Using an existing SparkSession; the static sql configurations will not take" +
" effect.")
}
if (otherConfs.nonEmpty) {
logWarning("Using an existing SparkSession; some spark core configurations may not take" +
" effect.")
}
}
}
/**
* Creates a [[SparkSession.Builder]] for constructing a [[SparkSession]].
*
* @since 2.0.0
*/
def builder(): Builder = new Builder
/**
* 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.
*
* @since 2.0.0
*/
def setActiveSession(session: SparkSession): Unit = {
activeThreadSession.set(session)
}
/**