forked from apache/spark
/
Dataset.scala
757 lines (671 loc) · 26.1 KB
/
Dataset.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
/*
* 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 scala.collection.JavaConverters._
import org.apache.spark.annotation.Experimental
import org.apache.spark.api.java.function._
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.encoders._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.analysis.UnresolvedAlias
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.execution.{Queryable, QueryExecution}
import org.apache.spark.sql.types.StructType
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.Utils
/**
* :: Experimental ::
* A [[Dataset]] is a strongly typed collection of objects that can be transformed in parallel
* using functional or relational operations.
*
* A [[Dataset]] differs from an [[RDD]] in the following ways:
* - Internally, a [[Dataset]] is represented by a Catalyst logical plan and the data is stored
* in the encoded form. This representation allows for additional logical operations and
* enables many operations (sorting, shuffling, etc.) to be performed without deserializing to
* an object.
* - The creation of a [[Dataset]] requires the presence of an explicit [[Encoder]] that can be
* used to serialize the object into a binary format. Encoders are also capable of mapping the
* schema of a given object to the Spark SQL type system. In contrast, RDDs rely on runtime
* reflection based serialization. Operations that change the type of object stored in the
* dataset also need an encoder for the new type.
*
* A [[Dataset]] can be thought of as a specialized DataFrame, where the elements map to a specific
* JVM object type, instead of to a generic [[Row]] container. A DataFrame can be transformed into
* specific Dataset by calling `df.as[ElementType]`. Similarly you can transform a strongly-typed
* [[Dataset]] to a generic DataFrame by calling `ds.toDF()`.
*
* COMPATIBILITY NOTE: Long term we plan to make [[DataFrame]] extend `Dataset[Row]`. However,
* making this change to the class hierarchy would break the function signatures for the existing
* functional operations (map, flatMap, etc). As such, this class should be considered a preview
* of the final API. Changes will be made to the interface after Spark 1.6.
*
* @since 1.6.0
*/
@Experimental
class Dataset[T] private[sql](
@transient override val sqlContext: SQLContext,
@transient override val queryExecution: QueryExecution,
tEncoder: Encoder[T]) extends Queryable with Serializable {
/**
* An unresolved version of the internal encoder for the type of this dataset. This one is marked
* implicit so that we can use it when constructing new [[Dataset]] objects that have the same
* object type (that will be possibly resolved to a different schema).
*/
private[sql] implicit val unresolvedTEncoder: ExpressionEncoder[T] = encoderFor(tEncoder)
/** The encoder for this [[Dataset]] that has been resolved to its output schema. */
private[sql] val resolvedTEncoder: ExpressionEncoder[T] =
unresolvedTEncoder.resolve(queryExecution.analyzed.output, OuterScopes.outerScopes)
private implicit def classTag = resolvedTEncoder.clsTag
private[sql] def this(sqlContext: SQLContext, plan: LogicalPlan)(implicit encoder: Encoder[T]) =
this(sqlContext, new QueryExecution(sqlContext, plan), encoder)
/**
* Returns the schema of the encoded form of the objects in this [[Dataset]].
* @since 1.6.0
*/
override def schema: StructType = resolvedTEncoder.schema
/**
* Prints the schema of the underlying [[DataFrame]] to the console in a nice tree format.
* @since 1.6.0
*/
override def printSchema(): Unit = toDF().printSchema()
/**
* Prints the plans (logical and physical) to the console for debugging purposes.
* @since 1.6.0
*/
override def explain(extended: Boolean): Unit = toDF().explain(extended)
/**
* Prints the physical plan to the console for debugging purposes.
* @since 1.6.0
*/
override def explain(): Unit = toDF().explain()
/* ************* *
* Conversions *
* ************* */
/**
* Returns a new `Dataset` where each record has been mapped on to the specified type. The
* method used to map columns depend on the type of `U`:
* - When `U` is a class, fields for the class will be mapped to columns of the same name
* (case sensitivity is determined by `spark.sql.caseSensitive`)
* - When `U` is a tuple, the columns will be be mapped by ordinal (i.e. the first column will
* be assigned to `_1`).
* - When `U` is a primitive type (i.e. String, Int, etc). then the first column of the
* [[DataFrame]] will be used.
*
* If the schema of the [[DataFrame]] does not match the desired `U` type, you can use `select`
* along with `alias` or `as` to rearrange or rename as required.
* @since 1.6.0
*/
def as[U : Encoder]: Dataset[U] = {
new Dataset(sqlContext, queryExecution, encoderFor[U])
}
/**
* Applies a logical alias to this [[Dataset]] that can be used to disambiguate columns that have
* the same name after two Datasets have been joined.
* @since 1.6.0
*/
def as(alias: String): Dataset[T] = withPlan(Subquery(alias, _))
/**
* Converts this strongly typed collection of data to generic Dataframe. In contrast to the
* strongly typed objects that Dataset operations work on, a Dataframe returns generic [[Row]]
* objects that allow fields to be accessed by ordinal or name.
*/
// This is declared with parentheses to prevent the Scala compiler from treating
// `ds.toDF("1")` as invoking this toDF and then apply on the returned DataFrame.
def toDF(): DataFrame = DataFrame(sqlContext, logicalPlan)
/**
* Returns this Dataset.
* @since 1.6.0
*/
// This is declared with parentheses to prevent the Scala compiler from treating
// `ds.toDS("1")` as invoking this toDF and then apply on the returned Dataset.
def toDS(): Dataset[T] = this
/**
* Converts this Dataset to an RDD.
* @since 1.6.0
*/
def rdd: RDD[T] = {
val tEnc = resolvedTEncoder
val input = queryExecution.analyzed.output
queryExecution.toRdd.mapPartitions { iter =>
val bound = tEnc.bind(input)
iter.map(bound.fromRow)
}
}
/**
* Returns the number of elements in the [[Dataset]].
* @since 1.6.0
*/
def count(): Long = toDF().count()
/**
* Displays the content of this [[Dataset]] 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
*
* @since 1.6.0
*/
def show(numRows: Int): Unit = show(numRows, truncate = 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.
*
* @since 1.6.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
*
* @since 1.6.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
*
* @since 1.6.0
*/
def show(numRows: Int, truncate: Boolean): Unit = toDF().show(numRows, truncate)
/**
* Returns a new [[Dataset]] that has exactly `numPartitions` partitions.
* @since 1.6.0
*/
def repartition(numPartitions: Int): Dataset[T] = withPlan {
Repartition(numPartitions, shuffle = true, _)
}
/**
* Returns a new [[Dataset]] that has exactly `numPartitions` partitions.
* Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g.
* if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of
* the 100 new partitions will claim 10 of the current partitions.
* @since 1.6.0
*/
def coalesce(numPartitions: Int): Dataset[T] = withPlan {
Repartition(numPartitions, shuffle = false, _)
}
/* *********************** *
* Functional Operations *
* *********************** */
/**
* Concise syntax for chaining custom transformations.
* {{{
* def featurize(ds: Dataset[T]) = ...
*
* dataset
* .transform(featurize)
* .transform(...)
* }}}
* @since 1.6.0
*/
def transform[U](t: Dataset[T] => Dataset[U]): Dataset[U] = t(this)
/**
* (Scala-specific)
* Returns a new [[Dataset]] that only contains elements where `func` returns `true`.
* @since 1.6.0
*/
def filter(func: T => Boolean): Dataset[T] = mapPartitions(_.filter(func))
/**
* (Java-specific)
* Returns a new [[Dataset]] that only contains elements where `func` returns `true`.
* @since 1.6.0
*/
def filter(func: FilterFunction[T]): Dataset[T] = filter(t => func.call(t))
/**
* (Scala-specific)
* Returns a new [[Dataset]] that contains the result of applying `func` to each element.
* @since 1.6.0
*/
def map[U : Encoder](func: T => U): Dataset[U] = mapPartitions(_.map(func))
/**
* (Java-specific)
* Returns a new [[Dataset]] that contains the result of applying `func` to each element.
* @since 1.6.0
*/
def map[U](func: MapFunction[T, U], encoder: Encoder[U]): Dataset[U] =
map(t => func.call(t))(encoder)
/**
* (Scala-specific)
* Returns a new [[Dataset]] that contains the result of applying `func` to each element.
* @since 1.6.0
*/
def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = {
new Dataset[U](
sqlContext,
MapPartitions[T, U](
func,
resolvedTEncoder,
encoderFor[U],
encoderFor[U].schema.toAttributes,
logicalPlan))
}
/**
* (Java-specific)
* Returns a new [[Dataset]] that contains the result of applying `func` to each element.
* @since 1.6.0
*/
def mapPartitions[U](f: MapPartitionsFunction[T, U], encoder: Encoder[U]): Dataset[U] = {
val func: (Iterator[T]) => Iterator[U] = x => f.call(x.asJava).iterator.asScala
mapPartitions(func)(encoder)
}
/**
* (Scala-specific)
* Returns a new [[Dataset]] by first applying a function to all elements of this [[Dataset]],
* and then flattening the results.
* @since 1.6.0
*/
def flatMap[U : Encoder](func: T => TraversableOnce[U]): Dataset[U] =
mapPartitions(_.flatMap(func))
/**
* (Java-specific)
* Returns a new [[Dataset]] by first applying a function to all elements of this [[Dataset]],
* and then flattening the results.
* @since 1.6.0
*/
def flatMap[U](f: FlatMapFunction[T, U], encoder: Encoder[U]): Dataset[U] = {
val func: (T) => Iterable[U] = x => f.call(x).asScala
flatMap(func)(encoder)
}
/* ************** *
* Side effects *
* ************** */
/**
* (Scala-specific)
* Runs `func` on each element of this Dataset.
* @since 1.6.0
*/
def foreach(func: T => Unit): Unit = rdd.foreach(func)
/**
* (Java-specific)
* Runs `func` on each element of this Dataset.
* @since 1.6.0
*/
def foreach(func: ForeachFunction[T]): Unit = foreach(func.call(_))
/**
* (Scala-specific)
* Runs `func` on each partition of this Dataset.
* @since 1.6.0
*/
def foreachPartition(func: Iterator[T] => Unit): Unit = rdd.foreachPartition(func)
/**
* (Java-specific)
* Runs `func` on each partition of this Dataset.
* @since 1.6.0
*/
def foreachPartition(func: ForeachPartitionFunction[T]): Unit =
foreachPartition(it => func.call(it.asJava))
/* ************* *
* Aggregation *
* ************* */
/**
* (Scala-specific)
* Reduces the elements of this Dataset using the specified binary function. The given function
* must be commutative and associative or the result may be non-deterministic.
* @since 1.6.0
*/
def reduce(func: (T, T) => T): T = rdd.reduce(func)
/**
* (Java-specific)
* Reduces the elements of this Dataset using the specified binary function. The given function
* must be commutative and associative or the result may be non-deterministic.
* @since 1.6.0
*/
def reduce(func: ReduceFunction[T]): T = reduce(func.call(_, _))
/**
* (Scala-specific)
* Returns a [[GroupedDataset]] where the data is grouped by the given key function.
* @since 1.6.0
*/
def groupBy[K : Encoder](func: T => K): GroupedDataset[K, T] = {
val inputPlan = queryExecution.analyzed
val withGroupingKey = AppendColumns(func, resolvedTEncoder, inputPlan)
val executed = sqlContext.executePlan(withGroupingKey)
new GroupedDataset(
encoderFor[K],
encoderFor[T],
executed,
inputPlan.output,
withGroupingKey.newColumns)
}
/**
* Returns a [[GroupedDataset]] where the data is grouped by the given [[Column]] expressions.
* @since 1.6.0
*/
@scala.annotation.varargs
def groupBy(cols: Column*): GroupedDataset[Row, T] = {
val withKeyColumns = logicalPlan.output ++ cols.map(_.expr).map(UnresolvedAlias)
val withKey = Project(withKeyColumns, logicalPlan)
val executed = sqlContext.executePlan(withKey)
val dataAttributes = executed.analyzed.output.dropRight(cols.size)
val keyAttributes = executed.analyzed.output.takeRight(cols.size)
new GroupedDataset(
RowEncoder(keyAttributes.toStructType),
encoderFor[T],
executed,
dataAttributes,
keyAttributes)
}
/**
* (Java-specific)
* Returns a [[GroupedDataset]] where the data is grouped by the given key function.
* @since 1.6.0
*/
def groupBy[K](f: MapFunction[T, K], encoder: Encoder[K]): GroupedDataset[K, T] =
groupBy(f.call(_))(encoder)
/* ****************** *
* Typed Relational *
* ****************** */
/**
* Selects a set of column based expressions.
* {{{
* df.select($"colA", $"colB" + 1)
* }}}
* @since 1.6.0
*/
// Copied from Dataframe to make sure we don't have invalid overloads.
@scala.annotation.varargs
protected def select(cols: Column*): DataFrame = toDF().select(cols: _*)
/**
* Returns a new [[Dataset]] by computing the given [[Column]] expression for each element.
*
* {{{
* val ds = Seq(1, 2, 3).toDS()
* val newDS = ds.select(expr("value + 1").as[Int])
* }}}
* @since 1.6.0
*/
def select[U1: Encoder](c1: TypedColumn[T, U1]): Dataset[U1] = {
new Dataset[U1](
sqlContext,
Project(
c1.withInputType(
resolvedTEncoder.bind(queryExecution.analyzed.output),
queryExecution.analyzed.output).named :: Nil,
logicalPlan))
}
/**
* Internal helper function for building typed selects that return tuples. For simplicity and
* code reuse, we do this without the help of the type system and then use helper functions
* that cast appropriately for the user facing interface.
*/
protected def selectUntyped(columns: TypedColumn[_, _]*): Dataset[_] = {
val encoders = columns.map(_.encoder)
val namedColumns =
columns.map(_.withInputType(resolvedTEncoder, queryExecution.analyzed.output).named)
val execution = new QueryExecution(sqlContext, Project(namedColumns, logicalPlan))
new Dataset(sqlContext, execution, ExpressionEncoder.tuple(encoders))
}
/**
* Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element.
* @since 1.6.0
*/
def select[U1, U2](c1: TypedColumn[T, U1], c2: TypedColumn[T, U2]): Dataset[(U1, U2)] =
selectUntyped(c1, c2).asInstanceOf[Dataset[(U1, U2)]]
/**
* Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element.
* @since 1.6.0
*/
def select[U1, U2, U3](
c1: TypedColumn[T, U1],
c2: TypedColumn[T, U2],
c3: TypedColumn[T, U3]): Dataset[(U1, U2, U3)] =
selectUntyped(c1, c2, c3).asInstanceOf[Dataset[(U1, U2, U3)]]
/**
* Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element.
* @since 1.6.0
*/
def select[U1, U2, U3, U4](
c1: TypedColumn[T, U1],
c2: TypedColumn[T, U2],
c3: TypedColumn[T, U3],
c4: TypedColumn[T, U4]): Dataset[(U1, U2, U3, U4)] =
selectUntyped(c1, c2, c3, c4).asInstanceOf[Dataset[(U1, U2, U3, U4)]]
/**
* Returns a new [[Dataset]] by computing the given [[Column]] expressions for each element.
* @since 1.6.0
*/
def select[U1, U2, U3, U4, U5](
c1: TypedColumn[T, U1],
c2: TypedColumn[T, U2],
c3: TypedColumn[T, U3],
c4: TypedColumn[T, U4],
c5: TypedColumn[T, U5]): Dataset[(U1, U2, U3, U4, U5)] =
selectUntyped(c1, c2, c3, c4, c5).asInstanceOf[Dataset[(U1, U2, U3, U4, U5)]]
/**
* Returns a new [[Dataset]] by sampling a fraction of records.
* @since 1.6.0
*/
def sample(withReplacement: Boolean, fraction: Double, seed: Long) : Dataset[T] =
withPlan(Sample(0.0, fraction, withReplacement, seed, _))
/**
* Returns a new [[Dataset]] by sampling a fraction of records, using a random seed.
* @since 1.6.0
*/
def sample(withReplacement: Boolean, fraction: Double) : Dataset[T] = {
sample(withReplacement, fraction, Utils.random.nextLong)
}
/* **************** *
* Set operations *
* **************** */
/**
* Returns a new [[Dataset]] that contains only the unique elements of this [[Dataset]].
*
* Note that, equality checking is performed directly on the encoded representation of the data
* and thus is not affected by a custom `equals` function defined on `T`.
* @since 1.6.0
*/
def distinct: Dataset[T] = withPlan(Distinct)
/**
* Returns a new [[Dataset]] that contains only the elements of this [[Dataset]] that are also
* present in `other`.
*
* Note that, equality checking is performed directly on the encoded representation of the data
* and thus is not affected by a custom `equals` function defined on `T`.
* @since 1.6.0
*/
def intersect(other: Dataset[T]): Dataset[T] = withPlan[T](other)(Intersect)
/**
* Returns a new [[Dataset]] that contains the elements of both this and the `other` [[Dataset]]
* combined.
*
* Note that, this function is not a typical set union operation, in that it does not eliminate
* duplicate items. As such, it is analogous to `UNION ALL` in SQL.
* @since 1.6.0
*/
def union(other: Dataset[T]): Dataset[T] = withPlan[T](other)(Union)
/**
* Returns a new [[Dataset]] where any elements present in `other` have been removed.
*
* Note that, equality checking is performed directly on the encoded representation of the data
* and thus is not affected by a custom `equals` function defined on `T`.
* @since 1.6.0
*/
def subtract(other: Dataset[T]): Dataset[T] = withPlan[T](other)(Except)
/* ****** *
* Joins *
* ****** */
/**
* Joins this [[Dataset]] returning a [[Tuple2]] for each pair where `condition` evaluates to
* true.
*
* This is similar to the relation `join` function with one important difference in the
* result schema. Since `joinWith` preserves objects present on either side of the join, the
* result schema is similarly nested into a tuple under the column names `_1` and `_2`.
*
* This type of join can be useful both for preserving type-safety with the original object
* types as well as working with relational data where either side of the join has column
* names in common.
*
* @param other Right side of the join.
* @param condition Join expression.
* @param joinType One of: `inner`, `outer`, `left_outer`, `right_outer`, `leftsemi`.
* @since 1.6.0
*/
def joinWith[U](other: Dataset[U], condition: Column, joinType: String): Dataset[(T, U)] = {
val left = this.logicalPlan
val right = other.logicalPlan
val joined = sqlContext.executePlan(Join(left, right, joinType =
JoinType(joinType), Some(condition.expr)))
val leftOutput = joined.analyzed.output.take(left.output.length)
val rightOutput = joined.analyzed.output.takeRight(right.output.length)
val leftData = this.unresolvedTEncoder match {
case e if e.flat => Alias(leftOutput.head, "_1")()
case _ => Alias(CreateStruct(leftOutput), "_1")()
}
val rightData = other.unresolvedTEncoder match {
case e if e.flat => Alias(rightOutput.head, "_2")()
case _ => Alias(CreateStruct(rightOutput), "_2")()
}
implicit val tuple2Encoder: Encoder[(T, U)] =
ExpressionEncoder.tuple(this.unresolvedTEncoder, other.unresolvedTEncoder)
withPlan[(T, U)](other) { (left, right) =>
Project(
leftData :: rightData :: Nil,
joined.analyzed)
}
}
/**
* Using inner equi-join to join this [[Dataset]] returning a [[Tuple2]] for each pair
* where `condition` evaluates to true.
*
* @param other Right side of the join.
* @param condition Join expression.
* @since 1.6.0
*/
def joinWith[U](other: Dataset[U], condition: Column): Dataset[(T, U)] = {
joinWith(other, condition, "inner")
}
/* ************************** *
* Gather to Driver Actions *
* ************************** */
/**
* Returns the first element in this [[Dataset]].
* @since 1.6.0
*/
def first(): T = take(1).head
/**
* Returns an array that contains all the elements in this [[Dataset]].
*
* Running collect requires moving all the data into the application's driver process, and
* doing so on a very large dataset can crash the driver process with OutOfMemoryError.
*
* For Java API, use [[collectAsList]].
* @since 1.6.0
*/
def collect(): Array[T] = {
// This is different from Dataset.rdd in that it collects Rows, and then runs the encoders
// to convert the rows into objects of type T.
val tEnc = resolvedTEncoder
val input = queryExecution.analyzed.output
val bound = tEnc.bind(input)
queryExecution.toRdd.map(_.copy()).collect().map(bound.fromRow)
}
/**
* Returns an array that contains all the elements in this [[Dataset]].
*
* Running collect requires moving all the data into the application's driver process, and
* doing so on a very large dataset can crash the driver process with OutOfMemoryError.
*
* For Java API, use [[collectAsList]].
* @since 1.6.0
*/
def collectAsList(): java.util.List[T] = collect().toSeq.asJava
/**
* Returns the first `num` elements of this [[Dataset]] as an array.
*
* Running take requires moving data into the application's driver process, and doing so with
* a very large `n` can crash the driver process with OutOfMemoryError.
* @since 1.6.0
*/
def take(num: Int): Array[T] = withPlan(Limit(Literal(num), _)).collect()
/**
* Returns the first `num` elements of this [[Dataset]] as an array.
*
* Running take requires moving data into the application's driver process, and doing so with
* a very large `n` can crash the driver process with OutOfMemoryError.
* @since 1.6.0
*/
def takeAsList(num: Int): java.util.List[T] = java.util.Arrays.asList(take(num) : _*)
/**
* Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`).
* @since 1.6.0
*/
def persist(): this.type = {
sqlContext.cacheManager.cacheQuery(this)
this
}
/**
* Persist this [[Dataset]] with the default storage level (`MEMORY_AND_DISK`).
* @since 1.6.0
*/
def cache(): this.type = persist()
/**
* Persist this [[Dataset]] with the given storage level.
* @param newLevel One of: `MEMORY_ONLY`, `MEMORY_AND_DISK`, `MEMORY_ONLY_SER`,
* `MEMORY_AND_DISK_SER`, `DISK_ONLY`, `MEMORY_ONLY_2`,
* `MEMORY_AND_DISK_2`, etc.
* @group basic
* @since 1.6.0
*/
def persist(newLevel: StorageLevel): this.type = {
sqlContext.cacheManager.cacheQuery(this, None, newLevel)
this
}
/**
* Mark the [[Dataset]] as non-persistent, and remove all blocks for it from memory and disk.
* @param blocking Whether to block until all blocks are deleted.
* @since 1.6.0
*/
def unpersist(blocking: Boolean): this.type = {
sqlContext.cacheManager.tryUncacheQuery(this, blocking)
this
}
/**
* Mark the [[Dataset]] as non-persistent, and remove all blocks for it from memory and disk.
* @since 1.6.0
*/
def unpersist(): this.type = unpersist(blocking = false)
/* ******************** *
* Internal Functions *
* ******************** */
private[sql] def logicalPlan: LogicalPlan = queryExecution.analyzed
private[sql] def withPlan(f: LogicalPlan => LogicalPlan): Dataset[T] =
new Dataset[T](sqlContext, sqlContext.executePlan(f(logicalPlan)), tEncoder)
private[sql] def withPlan[R : Encoder](
other: Dataset[_])(
f: (LogicalPlan, LogicalPlan) => LogicalPlan): Dataset[R] =
new Dataset[R](sqlContext, f(logicalPlan, other.logicalPlan))
}