-
Notifications
You must be signed in to change notification settings - Fork 28k
/
DataFrame.scala
1855 lines (1691 loc) · 62.3 KB
/
DataFrame.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.CharArrayWriter
import java.util.Properties
import scala.language.implicitConversions
import scala.reflect.ClassTag
import scala.reflect.runtime.universe.TypeTag
import scala.util.control.NonFatal
import com.fasterxml.jackson.core.JsonFactory
import org.apache.commons.lang3.StringUtils
import org.apache.spark.annotation.{DeveloperApi, Experimental}
import org.apache.spark.api.java.JavaRDD
import org.apache.spark.api.python.SerDeUtil
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{Filter, _}
import org.apache.spark.sql.catalyst.plans.{Inner, JoinType}
import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser}
import org.apache.spark.sql.execution.{EvaluatePython, ExplainCommand, LogicalRDD}
import org.apache.spark.sql.json.JacksonGenerator
import org.apache.spark.sql.sources.CreateTableUsingAsSelect
import org.apache.spark.sql.types._
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils
private[sql] object DataFrame {
def apply(sqlContext: SQLContext, logicalPlan: LogicalPlan): DataFrame = {
new DataFrame(sqlContext, logicalPlan)
}
}
/**
* :: Experimental ::
* A distributed collection of data organized into named columns.
*
* A [[DataFrame]] is equivalent to a relational table in Spark SQL. The following example creates
* a [[DataFrame]] by pointing Spark SQL to a Parquet data set.
* {{{
* val people = sqlContext.read.parquet("...") // in Scala
* DataFrame people = sqlContext.read().parquet("...") // in Java
* }}}
*
* Once created, it can be manipulated using the various domain-specific-language (DSL) functions
* defined in: [[DataFrame]] (this class), [[Column]], and [[functions]].
*
* To select a column from the data frame, use `apply` method in Scala and `col` in Java.
* {{{
* val ageCol = people("age") // in Scala
* Column ageCol = people.col("age") // in Java
* }}}
*
* Note that the [[Column]] type can also be manipulated through its various functions.
* {{{
* // The following creates a new column that increases everybody's age by 10.
* people("age") + 10 // in Scala
* people.col("age").plus(10); // in Java
* }}}
*
* A more concrete example in Scala:
* {{{
* // To create DataFrame using SQLContext
* val people = sqlContext.read.parquet("...")
* val department = sqlContext.read.parquet("...")
*
* people.filter("age > 30")
* .join(department, people("deptId") === department("id"))
* .groupBy(department("name"), "gender")
* .agg(avg(people("salary")), max(people("age")))
* }}}
*
* and in Java:
* {{{
* // To create DataFrame using SQLContext
* DataFrame people = sqlContext.read().parquet("...");
* DataFrame department = sqlContext.read().parquet("...");
*
* people.filter("age".gt(30))
* .join(department, people.col("deptId").equalTo(department("id")))
* .groupBy(department.col("name"), "gender")
* .agg(avg(people.col("salary")), max(people.col("age")));
* }}}
*
* @groupname basic Basic DataFrame functions
* @groupname dfops Language Integrated Queries
* @groupname rdd RDD Operations
* @groupname output Output Operations
* @groupname action Actions
* @since 1.3.0
*/
// TODO: Improve documentation.
@Experimental
class DataFrame private[sql](
@transient val sqlContext: SQLContext,
@DeveloperApi @transient val queryExecution: SQLContext#QueryExecution) extends Serializable {
/**
* A constructor that automatically analyzes the logical plan.
*
* This reports error eagerly as the [[DataFrame]] is constructed, unless
* [[SQLConf.dataFrameEagerAnalysis]] is turned off.
*/
def this(sqlContext: SQLContext, logicalPlan: LogicalPlan) = {
this(sqlContext, {
val qe = sqlContext.executePlan(logicalPlan)
if (sqlContext.conf.dataFrameEagerAnalysis) {
qe.assertAnalyzed() // This should force analysis and throw errors if there are any
}
qe
})
}
@transient protected[sql] val logicalPlan: LogicalPlan = queryExecution.logical match {
// For various commands (like DDL) and queries with side effects, we force query optimization to
// happen right away to let these side effects take place eagerly.
case _: Command |
_: InsertIntoTable |
_: CreateTableUsingAsSelect |
_: WriteToFile =>
LogicalRDD(queryExecution.analyzed.output, queryExecution.toRdd)(sqlContext)
case _ =>
queryExecution.analyzed
}
/**
* An implicit conversion function internal to this class for us to avoid doing
* "new DataFrame(...)" everywhere.
*/
@inline private implicit def logicalPlanToDataFrame(logicalPlan: LogicalPlan): DataFrame = {
new DataFrame(sqlContext, logicalPlan)
}
protected[sql] def resolve(colName: String): NamedExpression = {
queryExecution.analyzed.resolveQuoted(colName, sqlContext.analyzer.resolver).getOrElse {
throw new AnalysisException(
s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""")
}
}
protected[sql] def numericColumns: Seq[Expression] = {
schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n =>
queryExecution.analyzed.resolveQuoted(n.name, sqlContext.analyzer.resolver).get
}
}
/**
* Internal API for Python
* @param _numRows Number of rows to show
* @param truncate Whether truncate long strings and align cells right
*/
private[sql] def showString(_numRows: Int, truncate: Boolean = true): String = {
val numRows = _numRows.max(0)
val sb = new StringBuilder
val takeResult = take(numRows + 1)
val hasMoreData = takeResult.length > numRows
val data = takeResult.take(numRows)
val numCols = schema.fieldNames.length
// For array values, replace Seq and Array with square brackets
// For cells that are beyond 20 characters, replace it with the first 17 and "..."
val rows: Seq[Seq[String]] = schema.fieldNames.toSeq +: data.map { row =>
row.toSeq.map { cell =>
val str = cell match {
case null => "null"
case array: Array[_] => array.mkString("[", ", ", "]")
case seq: Seq[_] => seq.mkString("[", ", ", "]")
case _ => cell.toString
}
if (truncate && str.length > 20) str.substring(0, 17) + "..." else str
}: Seq[String]
}
// Initialise the width of each column to a minimum value of '3'
val colWidths = Array.fill(numCols)(3)
// Compute the width of each column
for (row <- rows) {
for ((cell, i) <- row.zipWithIndex) {
colWidths(i) = math.max(colWidths(i), cell.length)
}
}
// Create SeparateLine
val sep: String = colWidths.map("-" * _).addString(sb, "+", "+", "+\n").toString()
// column names
rows.head.zipWithIndex.map { case (cell, i) =>
if (truncate) {
StringUtils.leftPad(cell, colWidths(i))
} else {
StringUtils.rightPad(cell, colWidths(i))
}
}.addString(sb, "|", "|", "|\n")
sb.append(sep)
// data
rows.tail.map {
_.zipWithIndex.map { case (cell, i) =>
if (truncate) {
StringUtils.leftPad(cell.toString, colWidths(i))
} else {
StringUtils.rightPad(cell.toString, colWidths(i))
}
}.addString(sb, "|", "|", "|\n")
}
sb.append(sep)
// For Data that has more than "numRows" records
if (hasMoreData) {
val rowsString = if (numRows == 1) "row" else "rows"
sb.append(s"only showing top $numRows ${rowsString}\n")
}
sb.toString()
}
override def toString: String = {
try {
schema.map(f => s"${f.name}: ${f.dataType.simpleString}").mkString("[", ", ", "]")
} catch {
case NonFatal(e) =>
s"Invalid tree; ${e.getMessage}:\n$queryExecution"
}
}
/**
* Returns the object itself.
* @group basic
* @since 1.3.0
*/
// This is declared with parentheses to prevent the Scala compiler from treating
// `rdd.toDF("1")` as invoking this toDF and then apply on the returned DataFrame.
def toDF(): DataFrame = this
/**
* Returns a new [[DataFrame]] with columns renamed. This can be quite convenient in conversion
* from a RDD of tuples into a [[DataFrame]] with meaningful names. For example:
* {{{
* val rdd: RDD[(Int, String)] = ...
* rdd.toDF() // this implicit conversion creates a DataFrame with column name _1 and _2
* rdd.toDF("id", "name") // this creates a DataFrame with column name "id" and "name"
* }}}
* @group basic
* @since 1.3.0
*/
@scala.annotation.varargs
def toDF(colNames: String*): DataFrame = {
require(schema.size == colNames.size,
"The number of columns doesn't match.\n" +
s"Old column names (${schema.size}): " + schema.fields.map(_.name).mkString(", ") + "\n" +
s"New column names (${colNames.size}): " + colNames.mkString(", "))
val newCols = logicalPlan.output.zip(colNames).map { case (oldAttribute, newName) =>
Column(oldAttribute).as(newName)
}
select(newCols : _*)
}
/**
* Returns the schema of this [[DataFrame]].
* @group basic
* @since 1.3.0
*/
def schema: StructType = queryExecution.analyzed.schema
/**
* Returns all column names and their data types as an array.
* @group basic
* @since 1.3.0
*/
def dtypes: Array[(String, String)] = schema.fields.map { field =>
(field.name, field.dataType.toString)
}
/**
* Returns all column names as an array.
* @group basic
* @since 1.3.0
*/
def columns: Array[String] = schema.fields.map(_.name)
/**
* Prints the schema to the console in a nice tree format.
* @group basic
* @since 1.3.0
*/
def printSchema(): Unit = println(schema.treeString)
/**
* Prints the plans (logical and physical) to the console for debugging purposes.
* @group basic
* @since 1.3.0
*/
def explain(extended: Boolean): Unit = {
ExplainCommand(
queryExecution.logical,
extended = extended).queryExecution.executedPlan.executeCollect().map {
r => println(r.getString(0))
}
}
/**
* Only prints the physical plan to the console for debugging purposes.
* @group basic
* @since 1.3.0
*/
def explain(): Unit = explain(extended = false)
/**
* Returns true if the `collect` and `take` methods can be run locally
* (without any Spark executors).
* @group basic
* @since 1.3.0
*/
def isLocal: Boolean = logicalPlan.isInstanceOf[LocalRelation]
/**
* Displays the [[DataFrame]] in a tabular form. Strings more than 20 characters will be
* truncated, and all cells will be aligned right. For example:
* {{{
* year month AVG('Adj Close) MAX('Adj Close)
* 1980 12 0.503218 0.595103
* 1981 01 0.523289 0.570307
* 1982 02 0.436504 0.475256
* 1983 03 0.410516 0.442194
* 1984 04 0.450090 0.483521
* }}}
* @param numRows Number of rows to show
*
* @group action
* @since 1.3.0
*/
def show(numRows: Int): Unit = show(numRows, true)
/**
* Displays the top 20 rows of [[DataFrame]] in a tabular form. Strings more than 20 characters
* will be truncated, and all cells will be aligned right.
* @group action
* @since 1.3.0
*/
def show(): Unit = show(20)
/**
* Displays the top 20 rows of [[DataFrame]] in a tabular form.
*
* @param truncate Whether truncate long strings. If true, strings more than 20 characters will
* be truncated and all cells will be aligned right
*
* @group action
* @since 1.5.0
*/
def show(truncate: Boolean): Unit = show(20, truncate)
/**
* Displays the [[DataFrame]] in a tabular form. For example:
* {{{
* year month AVG('Adj Close) MAX('Adj Close)
* 1980 12 0.503218 0.595103
* 1981 01 0.523289 0.570307
* 1982 02 0.436504 0.475256
* 1983 03 0.410516 0.442194
* 1984 04 0.450090 0.483521
* }}}
* @param numRows Number of rows to show
* @param truncate Whether truncate long strings. If true, strings more than 20 characters will
* be truncated and all cells will be aligned right
*
* @group action
* @since 1.5.0
*/
def show(numRows: Int, truncate: Boolean): Unit = println(showString(numRows, truncate))
/**
* Returns a [[DataFrameNaFunctions]] for working with missing data.
* {{{
* // Dropping rows containing any null values.
* df.na.drop()
* }}}
*
* @group dfops
* @since 1.3.1
*/
def na: DataFrameNaFunctions = new DataFrameNaFunctions(this)
/**
* Returns a [[DataFrameStatFunctions]] for working statistic functions support.
* {{{
* // Finding frequent items in column with name 'a'.
* df.stat.freqItems(Seq("a"))
* }}}
*
* @group dfops
* @since 1.4.0
*/
def stat: DataFrameStatFunctions = new DataFrameStatFunctions(this)
/**
* Cartesian join with another [[DataFrame]].
*
* Note that cartesian joins are very expensive without an extra filter that can be pushed down.
*
* @param right Right side of the join operation.
* @group dfops
* @since 1.3.0
*/
def join(right: DataFrame): DataFrame = {
Join(logicalPlan, right.logicalPlan, joinType = Inner, None)
}
/**
* Inner equi-join with another [[DataFrame]] using the given column.
*
* Different from other join functions, the join column will only appear once in the output,
* i.e. similar to SQL's `JOIN USING` syntax.
*
* {{{
* // Joining df1 and df2 using the column "user_id"
* df1.join(df2, "user_id")
* }}}
*
* Note that if you perform a self-join using this function without aliasing the input
* [[DataFrame]]s, you will NOT be able to reference any columns after the join, since
* there is no way to disambiguate which side of the join you would like to reference.
*
* @param right Right side of the join operation.
* @param usingColumn Name of the column to join on. This column must exist on both sides.
* @group dfops
* @since 1.4.0
*/
def join(right: DataFrame, usingColumn: String): DataFrame = {
join(right, Seq(usingColumn))
}
/**
* Inner equi-join with another [[DataFrame]] using the given columns.
*
* Different from other join functions, the join columns will only appear once in the output,
* i.e. similar to SQL's `JOIN USING` syntax.
*
* {{{
* // Joining df1 and df2 using the columns "user_id" and "user_name"
* df1.join(df2, Seq("user_id", "user_name"))
* }}}
*
* Note that if you perform a self-join using this function without aliasing the input
* [[DataFrame]]s, you will NOT be able to reference any columns after the join, since
* there is no way to disambiguate which side of the join you would like to reference.
*
* @param right Right side of the join operation.
* @param usingColumns Names of the columns to join on. This columns must exist on both sides.
* @group dfops
* @since 1.4.0
*/
def join(right: DataFrame, usingColumns: Seq[String]): DataFrame = {
// Analyze the self join. The assumption is that the analyzer will disambiguate left vs right
// by creating a new instance for one of the branch.
val joined = sqlContext.executePlan(
Join(logicalPlan, right.logicalPlan, joinType = Inner, None)).analyzed.asInstanceOf[Join]
// Project only one of the join columns.
val joinedCols = usingColumns.map(col => joined.right.resolve(col))
val condition = usingColumns.map { col =>
catalyst.expressions.EqualTo(joined.left.resolve(col), joined.right.resolve(col))
}.reduceLeftOption[catalyst.expressions.BinaryExpression] { (cond, eqTo) =>
catalyst.expressions.And(cond, eqTo)
}
Project(
joined.output.filterNot(joinedCols.contains(_)),
Join(
joined.left,
joined.right,
joinType = Inner,
condition)
)
}
/**
* Inner join with another [[DataFrame]], using the given join expression.
*
* {{{
* // The following two are equivalent:
* df1.join(df2, $"df1Key" === $"df2Key")
* df1.join(df2).where($"df1Key" === $"df2Key")
* }}}
* @group dfops
* @since 1.3.0
*/
def join(right: DataFrame, joinExprs: Column): DataFrame = join(right, joinExprs, "inner")
/**
* Join with another [[DataFrame]], using the given join expression. The following performs
* a full outer join between `df1` and `df2`.
*
* {{{
* // Scala:
* import org.apache.spark.sql.functions._
* df1.join(df2, $"df1Key" === $"df2Key", "outer")
*
* // Java:
* import static org.apache.spark.sql.functions.*;
* df1.join(df2, col("df1Key").equalTo(col("df2Key")), "outer");
* }}}
*
* @param right Right side of the join.
* @param joinExprs Join expression.
* @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `leftsemi`.
* @group dfops
* @since 1.3.0
*/
def join(right: DataFrame, joinExprs: Column, joinType: String): DataFrame = {
// Note that in this function, we introduce a hack in the case of self-join to automatically
// resolve ambiguous join conditions into ones that might make sense [SPARK-6231].
// Consider this case: df.join(df, df("key") === df("key"))
// Since df("key") === df("key") is a trivially true condition, this actually becomes a
// cartesian join. However, most likely users expect to perform a self join using "key".
// With that assumption, this hack turns the trivially true condition into equality on join
// keys that are resolved to both sides.
// Trigger analysis so in the case of self-join, the analyzer will clone the plan.
// After the cloning, left and right side will have distinct expression ids.
val plan = Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr))
.queryExecution.analyzed.asInstanceOf[Join]
// If auto self join alias is disabled, return the plan.
if (!sqlContext.conf.dataFrameSelfJoinAutoResolveAmbiguity) {
return plan
}
// If left/right have no output set intersection, return the plan.
val lanalyzed = this.logicalPlan.queryExecution.analyzed
val ranalyzed = right.logicalPlan.queryExecution.analyzed
if (lanalyzed.outputSet.intersect(ranalyzed.outputSet).isEmpty) {
return plan
}
// Otherwise, find the trivially true predicates and automatically resolves them to both sides.
// By the time we get here, since we have already run analysis, all attributes should've been
// resolved and become AttributeReference.
val cond = plan.condition.map { _.transform {
case catalyst.expressions.EqualTo(a: AttributeReference, b: AttributeReference)
if a.sameRef(b) =>
catalyst.expressions.EqualTo(plan.left.resolve(a.name), plan.right.resolve(b.name))
}}
plan.copy(condition = cond)
}
/**
* Returns a new [[DataFrame]] sorted by the specified column, all in ascending order.
* {{{
* // The following 3 are equivalent
* df.sort("sortcol")
* df.sort($"sortcol")
* df.sort($"sortcol".asc)
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def sort(sortCol: String, sortCols: String*): DataFrame = {
sort((sortCol +: sortCols).map(apply) : _*)
}
/**
* Returns a new [[DataFrame]] sorted by the given expressions. For example:
* {{{
* df.sort($"col1", $"col2".desc)
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def sort(sortExprs: Column*): DataFrame = {
val sortOrder: Seq[SortOrder] = sortExprs.map { col =>
col.expr match {
case expr: SortOrder =>
expr
case expr: Expression =>
SortOrder(expr, Ascending)
}
}
Sort(sortOrder, global = true, logicalPlan)
}
/**
* Returns a new [[DataFrame]] sorted by the given expressions.
* This is an alias of the `sort` function.
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def orderBy(sortCol: String, sortCols: String*): DataFrame = sort(sortCol, sortCols : _*)
/**
* Returns a new [[DataFrame]] sorted by the given expressions.
* This is an alias of the `sort` function.
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def orderBy(sortExprs: Column*): DataFrame = sort(sortExprs : _*)
/**
* Selects column based on the column name and return it as a [[Column]].
* @group dfops
* @since 1.3.0
*/
def apply(colName: String): Column = col(colName)
/**
* Selects column based on the column name and return it as a [[Column]].
* @group dfops
* @since 1.3.0
*/
def col(colName: String): Column = colName match {
case "*" =>
Column(ResolvedStar(schema.fieldNames.map(resolve)))
case _ =>
val expr = resolve(colName)
Column(expr)
}
/**
* Returns a new [[DataFrame]] with an alias set.
* @group dfops
* @since 1.3.0
*/
def as(alias: String): DataFrame = Subquery(alias, logicalPlan)
/**
* (Scala-specific) Returns a new [[DataFrame]] with an alias set.
* @group dfops
* @since 1.3.0
*/
def as(alias: Symbol): DataFrame = as(alias.name)
/**
* Selects a set of column based expressions.
* {{{
* df.select($"colA", $"colB" + 1)
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def select(cols: Column*): DataFrame = {
val namedExpressions = cols.map {
// Wrap UnresolvedAttribute with UnresolvedAlias, as when we resolve UnresolvedAttribute, we
// will remove intermediate Alias for ExtractValue chain, and we need to alias it again to
// make it a NamedExpression.
case Column(u: UnresolvedAttribute) => UnresolvedAlias(u)
case Column(expr: NamedExpression) => expr
// Leave an unaliased explode with an empty list of names since the analzyer will generate the
// correct defaults after the nested expression's type has been resolved.
case Column(explode: Explode) => MultiAlias(explode, Nil)
case Column(expr: Expression) => Alias(expr, expr.prettyString)()
}
// When user continuously call `select`, speed up analysis by collapsing `Project`
import org.apache.spark.sql.catalyst.optimizer.ProjectCollapsing
Project(namedExpressions.toSeq, ProjectCollapsing(logicalPlan))
}
/**
* Selects a set of columns. This is a variant of `select` that can only select
* existing columns using column names (i.e. cannot construct expressions).
*
* {{{
* // The following two are equivalent:
* df.select("colA", "colB")
* df.select($"colA", $"colB")
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def select(col: String, cols: String*): DataFrame = select((col +: cols).map(Column(_)) : _*)
/**
* Selects a set of SQL expressions. This is a variant of `select` that accepts
* SQL expressions.
*
* {{{
* df.selectExpr("colA", "colB as newName", "abs(colC)")
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def selectExpr(exprs: String*): DataFrame = {
select(exprs.map { expr =>
Column(new SqlParser().parseExpression(expr))
}: _*)
}
/**
* Filters rows using the given condition.
* {{{
* // The following are equivalent:
* peopleDf.filter($"age" > 15)
* peopleDf.where($"age" > 15)
* }}}
* @group dfops
* @since 1.3.0
*/
def filter(condition: Column): DataFrame = Filter(condition.expr, logicalPlan)
/**
* Filters rows using the given SQL expression.
* {{{
* peopleDf.filter("age > 15")
* }}}
* @group dfops
* @since 1.3.0
*/
def filter(conditionExpr: String): DataFrame = {
filter(Column(new SqlParser().parseExpression(conditionExpr)))
}
/**
* Filters rows using the given condition. This is an alias for `filter`.
* {{{
* // The following are equivalent:
* peopleDf.filter($"age" > 15)
* peopleDf.where($"age" > 15)
* }}}
* @group dfops
* @since 1.3.0
*/
def where(condition: Column): DataFrame = filter(condition)
/**
* Filters rows using the given SQL expression.
* {{{
* peopleDf.where("age > 15")
* }}}
* @group dfops
* @since 1.5.0
*/
def where(conditionExpr: String): DataFrame = {
filter(Column(new SqlParser().parseExpression(conditionExpr)))
}
/**
* Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them.
* See [[GroupedData]] for all the available aggregate functions.
*
* {{{
* // Compute the average for all numeric columns grouped by department.
* df.groupBy($"department").avg()
*
* // Compute the max age and average salary, grouped by department and gender.
* df.groupBy($"department", $"gender").agg(Map(
* "salary" -> "avg",
* "age" -> "max"
* ))
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def groupBy(cols: Column*): GroupedData = {
GroupedData(this, cols.map(_.expr), GroupedData.GroupByType)
}
/**
* Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns,
* so we can run aggregation on them.
* See [[GroupedData]] for all the available aggregate functions.
*
* {{{
* // Compute the average for all numeric columns rolluped by department and group.
* df.rollup($"department", $"group").avg()
*
* // Compute the max age and average salary, rolluped by department and gender.
* df.rollup($"department", $"gender").agg(Map(
* "salary" -> "avg",
* "age" -> "max"
* ))
* }}}
* @group dfops
* @since 1.4.0
*/
@scala.annotation.varargs
def rollup(cols: Column*): GroupedData = {
GroupedData(this, cols.map(_.expr), GroupedData.RollupType)
}
/**
* Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns,
* so we can run aggregation on them.
* See [[GroupedData]] for all the available aggregate functions.
*
* {{{
* // Compute the average for all numeric columns cubed by department and group.
* df.cube($"department", $"group").avg()
*
* // Compute the max age and average salary, cubed by department and gender.
* df.cube($"department", $"gender").agg(Map(
* "salary" -> "avg",
* "age" -> "max"
* ))
* }}}
* @group dfops
* @since 1.4.0
*/
@scala.annotation.varargs
def cube(cols: Column*): GroupedData = GroupedData(this, cols.map(_.expr), GroupedData.CubeType)
/**
* Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them.
* See [[GroupedData]] for all the available aggregate functions.
*
* This is a variant of groupBy that can only group by existing columns using column names
* (i.e. cannot construct expressions).
*
* {{{
* // Compute the average for all numeric columns grouped by department.
* df.groupBy("department").avg()
*
* // Compute the max age and average salary, grouped by department and gender.
* df.groupBy($"department", $"gender").agg(Map(
* "salary" -> "avg",
* "age" -> "max"
* ))
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def groupBy(col1: String, cols: String*): GroupedData = {
val colNames: Seq[String] = col1 +: cols
GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.GroupByType)
}
/**
* Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns,
* so we can run aggregation on them.
* See [[GroupedData]] for all the available aggregate functions.
*
* This is a variant of rollup that can only group by existing columns using column names
* (i.e. cannot construct expressions).
*
* {{{
* // Compute the average for all numeric columns rolluped by department and group.
* df.rollup("department", "group").avg()
*
* // Compute the max age and average salary, rolluped by department and gender.
* df.rollup($"department", $"gender").agg(Map(
* "salary" -> "avg",
* "age" -> "max"
* ))
* }}}
* @group dfops
* @since 1.4.0
*/
@scala.annotation.varargs
def rollup(col1: String, cols: String*): GroupedData = {
val colNames: Seq[String] = col1 +: cols
GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.RollupType)
}
/**
* Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns,
* so we can run aggregation on them.
* See [[GroupedData]] for all the available aggregate functions.
*
* This is a variant of cube that can only group by existing columns using column names
* (i.e. cannot construct expressions).
*
* {{{
* // Compute the average for all numeric columns cubed by department and group.
* df.cube("department", "group").avg()
*
* // Compute the max age and average salary, cubed by department and gender.
* df.cube($"department", $"gender").agg(Map(
* "salary" -> "avg",
* "age" -> "max"
* ))
* }}}
* @group dfops
* @since 1.4.0
*/
@scala.annotation.varargs
def cube(col1: String, cols: String*): GroupedData = {
val colNames: Seq[String] = col1 +: cols
GroupedData(this, colNames.map(colName => resolve(colName)), GroupedData.CubeType)
}
/**
* (Scala-specific) Aggregates on the entire [[DataFrame]] without groups.
* {{{
* // df.agg(...) is a shorthand for df.groupBy().agg(...)
* df.agg("age" -> "max", "salary" -> "avg")
* df.groupBy().agg("age" -> "max", "salary" -> "avg")
* }}}
* @group dfops
* @since 1.3.0
*/
def agg(aggExpr: (String, String), aggExprs: (String, String)*): DataFrame = {
groupBy().agg(aggExpr, aggExprs : _*)
}
/**
* (Scala-specific) Aggregates on the entire [[DataFrame]] without groups.
* {{{
* // df.agg(...) is a shorthand for df.groupBy().agg(...)
* df.agg(Map("age" -> "max", "salary" -> "avg"))
* df.groupBy().agg(Map("age" -> "max", "salary" -> "avg"))
* }}}
* @group dfops
* @since 1.3.0
*/
def agg(exprs: Map[String, String]): DataFrame = groupBy().agg(exprs)
/**
* (Java-specific) Aggregates on the entire [[DataFrame]] without groups.
* {{{
* // df.agg(...) is a shorthand for df.groupBy().agg(...)
* df.agg(Map("age" -> "max", "salary" -> "avg"))
* df.groupBy().agg(Map("age" -> "max", "salary" -> "avg"))
* }}}
* @group dfops
* @since 1.3.0
*/
def agg(exprs: java.util.Map[String, String]): DataFrame = groupBy().agg(exprs)
/**
* Aggregates on the entire [[DataFrame]] without groups.
* {{{
* // df.agg(...) is a shorthand for df.groupBy().agg(...)
* df.agg(max($"age"), avg($"salary"))
* df.groupBy().agg(max($"age"), avg($"salary"))
* }}}
* @group dfops
* @since 1.3.0
*/
@scala.annotation.varargs
def agg(expr: Column, exprs: Column*): DataFrame = groupBy().agg(expr, exprs : _*)
/**
* Returns a new [[DataFrame]] by taking the first `n` rows. The difference between this function
* and `head` is that `head` returns an array while `limit` returns a new [[DataFrame]].
* @group dfops
* @since 1.3.0
*/
def limit(n: Int): DataFrame = Limit(Literal(n), logicalPlan)
/**
* Returns a new [[DataFrame]] containing union of rows in this frame and another frame.
* This is equivalent to `UNION ALL` in SQL.
* @group dfops
* @since 1.3.0
*/
def unionAll(other: DataFrame): DataFrame = Union(logicalPlan, other.logicalPlan)
/**
* Returns a new [[DataFrame]] containing rows only in both this frame and another frame.
* This is equivalent to `INTERSECT` in SQL.
* @group dfops
* @since 1.3.0
*/
def intersect(other: DataFrame): DataFrame = Intersect(logicalPlan, other.logicalPlan)
/**
* Returns a new [[DataFrame]] containing rows in this frame but not in another frame.
* This is equivalent to `EXCEPT` in SQL.
* @group dfops
* @since 1.3.0
*/
def except(other: DataFrame): DataFrame = Except(logicalPlan, other.logicalPlan)
/**
* Returns a new [[DataFrame]] by sampling a fraction of rows.
*
* @param withReplacement Sample with replacement or not.