Class Dataset<T>

Object
org.apache.spark.sql.Dataset<T>
All Implemented Interfaces:
Serializable, scala.Serializable

public class Dataset<T> extends Object implements scala.Serializable
A Dataset is a strongly typed collection of domain-specific objects that can be transformed in parallel using functional or relational operations. Each Dataset also has an untyped view called a DataFrame, which is a Dataset of Row.

Operations available on Datasets are divided into transformations and actions. Transformations are the ones that produce new Datasets, and actions are the ones that trigger computation and return results. Example transformations include map, filter, select, and aggregate (groupBy). Example actions count, show, or writing data out to file systems.

Datasets are "lazy", i.e. computations are only triggered when an action is invoked. Internally, a Dataset represents a logical plan that describes the computation required to produce the data. When an action is invoked, Spark's query optimizer optimizes the logical plan and generates a physical plan for efficient execution in a parallel and distributed manner. To explore the logical plan as well as optimized physical plan, use the explain function.

To efficiently support domain-specific objects, an Encoder is required. The encoder maps the domain specific type T to Spark's internal type system. For example, given a class Person with two fields, name (string) and age (int), an encoder is used to tell Spark to generate code at runtime to serialize the Person object into a binary structure. This binary structure often has much lower memory footprint as well as are optimized for efficiency in data processing (e.g. in a columnar format). To understand the internal binary representation for data, use the schema function.

There are typically two ways to create a Dataset. The most common way is by pointing Spark to some files on storage systems, using the read function available on a SparkSession.


   val people = spark.read.parquet("...").as[Person]  // Scala
   Dataset<Person> people = spark.read().parquet("...").as(Encoders.bean(Person.class)); // Java
 

Datasets can also be created through transformations available on existing Datasets. For example, the following creates a new Dataset by applying a filter on the existing one:


   val names = people.map(_.name)  // in Scala; names is a Dataset[String]
   Dataset<String> names = people.map((Person p) -> p.name, Encoders.STRING));
 

Dataset operations can also be untyped, through various domain-specific-language (DSL) functions defined in: Dataset (this class), Column, and functions. These operations are very similar to the operations available in the data frame abstraction in R or Python.

To select a column from the Dataset, 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 Dataset[Row] using SparkSession
   val people = spark.read.parquet("...")
   val department = spark.read.parquet("...")

   people.filter("age > 30")
     .join(department, people("deptId") === department("id"))
     .groupBy(department("name"), people("gender"))
     .agg(avg(people("salary")), max(people("age")))
 

and in Java:


   // To create Dataset<Row> using SparkSession
   Dataset<Row> people = spark.read().parquet("...");
   Dataset<Row> department = spark.read().parquet("...");

   people.filter(people.col("age").gt(30))
     .join(department, people.col("deptId").equalTo(department.col("id")))
     .groupBy(department.col("name"), people.col("gender"))
     .agg(avg(people.col("salary")), max(people.col("age")));
 

Since:
1.6.0
See Also:
  • Constructor Details

    • Dataset

      public Dataset(SparkSession sparkSession, org.apache.spark.sql.catalyst.plans.logical.LogicalPlan logicalPlan, Encoder<T> encoder)
    • Dataset

      public Dataset(SQLContext sqlContext, org.apache.spark.sql.catalyst.plans.logical.LogicalPlan logicalPlan, Encoder<T> encoder)
  • Method Details

    • curId

      public static AtomicLong curId()
    • DATASET_ID_KEY

      public static String DATASET_ID_KEY()
    • COL_POS_KEY

      public static String COL_POS_KEY()
    • DATASET_ID_TAG

      public static org.apache.spark.sql.catalyst.trees.TreeNodeTag<scala.collection.mutable.HashSet<Object>> DATASET_ID_TAG()
    • ofRows

      public static Dataset<Row> ofRows(SparkSession sparkSession, org.apache.spark.sql.catalyst.plans.logical.LogicalPlan logicalPlan)
    • ofRows

      public static Dataset<Row> ofRows(SparkSession sparkSession, org.apache.spark.sql.catalyst.plans.logical.LogicalPlan logicalPlan, org.apache.spark.sql.catalyst.QueryPlanningTracker tracker)
      A variant of ofRows that allows passing in a tracker so we can track query parsing time.
    • toDF

      public Dataset<Row> toDF(String... colNames)
      Converts this strongly typed collection of data to generic DataFrame with columns renamed. This can be quite convenient in conversion from an 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"
       

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sortWithinPartitions

      public Dataset<T> sortWithinPartitions(String sortCol, String... sortCols)
      Returns a new Dataset with each partition sorted by the given expressions.

      This is the same operation as "SORT BY" in SQL (Hive QL).

      Parameters:
      sortCol - (undocumented)
      sortCols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sortWithinPartitions

      public Dataset<T> sortWithinPartitions(Column... sortExprs)
      Returns a new Dataset with each partition sorted by the given expressions.

      This is the same operation as "SORT BY" in SQL (Hive QL).

      Parameters:
      sortExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sort

      public Dataset<T> sort(String sortCol, String... sortCols)
      Returns a new Dataset sorted by the specified column, all in ascending order.
      
         // The following 3 are equivalent
         ds.sort("sortcol")
         ds.sort($"sortcol")
         ds.sort($"sortcol".asc)
       

      Parameters:
      sortCol - (undocumented)
      sortCols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sort

      public Dataset<T> sort(Column... sortExprs)
      Returns a new Dataset sorted by the given expressions. For example:
      
         ds.sort($"col1", $"col2".desc)
       

      Parameters:
      sortExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • orderBy

      public Dataset<T> orderBy(String sortCol, String... sortCols)
      Returns a new Dataset sorted by the given expressions. This is an alias of the sort function.

      Parameters:
      sortCol - (undocumented)
      sortCols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • orderBy

      public Dataset<T> orderBy(Column... sortExprs)
      Returns a new Dataset sorted by the given expressions. This is an alias of the sort function.

      Parameters:
      sortExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • hint

      public Dataset<T> hint(String name, Object... parameters)
      Specifies some hint on the current Dataset. As an example, the following code specifies that one of the plan can be broadcasted:

      
         df1.join(df2.hint("broadcast"))
       

      Parameters:
      name - (undocumented)
      parameters - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.2.0
    • select

      public Dataset<Row> select(Column... cols)
      Selects a set of column based expressions.
      
         ds.select($"colA", $"colB" + 1)
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • select

      public Dataset<Row> select(String col, String... cols)
      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:
         ds.select("colA", "colB")
         ds.select($"colA", $"colB")
       

      Parameters:
      col - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • selectExpr

      public Dataset<Row> selectExpr(String... exprs)
      Selects a set of SQL expressions. This is a variant of select that accepts SQL expressions.

      
         // The following are equivalent:
         ds.selectExpr("colA", "colB as newName", "abs(colC)")
         ds.select(expr("colA"), expr("colB as newName"), expr("abs(colC)"))
       

      Parameters:
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • groupBy

      public RelationalGroupedDataset groupBy(Column... cols)
      Groups the Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset for all the available aggregate functions.

      
         // Compute the average for all numeric columns grouped by department.
         ds.groupBy($"department").avg()
      
         // Compute the max age and average salary, grouped by department and gender.
         ds.groupBy($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • rollup

      public RelationalGroupedDataset rollup(Column... cols)
      Create a multi-dimensional rollup for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset for all the available aggregate functions.

      
         // Compute the average for all numeric columns rolled up by department and group.
         ds.rollup($"department", $"group").avg()
      
         // Compute the max age and average salary, rolled up by department and gender.
         ds.rollup($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • cube

      public RelationalGroupedDataset cube(Column... cols)
      Create a multi-dimensional cube for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset for all the available aggregate functions.

      
         // Compute the average for all numeric columns cubed by department and group.
         ds.cube($"department", $"group").avg()
      
         // Compute the max age and average salary, cubed by department and gender.
         ds.cube($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • groupBy

      public RelationalGroupedDataset groupBy(String col1, String... cols)
      Groups the Dataset using the specified columns, so that we can run aggregation on them. See RelationalGroupedDataset 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.
         ds.groupBy("department").avg()
      
         // Compute the max age and average salary, grouped by department and gender.
         ds.groupBy($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       
      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • rollup

      public RelationalGroupedDataset rollup(String col1, String... cols)
      Create a multi-dimensional rollup for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset 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 rolled up by department and group.
         ds.rollup("department", "group").avg()
      
         // Compute the max age and average salary, rolled up by department and gender.
         ds.rollup($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • cube

      public RelationalGroupedDataset cube(String col1, String... cols)
      Create a multi-dimensional cube for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset 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.
         ds.cube("department", "group").avg()
      
         // Compute the max age and average salary, cubed by department and gender.
         ds.cube($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       
      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • agg

      public Dataset<Row> agg(Column expr, Column... exprs)
      Aggregates on the entire Dataset without groups.
      
         // ds.agg(...) is a shorthand for ds.groupBy().agg(...)
         ds.agg(max($"age"), avg($"salary"))
         ds.groupBy().agg(max($"age"), avg($"salary"))
       

      Parameters:
      expr - (undocumented)
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • observe

      public Dataset<T> observe(String name, Column expr, Column... exprs)
      Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset that returns the same result as the input, with the following guarantees:
      • It will compute the defined aggregates (metrics) on all the data that is flowing through the Dataset at that point.
      • It will report the value of the defined aggregate columns as soon as we reach a completion point. A completion point is either the end of a query (batch mode) or the end of a streaming epoch. The value of the aggregates only reflects the data processed since the previous completion point.
      Please note that continuous execution is currently not supported.

      The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that contain references to the input Dataset's columns must always be wrapped in an aggregate function.

      A user can observe these metrics by either adding StreamingQueryListener or a QueryExecutionListener to the spark session.

      
         // Monitor the metrics using a listener.
         spark.streams.addListener(new StreamingQueryListener() {
           override def onQueryStarted(event: QueryStartedEvent): Unit = {}
           override def onQueryProgress(event: QueryProgressEvent): Unit = {
             event.progress.observedMetrics.asScala.get("my_event").foreach { row =>
               // Trigger if the number of errors exceeds 5 percent
               val num_rows = row.getAs[Long]("rc")
               val num_error_rows = row.getAs[Long]("erc")
               val ratio = num_error_rows.toDouble / num_rows
               if (ratio > 0.05) {
                 // Trigger alert
               }
             }
           }
           override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {}
         })
         // Observe row count (rc) and error row count (erc) in the streaming Dataset
         val observed_ds = ds.observe("my_event", count(lit(1)).as("rc"), count($"error").as("erc"))
         observed_ds.writeStream.format("...").start()
       

      Parameters:
      name - (undocumented)
      expr - (undocumented)
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.0.0
    • observe

      public Dataset<T> observe(Observation observation, Column expr, Column... exprs)
      Observe (named) metrics through an org.apache.spark.sql.Observation instance. This is equivalent to calling observe(String, Column, Column*) but does not require adding org.apache.spark.sql.util.QueryExecutionListener to the spark session. This method does not support streaming datasets.

      A user can retrieve the metrics by accessing org.apache.spark.sql.Observation.get.

      
         // Observe row count (rows) and highest id (maxid) in the Dataset while writing it
         val observation = Observation("my_metrics")
         val observed_ds = ds.observe(observation, count(lit(1)).as("rows"), max($"id").as("maxid"))
         observed_ds.write.parquet("ds.parquet")
         val metrics = observation.get
       

      Parameters:
      observation - (undocumented)
      expr - (undocumented)
      exprs - (undocumented)
      Returns:
      (undocumented)
      Throws:
      IllegalArgumentException - If this is a streaming Dataset (this.isStreaming == true)

      Since:
      3.3.0
    • drop

      public Dataset<Row> drop(String... colNames)
      Returns a new Dataset with columns dropped. This is a no-op if schema doesn't contain column name(s).

      This method can only be used to drop top level columns. the colName string is treated literally without further interpretation.

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • drop

      public Dataset<Row> drop(Column col, Column... cols)
      Returns a new Dataset with columns dropped.

      This method can only be used to drop top level columns. This is a no-op if the Dataset doesn't have a columns with an equivalent expression.

      Parameters:
      col - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.4.0
    • dropDuplicates

      public Dataset<T> dropDuplicates(String col1, String... cols)
      Returns a new Dataset with duplicate rows removed, considering only the subset of columns.

      For a static batch Dataset, it just drops duplicate rows. For a streaming Dataset, it will keep all data across triggers as intermediate state to drop duplicates rows. You can use withWatermark(java.lang.String,java.lang.String) to limit how late the duplicate data can be and system will accordingly limit the state. In addition, too late data older than watermark will be dropped to avoid any possibility of duplicates.

      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • dropDuplicatesWithinWatermark

      public Dataset<T> dropDuplicatesWithinWatermark(String col1, String... cols)
      Returns a new Dataset with duplicates rows removed, considering only the subset of columns, within watermark.

      This only works with streaming Dataset, and watermark for the input Dataset must be set via withWatermark(java.lang.String,java.lang.String).

      For a streaming Dataset, this will keep all data across triggers as intermediate state to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated as long as the time distance of earliest and latest events are smaller than the delay threshold of watermark." Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.

      Note: too late data older than watermark will be dropped.

      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.5.0
    • describe

      public Dataset<Row> describe(String... cols)
      Computes basic statistics for numeric and string columns, including count, mean, stddev, min, and max. If no columns are given, this function computes statistics for all numerical or string columns.

      This function is meant for exploratory data analysis, as we make no guarantee about the backward compatibility of the schema of the resulting Dataset. If you want to programmatically compute summary statistics, use the agg function instead.

      
         ds.describe("age", "height").show()
      
         // output:
         // summary age   height
         // count   10.0  10.0
         // mean    53.3  178.05
         // stddev  11.6  15.7
         // min     18.0  163.0
         // max     92.0  192.0
       

      Use summary(java.lang.String...) for expanded statistics and control over which statistics to compute.

      Parameters:
      cols - Columns to compute statistics on.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • summary

      public Dataset<Row> summary(String... statistics)
      Computes specified statistics for numeric and string columns. Available statistics are:
      • count
      • mean
      • stddev
      • min
      • max
      • arbitrary approximate percentiles specified as a percentage (e.g. 75%)
      • count_distinct
      • approx_count_distinct

      If no statistics are given, this function computes count, mean, stddev, min, approximate quartiles (percentiles at 25%, 50%, and 75%), and max.

      This function is meant for exploratory data analysis, as we make no guarantee about the backward compatibility of the schema of the resulting Dataset. If you want to programmatically compute summary statistics, use the agg function instead.

      
         ds.summary().show()
      
         // output:
         // summary age   height
         // count   10.0  10.0
         // mean    53.3  178.05
         // stddev  11.6  15.7
         // min     18.0  163.0
         // 25%     24.0  176.0
         // 50%     24.0  176.0
         // 75%     32.0  180.0
         // max     92.0  192.0
       

      
         ds.summary("count", "min", "25%", "75%", "max").show()
      
         // output:
         // summary age   height
         // count   10.0  10.0
         // min     18.0  163.0
         // 25%     24.0  176.0
         // 75%     32.0  180.0
         // max     92.0  192.0
       

      To do a summary for specific columns first select them:

      
         ds.select("age", "height").summary().show()
       

      Specify statistics to output custom summaries:

      
         ds.summary("count", "count_distinct").show()
       

      The distinct count isn't included by default.

      You can also run approximate distinct counts which are faster:

      
         ds.summary("count", "approx_count_distinct").show()
       

      See also describe(java.lang.String...) for basic statistics.

      Parameters:
      statistics - Statistics from above list to be computed.

      Returns:
      (undocumented)
      Since:
      2.3.0
    • repartition

      public Dataset<T> repartition(int numPartitions, Column... partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions into numPartitions. The resulting Dataset is hash partitioned.

      This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL).

      Parameters:
      numPartitions - (undocumented)
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • repartition

      public Dataset<T> repartition(Column... partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions, using spark.sql.shuffle.partitions as number of partitions. The resulting Dataset is hash partitioned.

      This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL).

      Parameters:
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • repartitionByRange

      public Dataset<T> repartitionByRange(int numPartitions, Column... partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions into numPartitions. The resulting Dataset is range partitioned.

      At least one partition-by expression must be specified. When no explicit sort order is specified, "ascending nulls first" is assumed. Note, the rows are not sorted in each partition of the resulting Dataset.

      Note that due to performance reasons this method uses sampling to estimate the ranges. Hence, the output may not be consistent, since sampling can return different values. The sample size can be controlled by the config spark.sql.execution.rangeExchange.sampleSizePerPartition.

      Parameters:
      numPartitions - (undocumented)
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • repartitionByRange

      public Dataset<T> repartitionByRange(Column... partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions, using spark.sql.shuffle.partitions as number of partitions. The resulting Dataset is range partitioned.

      At least one partition-by expression must be specified. When no explicit sort order is specified, "ascending nulls first" is assumed. Note, the rows are not sorted in each partition of the resulting Dataset.

      Note that due to performance reasons this method uses sampling to estimate the ranges. Hence, the output may not be consistent, since sampling can return different values. The sample size can be controlled by the config spark.sql.execution.rangeExchange.sampleSizePerPartition.

      Parameters:
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • queryExecution

      public org.apache.spark.sql.execution.QueryExecution queryExecution()
    • encoder

      public Encoder<T> encoder()
    • sparkSession

      public SparkSession sparkSession()
    • classTag

      public scala.reflect.ClassTag<T> classTag()
    • sqlContext

      public SQLContext sqlContext()
    • toString

      public String toString()
      Overrides:
      toString in class Object
    • toDF

      public Dataset<Row> toDF()
      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.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • as

      public <U> Dataset<U> as(Encoder<U> evidence$2)
      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 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 Dataset does not match the desired U type, you can use select along with alias or as to rearrange or rename as required.

      Note that as[] only changes the view of the data that is passed into typed operations, such as map(), and does not eagerly project away any columns that are not present in the specified class.

      Parameters:
      evidence$2 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • to

      public Dataset<Row> to(StructType schema)
      Returns a new DataFrame where each row is reconciled to match the specified schema. Spark will:
      • Reorder columns and/or inner fields by name to match the specified schema.
      • Project away columns and/or inner fields that are not needed by the specified schema. Missing columns and/or inner fields (present in the specified schema but not input DataFrame) lead to failures.
      • Cast the columns and/or inner fields to match the data types in the specified schema, if the types are compatible, e.g., numeric to numeric (error if overflows), but not string to int.
      • Carry over the metadata from the specified schema, while the columns and/or inner fields still keep their own metadata if not overwritten by the specified schema.
      • Fail if the nullability is not compatible. For example, the column and/or inner field is nullable but the specified schema requires them to be not nullable.

      Parameters:
      schema - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.4.0
    • toDF

      public Dataset<Row> toDF(scala.collection.Seq<String> colNames)
      Converts this strongly typed collection of data to generic DataFrame with columns renamed. This can be quite convenient in conversion from an 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"
       

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • schema

      public StructType schema()
      Returns the schema of this Dataset.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • printSchema

      public void printSchema()
      Prints the schema to the console in a nice tree format.

      Since:
      1.6.0
    • printSchema

      public void printSchema(int level)
      Prints the schema up to the given level to the console in a nice tree format.

      Parameters:
      level - (undocumented)
      Since:
      3.0.0
    • explain

      public void explain(String mode)
      Prints the plans (logical and physical) with a format specified by a given explain mode.

      Parameters:
      mode - specifies the expected output format of plans.
      • simple Print only a physical plan.
      • extended: Print both logical and physical plans.
      • codegen: Print a physical plan and generated codes if they are available.
      • cost: Print a logical plan and statistics if they are available.
      • formatted: Split explain output into two sections: a physical plan outline and node details.
      Since:
      3.0.0
    • explain

      public void explain(boolean extended)
      Prints the plans (logical and physical) to the console for debugging purposes.

      Parameters:
      extended - default false. If false, prints only the physical plan.

      Since:
      1.6.0
    • explain

      public void explain()
      Prints the physical plan to the console for debugging purposes.

      Since:
      1.6.0
    • dtypes

      public scala.Tuple2<String,String>[] dtypes()
      Returns all column names and their data types as an array.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • columns

      public String[] columns()
      Returns all column names as an array.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • isLocal

      public boolean isLocal()
      Returns true if the collect and take methods can be run locally (without any Spark executors).

      Returns:
      (undocumented)
      Since:
      1.6.0
    • isEmpty

      public boolean isEmpty()
      Returns true if the Dataset is empty.

      Returns:
      (undocumented)
      Since:
      2.4.0
    • isStreaming

      public boolean isStreaming()
      Returns true if this Dataset contains one or more sources that continuously return data as it arrives. A Dataset that reads data from a streaming source must be executed as a StreamingQuery using the start() method in DataStreamWriter. Methods that return a single answer, e.g. count() or collect(), will throw an AnalysisException when there is a streaming source present.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • checkpoint

      public Dataset<T> checkpoint()
      Eagerly checkpoint a Dataset and return the new Dataset. Checkpointing can be used to truncate the logical plan of this Dataset, which is especially useful in iterative algorithms where the plan may grow exponentially. It will be saved to files inside the checkpoint directory set with SparkContext#setCheckpointDir.

      Returns:
      (undocumented)
      Since:
      2.1.0
    • checkpoint

      public Dataset<T> checkpoint(boolean eager)
      Returns a checkpointed version of this Dataset. Checkpointing can be used to truncate the logical plan of this Dataset, which is especially useful in iterative algorithms where the plan may grow exponentially. It will be saved to files inside the checkpoint directory set with SparkContext#setCheckpointDir.

      Parameters:
      eager - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.1.0
    • localCheckpoint

      public Dataset<T> localCheckpoint()
      Eagerly locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to truncate the logical plan of this Dataset, which is especially useful in iterative algorithms where the plan may grow exponentially. Local checkpoints are written to executor storage and despite potentially faster they are unreliable and may compromise job completion.

      Returns:
      (undocumented)
      Since:
      2.3.0
    • localCheckpoint

      public Dataset<T> localCheckpoint(boolean eager)
      Locally checkpoints a Dataset and return the new Dataset. Checkpointing can be used to truncate the logical plan of this Dataset, which is especially useful in iterative algorithms where the plan may grow exponentially. Local checkpoints are written to executor storage and despite potentially faster they are unreliable and may compromise job completion.

      Parameters:
      eager - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • withWatermark

      public Dataset<T> withWatermark(String eventTime, String delayThreshold)
      Defines an event time watermark for this Dataset. A watermark tracks a point in time before which we assume no more late data is going to arrive.

      Spark will use this watermark for several purposes:

      • To know when a given time window aggregation can be finalized and thus can be emitted when using output modes that do not allow updates.
      • To minimize the amount of state that we need to keep for on-going aggregations, mapGroupsWithState and dropDuplicates operators.
      The current watermark is computed by looking at the MAX(eventTime) seen across all of the partitions in the query minus a user specified delayThreshold. Due to the cost of coordinating this value across partitions, the actual watermark used is only guaranteed to be at least delayThreshold behind the actual event time. In some cases we may still process records that arrive more than delayThreshold late.

      Parameters:
      eventTime - the name of the column that contains the event time of the row.
      delayThreshold - the minimum delay to wait to data to arrive late, relative to the latest record that has been processed in the form of an interval (e.g. "1 minute" or "5 hours"). NOTE: This should not be negative.

      Returns:
      (undocumented)
      Since:
      2.1.0
    • show

      public void show(int numRows)
      Displays the 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
       

      Parameters:
      numRows - Number of rows to show

      Since:
      1.6.0
    • show

      public void show()
      Displays the top 20 rows of Dataset in a tabular form. Strings more than 20 characters will be truncated, and all cells will be aligned right.

      Since:
      1.6.0
    • show

      public void show(boolean truncate)
      Displays the top 20 rows of Dataset in a tabular form.

      Parameters:
      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
    • show

      public void show(int numRows, boolean truncate)
      Displays the Dataset 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
       
      Parameters:
      numRows - Number of rows to show
      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
    • show

      public void show(int numRows, int truncate)
      Displays the Dataset 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
       

      Parameters:
      numRows - Number of rows to show
      truncate - If set to more than 0, truncates strings to truncate characters and all cells will be aligned right.
      Since:
      1.6.0
    • show

      public void show(int numRows, int truncate, boolean vertical)
      Displays the Dataset 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
       

      If vertical enabled, this command prints output rows vertically (one line per column value)?

      
       -RECORD 0-------------------
        year            | 1980
        month           | 12
        AVG('Adj Close) | 0.503218
        AVG('Adj Close) | 0.595103
       -RECORD 1-------------------
        year            | 1981
        month           | 01
        AVG('Adj Close) | 0.523289
        AVG('Adj Close) | 0.570307
       -RECORD 2-------------------
        year            | 1982
        month           | 02
        AVG('Adj Close) | 0.436504
        AVG('Adj Close) | 0.475256
       -RECORD 3-------------------
        year            | 1983
        month           | 03
        AVG('Adj Close) | 0.410516
        AVG('Adj Close) | 0.442194
       -RECORD 4-------------------
        year            | 1984
        month           | 04
        AVG('Adj Close) | 0.450090
        AVG('Adj Close) | 0.483521
       

      Parameters:
      numRows - Number of rows to show
      truncate - If set to more than 0, truncates strings to truncate characters and all cells will be aligned right.
      vertical - If set to true, prints output rows vertically (one line per column value).
      Since:
      2.3.0
    • na

      public DataFrameNaFunctions na()
      Returns a DataFrameNaFunctions for working with missing data.
      
         // Dropping rows containing any null values.
         ds.na.drop()
       

      Returns:
      (undocumented)
      Since:
      1.6.0
    • stat

      public DataFrameStatFunctions stat()
      Returns a DataFrameStatFunctions for working statistic functions support.
      
         // Finding frequent items in column with name 'a'.
         ds.stat.freqItems(Seq("a"))
       

      Returns:
      (undocumented)
      Since:
      1.6.0
    • join

      public Dataset<Row> join(Dataset<?> right)
      Join with another DataFrame.

      Behaves as an INNER JOIN and requires a subsequent join predicate.

      Parameters:
      right - Right side of the join operation.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • join

      public Dataset<Row> join(Dataset<?> right, String usingColumn)
      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")
       

      Parameters:
      right - Right side of the join operation.
      usingColumn - Name of the column to join on. This column must exist on both sides.

      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      If you perform a self-join using this function without aliasing the input DataFrames, 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.

    • join

      public Dataset<Row> join(Dataset<?> right, String[] usingColumns)
      (Java-specific) Inner equi-join with another DataFrame using the given columns. See the Scala-specific overload for more details.

      Parameters:
      right - Right side of the join operation.
      usingColumns - Names of the columns to join on. This columns must exist on both sides.

      Returns:
      (undocumented)
      Since:
      3.4.0
    • join

      public Dataset<Row> join(Dataset<?> right, scala.collection.Seq<String> usingColumns)
      (Scala-specific) 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"))
       

      Parameters:
      right - Right side of the join operation.
      usingColumns - Names of the columns to join on. This columns must exist on both sides.

      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      If you perform a self-join using this function without aliasing the input DataFrames, 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.

    • join

      public Dataset<Row> join(Dataset<?> right, String usingColumn, String joinType)
      Equi-join with another DataFrame using the given column. A cross join with a predicate is specified as an inner join. If you would explicitly like to perform a cross join use the crossJoin method.

      Different from other join functions, the join column will only appear once in the output, i.e. similar to SQL's JOIN USING syntax.

      Parameters:
      right - Right side of the join operation.
      usingColumn - Name of the column to join on. This column must exist on both sides.
      joinType - Type of join to perform. Default inner. Must be one of: inner, cross, outer, full, fullouter, full_outer, left, leftouter, left_outer, right, rightouter, right_outer, semi, leftsemi, left_semi, anti, leftanti, left_anti.

      Returns:
      (undocumented)
      Since:
      3.4.0
      Note:
      If you perform a self-join using this function without aliasing the input DataFrames, 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.

    • join

      public Dataset<Row> join(Dataset<?> right, String[] usingColumns, String joinType)
      (Java-specific) Equi-join with another DataFrame using the given columns. See the Scala-specific overload for more details.

      Parameters:
      right - Right side of the join operation.
      usingColumns - Names of the columns to join on. This columns must exist on both sides.
      joinType - Type of join to perform. Default inner. Must be one of: inner, cross, outer, full, fullouter, full_outer, left, leftouter, left_outer, right, rightouter, right_outer, semi, leftsemi, left_semi, anti, leftanti, left_anti.

      Returns:
      (undocumented)
      Since:
      3.4.0
    • join

      public Dataset<Row> join(Dataset<?> right, scala.collection.Seq<String> usingColumns, String joinType)
      (Scala-specific) Equi-join with another DataFrame using the given columns. A cross join with a predicate is specified as an inner join. If you would explicitly like to perform a cross join use the crossJoin method.

      Different from other join functions, the join columns will only appear once in the output, i.e. similar to SQL's JOIN USING syntax.

      Parameters:
      right - Right side of the join operation.
      usingColumns - Names of the columns to join on. This columns must exist on both sides.
      joinType - Type of join to perform. Default inner. Must be one of: inner, cross, outer, full, fullouter, full_outer, left, leftouter, left_outer, right, rightouter, right_outer, semi, leftsemi, left_semi, anti, leftanti, left_anti.

      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      If you perform a self-join using this function without aliasing the input DataFrames, 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.

    • join

      public Dataset<Row> join(Dataset<?> right, Column joinExprs)
      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")
       

      Parameters:
      right - (undocumented)
      joinExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • join

      public Dataset<Row> join(Dataset<?> right, Column joinExprs, String joinType)
      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");
       

      Parameters:
      right - Right side of the join.
      joinExprs - Join expression.
      joinType - Type of join to perform. Default inner. Must be one of: inner, cross, outer, full, fullouter, full_outer, left, leftouter, left_outer, right, rightouter, right_outer, semi, leftsemi, left_semi, anti, leftanti, left_anti.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • crossJoin

      public Dataset<Row> crossJoin(Dataset<?> right)
      Explicit cartesian join with another DataFrame.

      Parameters:
      right - Right side of the join operation.

      Returns:
      (undocumented)
      Since:
      2.1.0
      Note:
      Cartesian joins are very expensive without an extra filter that can be pushed down.

    • joinWith

      public <U> Dataset<scala.Tuple2<T,U>> joinWith(Dataset<U> other, Column condition, String joinType)
      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.

      Parameters:
      other - Right side of the join.
      condition - Join expression.
      joinType - Type of join to perform. Default inner. Must be one of: inner, cross, outer, full, fullouter,full_outer, left, leftouter, left_outer, right, rightouter, right_outer.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • joinWith

      public <U> Dataset<scala.Tuple2<T,U>> joinWith(Dataset<U> other, Column condition)
      Using inner equi-join to join this Dataset returning a Tuple2 for each pair where condition evaluates to true.

      Parameters:
      other - Right side of the join.
      condition - Join expression.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • sortWithinPartitions

      public Dataset<T> sortWithinPartitions(String sortCol, scala.collection.Seq<String> sortCols)
      Returns a new Dataset with each partition sorted by the given expressions.

      This is the same operation as "SORT BY" in SQL (Hive QL).

      Parameters:
      sortCol - (undocumented)
      sortCols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sortWithinPartitions

      public Dataset<T> sortWithinPartitions(scala.collection.Seq<Column> sortExprs)
      Returns a new Dataset with each partition sorted by the given expressions.

      This is the same operation as "SORT BY" in SQL (Hive QL).

      Parameters:
      sortExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sort

      public Dataset<T> sort(String sortCol, scala.collection.Seq<String> sortCols)
      Returns a new Dataset sorted by the specified column, all in ascending order.
      
         // The following 3 are equivalent
         ds.sort("sortcol")
         ds.sort($"sortcol")
         ds.sort($"sortcol".asc)
       

      Parameters:
      sortCol - (undocumented)
      sortCols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • sort

      public Dataset<T> sort(scala.collection.Seq<Column> sortExprs)
      Returns a new Dataset sorted by the given expressions. For example:
      
         ds.sort($"col1", $"col2".desc)
       

      Parameters:
      sortExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • orderBy

      public Dataset<T> orderBy(String sortCol, scala.collection.Seq<String> sortCols)
      Returns a new Dataset sorted by the given expressions. This is an alias of the sort function.

      Parameters:
      sortCol - (undocumented)
      sortCols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • orderBy

      public Dataset<T> orderBy(scala.collection.Seq<Column> sortExprs)
      Returns a new Dataset sorted by the given expressions. This is an alias of the sort function.

      Parameters:
      sortExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • apply

      public Column apply(String colName)
      Selects column based on the column name and returns it as a Column.

      Parameters:
      colName - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      The column name can also reference to a nested column like a.b.

    • hint

      public Dataset<T> hint(String name, scala.collection.Seq<Object> parameters)
      Specifies some hint on the current Dataset. As an example, the following code specifies that one of the plan can be broadcasted:

      
         df1.join(df2.hint("broadcast"))
       

      Parameters:
      name - (undocumented)
      parameters - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.2.0
    • col

      public Column col(String colName)
      Selects column based on the column name and returns it as a Column.

      Parameters:
      colName - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      The column name can also reference to a nested column like a.b.

    • metadataColumn

      public Column metadataColumn(String colName)
      Selects a metadata column based on its logical column name, and returns it as a Column.

      A metadata column can be accessed this way even if the underlying data source defines a data column with a conflicting name.

      Parameters:
      colName - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.5.0
    • colRegex

      public Column colRegex(String colName)
      Selects column based on the column name specified as a regex and returns it as Column.
      Parameters:
      colName - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • as

      public Dataset<T> as(String alias)
      Returns a new Dataset with an alias set.

      Parameters:
      alias - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • as

      public Dataset<T> as(scala.Symbol alias)
      (Scala-specific) Returns a new Dataset with an alias set.

      Parameters:
      alias - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • alias

      public Dataset<T> alias(String alias)
      Returns a new Dataset with an alias set. Same as as.

      Parameters:
      alias - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • alias

      public Dataset<T> alias(scala.Symbol alias)
      (Scala-specific) Returns a new Dataset with an alias set. Same as as.

      Parameters:
      alias - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • select

      public Dataset<Row> select(scala.collection.Seq<Column> cols)
      Selects a set of column based expressions.
      
         ds.select($"colA", $"colB" + 1)
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • select

      public Dataset<Row> select(String col, scala.collection.Seq<String> cols)
      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:
         ds.select("colA", "colB")
         ds.select($"colA", $"colB")
       

      Parameters:
      col - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • selectExpr

      public Dataset<Row> selectExpr(scala.collection.Seq<String> exprs)
      Selects a set of SQL expressions. This is a variant of select that accepts SQL expressions.

      
         // The following are equivalent:
         ds.selectExpr("colA", "colB as newName", "abs(colC)")
         ds.select(expr("colA"), expr("colB as newName"), expr("abs(colC)"))
       

      Parameters:
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • select

      public <U1> Dataset<U1> select(TypedColumn<T,U1> c1)
      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])
       

      Parameters:
      c1 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • select

      public <U1, U2> Dataset<scala.Tuple2<U1,U2>> select(TypedColumn<T,U1> c1, TypedColumn<T,U2> c2)
      Returns a new Dataset by computing the given Column expressions for each element.

      Parameters:
      c1 - (undocumented)
      c2 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • select

      public <U1, U2, U3> Dataset<scala.Tuple3<U1,U2,U3>> select(TypedColumn<T,U1> c1, TypedColumn<T,U2> c2, TypedColumn<T,U3> c3)
      Returns a new Dataset by computing the given Column expressions for each element.

      Parameters:
      c1 - (undocumented)
      c2 - (undocumented)
      c3 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • select

      public <U1, U2, U3, U4> Dataset<scala.Tuple4<U1,U2,U3,U4>> select(TypedColumn<T,U1> c1, TypedColumn<T,U2> c2, TypedColumn<T,U3> c3, TypedColumn<T,U4> c4)
      Returns a new Dataset by computing the given Column expressions for each element.

      Parameters:
      c1 - (undocumented)
      c2 - (undocumented)
      c3 - (undocumented)
      c4 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • select

      public <U1, U2, U3, U4, U5> Dataset<scala.Tuple5<U1,U2,U3,U4,U5>> select(TypedColumn<T,U1> c1, TypedColumn<T,U2> c2, TypedColumn<T,U3> c3, TypedColumn<T,U4> c4, TypedColumn<T,U5> c5)
      Returns a new Dataset by computing the given Column expressions for each element.

      Parameters:
      c1 - (undocumented)
      c2 - (undocumented)
      c3 - (undocumented)
      c4 - (undocumented)
      c5 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • filter

      public Dataset<T> filter(Column condition)
      Filters rows using the given condition.
      
         // The following are equivalent:
         peopleDs.filter($"age" > 15)
         peopleDs.where($"age" > 15)
       

      Parameters:
      condition - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • filter

      public Dataset<T> filter(String conditionExpr)
      Filters rows using the given SQL expression.
      
         peopleDs.filter("age > 15")
       

      Parameters:
      conditionExpr - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • where

      public Dataset<T> where(Column condition)
      Filters rows using the given condition. This is an alias for filter.
      
         // The following are equivalent:
         peopleDs.filter($"age" > 15)
         peopleDs.where($"age" > 15)
       

      Parameters:
      condition - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • where

      public Dataset<T> where(String conditionExpr)
      Filters rows using the given SQL expression.
      
         peopleDs.where("age > 15")
       

      Parameters:
      conditionExpr - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • groupBy

      public RelationalGroupedDataset groupBy(scala.collection.Seq<Column> cols)
      Groups the Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset for all the available aggregate functions.

      
         // Compute the average for all numeric columns grouped by department.
         ds.groupBy($"department").avg()
      
         // Compute the max age and average salary, grouped by department and gender.
         ds.groupBy($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • rollup

      public RelationalGroupedDataset rollup(scala.collection.Seq<Column> cols)
      Create a multi-dimensional rollup for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset for all the available aggregate functions.

      
         // Compute the average for all numeric columns rolled up by department and group.
         ds.rollup($"department", $"group").avg()
      
         // Compute the max age and average salary, rolled up by department and gender.
         ds.rollup($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • cube

      public RelationalGroupedDataset cube(scala.collection.Seq<Column> cols)
      Create a multi-dimensional cube for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset for all the available aggregate functions.

      
         // Compute the average for all numeric columns cubed by department and group.
         ds.cube($"department", $"group").avg()
      
         // Compute the max age and average salary, cubed by department and gender.
         ds.cube($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • groupBy

      public RelationalGroupedDataset groupBy(String col1, scala.collection.Seq<String> cols)
      Groups the Dataset using the specified columns, so that we can run aggregation on them. See RelationalGroupedDataset 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.
         ds.groupBy("department").avg()
      
         // Compute the max age and average salary, grouped by department and gender.
         ds.groupBy($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       
      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • reduce

      public T reduce(scala.Function2<T,T,T> func)
      (Scala-specific) Reduces the elements of this Dataset using the specified binary function. The given func must be commutative and associative or the result may be non-deterministic.

      Parameters:
      func - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • reduce

      public T reduce(ReduceFunction<T> func)
      (Java-specific) Reduces the elements of this Dataset using the specified binary function. The given func must be commutative and associative or the result may be non-deterministic.

      Parameters:
      func - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • groupByKey

      public <K> KeyValueGroupedDataset<K,T> groupByKey(scala.Function1<T,K> func, Encoder<K> evidence$3)
      (Scala-specific) Returns a KeyValueGroupedDataset where the data is grouped by the given key func.

      Parameters:
      func - (undocumented)
      evidence$3 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • groupByKey

      public <K> KeyValueGroupedDataset<K,T> groupByKey(MapFunction<T,K> func, Encoder<K> encoder)
      (Java-specific) Returns a KeyValueGroupedDataset where the data is grouped by the given key func.

      Parameters:
      func - (undocumented)
      encoder - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • rollup

      public RelationalGroupedDataset rollup(String col1, scala.collection.Seq<String> cols)
      Create a multi-dimensional rollup for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset 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 rolled up by department and group.
         ds.rollup("department", "group").avg()
      
         // Compute the max age and average salary, rolled up by department and gender.
         ds.rollup($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       

      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • cube

      public RelationalGroupedDataset cube(String col1, scala.collection.Seq<String> cols)
      Create a multi-dimensional cube for the current Dataset using the specified columns, so we can run aggregation on them. See RelationalGroupedDataset 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.
         ds.cube("department", "group").avg()
      
         // Compute the max age and average salary, cubed by department and gender.
         ds.cube($"department", $"gender").agg(Map(
           "salary" -> "avg",
           "age" -> "max"
         ))
       
      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • agg

      public Dataset<Row> agg(scala.Tuple2<String,String> aggExpr, scala.collection.Seq<scala.Tuple2<String,String>> aggExprs)
      (Scala-specific) Aggregates on the entire Dataset without groups.
      
         // ds.agg(...) is a shorthand for ds.groupBy().agg(...)
         ds.agg("age" -> "max", "salary" -> "avg")
         ds.groupBy().agg("age" -> "max", "salary" -> "avg")
       

      Parameters:
      aggExpr - (undocumented)
      aggExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • agg

      public Dataset<Row> agg(scala.collection.immutable.Map<String,String> exprs)
      (Scala-specific) Aggregates on the entire Dataset without groups.
      
         // ds.agg(...) is a shorthand for ds.groupBy().agg(...)
         ds.agg(Map("age" -> "max", "salary" -> "avg"))
         ds.groupBy().agg(Map("age" -> "max", "salary" -> "avg"))
       

      Parameters:
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • agg

      public Dataset<Row> agg(Map<String,String> exprs)
      (Java-specific) Aggregates on the entire Dataset without groups.
      
         // ds.agg(...) is a shorthand for ds.groupBy().agg(...)
         ds.agg(Map("age" -> "max", "salary" -> "avg"))
         ds.groupBy().agg(Map("age" -> "max", "salary" -> "avg"))
       

      Parameters:
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • agg

      public Dataset<Row> agg(Column expr, scala.collection.Seq<Column> exprs)
      Aggregates on the entire Dataset without groups.
      
         // ds.agg(...) is a shorthand for ds.groupBy().agg(...)
         ds.agg(max($"age"), avg($"salary"))
         ds.groupBy().agg(max($"age"), avg($"salary"))
       

      Parameters:
      expr - (undocumented)
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • unpivot

      public Dataset<Row> unpivot(Column[] ids, Column[] values, String variableColumnName, String valueColumnName)
      Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. This is the reverse to groupBy(...).pivot(...).agg(...), except for the aggregation, which cannot be reversed.

      This function is useful to massage a DataFrame into a format where some columns are identifier columns ("ids"), while all other columns ("values") are "unpivoted" to the rows, leaving just two non-id columns, named as given by variableColumnName and valueColumnName.

      
         val df = Seq((1, 11, 12L), (2, 21, 22L)).toDF("id", "int", "long")
         df.show()
         // output:
         // +---+---+----+
         // | id|int|long|
         // +---+---+----+
         // |  1| 11|  12|
         // |  2| 21|  22|
         // +---+---+----+
      
         df.unpivot(Array($"id"), Array($"int", $"long"), "variable", "value").show()
         // output:
         // +---+--------+-----+
         // | id|variable|value|
         // +---+--------+-----+
         // |  1|     int|   11|
         // |  1|    long|   12|
         // |  2|     int|   21|
         // |  2|    long|   22|
         // +---+--------+-----+
         // schema:
         //root
         // |-- id: integer (nullable = false)
         // |-- variable: string (nullable = false)
         // |-- value: long (nullable = true)
       

      When no "id" columns are given, the unpivoted DataFrame consists of only the "variable" and "value" columns.

      All "value" columns must share a least common data type. Unless they are the same data type, all "value" columns are cast to the nearest common data type. For instance, types IntegerType and LongType are cast to LongType, while IntegerType and StringType do not have a common data type and unpivot fails with an AnalysisException.

      Parameters:
      ids - Id columns
      values - Value columns to unpivot
      variableColumnName - Name of the variable column
      valueColumnName - Name of the value column

      Returns:
      (undocumented)
      Since:
      3.4.0
    • unpivot

      public Dataset<Row> unpivot(Column[] ids, String variableColumnName, String valueColumnName)
      Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. This is the reverse to groupBy(...).pivot(...).agg(...), except for the aggregation, which cannot be reversed.

      Parameters:
      ids - Id columns
      variableColumnName - Name of the variable column
      valueColumnName - Name of the value column

      Returns:
      (undocumented)
      Since:
      3.4.0
      See Also:
      • org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)

        This is equivalent to calling Dataset#unpivot(Array, Array, String, String) where values is set to all non-id columns that exist in the DataFrame.

    • melt

      public Dataset<Row> melt(Column[] ids, Column[] values, String variableColumnName, String valueColumnName)
      Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. This is the reverse to groupBy(...).pivot(...).agg(...), except for the aggregation, which cannot be reversed. This is an alias for unpivot.

      Parameters:
      ids - Id columns
      values - Value columns to unpivot
      variableColumnName - Name of the variable column
      valueColumnName - Name of the value column

      Returns:
      (undocumented)
      Since:
      3.4.0
      See Also:
      • org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)

    • melt

      public Dataset<Row> melt(Column[] ids, String variableColumnName, String valueColumnName)
      Unpivot a DataFrame from wide format to long format, optionally leaving identifier columns set. This is the reverse to groupBy(...).pivot(...).agg(...), except for the aggregation, which cannot be reversed. This is an alias for unpivot.

      Parameters:
      ids - Id columns
      variableColumnName - Name of the variable column
      valueColumnName - Name of the value column

      Returns:
      (undocumented)
      Since:
      3.4.0
      See Also:
      • org.apache.spark.sql.Dataset.unpivot(Array, Array, String, String)

        This is equivalent to calling Dataset#unpivot(Array, Array, String, String) where values is set to all non-id columns that exist in the DataFrame.

    • observe

      public Dataset<T> observe(String name, Column expr, scala.collection.Seq<Column> exprs)
      Define (named) metrics to observe on the Dataset. This method returns an 'observed' Dataset that returns the same result as the input, with the following guarantees:
      • It will compute the defined aggregates (metrics) on all the data that is flowing through the Dataset at that point.
      • It will report the value of the defined aggregate columns as soon as we reach a completion point. A completion point is either the end of a query (batch mode) or the end of a streaming epoch. The value of the aggregates only reflects the data processed since the previous completion point.
      Please note that continuous execution is currently not supported.

      The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that contain references to the input Dataset's columns must always be wrapped in an aggregate function.

      A user can observe these metrics by either adding StreamingQueryListener or a QueryExecutionListener to the spark session.

      
         // Monitor the metrics using a listener.
         spark.streams.addListener(new StreamingQueryListener() {
           override def onQueryStarted(event: QueryStartedEvent): Unit = {}
           override def onQueryProgress(event: QueryProgressEvent): Unit = {
             event.progress.observedMetrics.asScala.get("my_event").foreach { row =>
               // Trigger if the number of errors exceeds 5 percent
               val num_rows = row.getAs[Long]("rc")
               val num_error_rows = row.getAs[Long]("erc")
               val ratio = num_error_rows.toDouble / num_rows
               if (ratio > 0.05) {
                 // Trigger alert
               }
             }
           }
           override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {}
         })
         // Observe row count (rc) and error row count (erc) in the streaming Dataset
         val observed_ds = ds.observe("my_event", count(lit(1)).as("rc"), count($"error").as("erc"))
         observed_ds.writeStream.format("...").start()
       

      Parameters:
      name - (undocumented)
      expr - (undocumented)
      exprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.0.0
    • observe

      public Dataset<T> observe(Observation observation, Column expr, scala.collection.Seq<Column> exprs)
      Observe (named) metrics through an org.apache.spark.sql.Observation instance. This is equivalent to calling observe(String, Column, Column*) but does not require adding org.apache.spark.sql.util.QueryExecutionListener to the spark session. This method does not support streaming datasets.

      A user can retrieve the metrics by accessing org.apache.spark.sql.Observation.get.

      
         // Observe row count (rows) and highest id (maxid) in the Dataset while writing it
         val observation = Observation("my_metrics")
         val observed_ds = ds.observe(observation, count(lit(1)).as("rows"), max($"id").as("maxid"))
         observed_ds.write.parquet("ds.parquet")
         val metrics = observation.get
       

      Parameters:
      observation - (undocumented)
      expr - (undocumented)
      exprs - (undocumented)
      Returns:
      (undocumented)
      Throws:
      IllegalArgumentException - If this is a streaming Dataset (this.isStreaming == true)

      Since:
      3.3.0
    • limit

      public Dataset<T> limit(int n)
      Returns a new Dataset by taking the first n rows. The difference between this function and head is that head is an action and returns an array (by triggering query execution) while limit returns a new Dataset.

      Parameters:
      n - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • offset

      public Dataset<T> offset(int n)
      Returns a new Dataset by skipping the first n rows.

      Parameters:
      n - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.4.0
    • union

      public Dataset<T> union(Dataset<T> other)
      Returns a new Dataset containing union of rows in this Dataset and another Dataset.

      This is equivalent to UNION ALL in SQL. To do a SQL-style set union (that does deduplication of elements), use this function followed by a distinct().

      Also as standard in SQL, this function resolves columns by position (not by name):

      
         val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2")
         val df2 = Seq((4, 5, 6)).toDF("col1", "col2", "col0")
         df1.union(df2).show
      
         // output:
         // +----+----+----+
         // |col0|col1|col2|
         // +----+----+----+
         // |   1|   2|   3|
         // |   4|   5|   6|
         // +----+----+----+
       

      Notice that the column positions in the schema aren't necessarily matched with the fields in the strongly typed objects in a Dataset. This function resolves columns by their positions in the schema, not the fields in the strongly typed objects. Use unionByName(org.apache.spark.sql.Dataset<T>) to resolve columns by field name in the typed objects.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • unionAll

      public Dataset<T> unionAll(Dataset<T> other)
      Returns a new Dataset containing union of rows in this Dataset and another Dataset. This is an alias for union.

      This is equivalent to UNION ALL in SQL. To do a SQL-style set union (that does deduplication of elements), use this function followed by a distinct().

      Also as standard in SQL, this function resolves columns by position (not by name).

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • unionByName

      public Dataset<T> unionByName(Dataset<T> other)
      Returns a new Dataset containing union of rows in this Dataset and another Dataset.

      This is different from both UNION ALL and UNION DISTINCT in SQL. To do a SQL-style set union (that does deduplication of elements), use this function followed by a distinct().

      The difference between this function and union(org.apache.spark.sql.Dataset<T>) is that this function resolves columns by name (not by position):

      
         val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2")
         val df2 = Seq((4, 5, 6)).toDF("col1", "col2", "col0")
         df1.unionByName(df2).show
      
         // output:
         // +----+----+----+
         // |col0|col1|col2|
         // +----+----+----+
         // |   1|   2|   3|
         // |   6|   4|   5|
         // +----+----+----+
       

      Note that this supports nested columns in struct and array types. Nested columns in map types are not currently supported.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • unionByName

      public Dataset<T> unionByName(Dataset<T> other, boolean allowMissingColumns)
      Returns a new Dataset containing union of rows in this Dataset and another Dataset.

      The difference between this function and union(org.apache.spark.sql.Dataset<T>) is that this function resolves columns by name (not by position).

      When the parameter allowMissingColumns is true, the set of column names in this and other Dataset can differ; missing columns will be filled with null. Further, the missing columns of this Dataset will be added at the end in the schema of the union result:

      
         val df1 = Seq((1, 2, 3)).toDF("col0", "col1", "col2")
         val df2 = Seq((4, 5, 6)).toDF("col1", "col0", "col3")
         df1.unionByName(df2, true).show
      
         // output: "col3" is missing at left df1 and added at the end of schema.
         // +----+----+----+----+
         // |col0|col1|col2|col3|
         // +----+----+----+----+
         // |   1|   2|   3|NULL|
         // |   5|   4|NULL|   6|
         // +----+----+----+----+
      
         df2.unionByName(df1, true).show
      
         // output: "col2" is missing at left df2 and added at the end of schema.
         // +----+----+----+----+
         // |col1|col0|col3|col2|
         // +----+----+----+----+
         // |   4|   5|   6|NULL|
         // |   2|   1|NULL|   3|
         // +----+----+----+----+
       

      Note that this supports nested columns in struct and array types. With allowMissingColumns, missing nested columns of struct columns with the same name will also be filled with null values and added to the end of struct. Nested columns in map types are not currently supported.

      Parameters:
      other - (undocumented)
      allowMissingColumns - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.1.0
    • intersect

      public Dataset<T> intersect(Dataset<T> other)
      Returns a new Dataset containing rows only in both this Dataset and another Dataset. This is equivalent to INTERSECT in SQL.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
      Note:
      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.

    • intersectAll

      public Dataset<T> intersectAll(Dataset<T> other)
      Returns a new Dataset containing rows only in both this Dataset and another Dataset while preserving the duplicates. This is equivalent to INTERSECT ALL in SQL.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.4.0
      Note:
      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. Also as standard in SQL, this function resolves columns by position (not by name).

    • except

      public Dataset<T> except(Dataset<T> other)
      Returns a new Dataset containing rows in this Dataset but not in another Dataset. This is equivalent to EXCEPT DISTINCT in SQL.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      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.

    • exceptAll

      public Dataset<T> exceptAll(Dataset<T> other)
      Returns a new Dataset containing rows in this Dataset but not in another Dataset while preserving the duplicates. This is equivalent to EXCEPT ALL in SQL.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.4.0
      Note:
      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. Also as standard in SQL, this function resolves columns by position (not by name).

    • sample

      public Dataset<T> sample(double fraction, long seed)
      Returns a new Dataset by sampling a fraction of rows (without replacement), using a user-supplied seed.

      Parameters:
      fraction - Fraction of rows to generate, range [0.0, 1.0].
      seed - Seed for sampling.

      Returns:
      (undocumented)
      Since:
      2.3.0
      Note:
      This is NOT guaranteed to provide exactly the fraction of the count of the given Dataset.

    • sample

      public Dataset<T> sample(double fraction)
      Returns a new Dataset by sampling a fraction of rows (without replacement), using a random seed.

      Parameters:
      fraction - Fraction of rows to generate, range [0.0, 1.0].

      Returns:
      (undocumented)
      Since:
      2.3.0
      Note:
      This is NOT guaranteed to provide exactly the fraction of the count of the given Dataset.

    • sample

      public Dataset<T> sample(boolean withReplacement, double fraction, long seed)
      Returns a new Dataset by sampling a fraction of rows, using a user-supplied seed.

      Parameters:
      withReplacement - Sample with replacement or not.
      fraction - Fraction of rows to generate, range [0.0, 1.0].
      seed - Seed for sampling.

      Returns:
      (undocumented)
      Since:
      1.6.0
      Note:
      This is NOT guaranteed to provide exactly the fraction of the count of the given Dataset.

    • sample

      public Dataset<T> sample(boolean withReplacement, double fraction)
      Returns a new Dataset by sampling a fraction of rows, using a random seed.

      Parameters:
      withReplacement - Sample with replacement or not.
      fraction - Fraction of rows to generate, range [0.0, 1.0].

      Returns:
      (undocumented)
      Since:
      1.6.0
      Note:
      This is NOT guaranteed to provide exactly the fraction of the total count of the given Dataset.

    • randomSplit

      public Dataset<T>[] randomSplit(double[] weights, long seed)
      Randomly splits this Dataset with the provided weights.

      Parameters:
      weights - weights for splits, will be normalized if they don't sum to 1.
      seed - Seed for sampling.

      For Java API, use randomSplitAsList(double[],long).

      Returns:
      (undocumented)
      Since:
      2.0.0
    • randomSplitAsList

      public List<Dataset<T>> randomSplitAsList(double[] weights, long seed)
      Returns a Java list that contains randomly split Dataset with the provided weights.

      Parameters:
      weights - weights for splits, will be normalized if they don't sum to 1.
      seed - Seed for sampling.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • randomSplit

      public Dataset<T>[] randomSplit(double[] weights)
      Randomly splits this Dataset with the provided weights.

      Parameters:
      weights - weights for splits, will be normalized if they don't sum to 1.
      Returns:
      (undocumented)
      Since:
      2.0.0
    • explode

      public <A extends scala.Product> Dataset<Row> explode(scala.collection.Seq<Column> input, scala.Function1<Row,scala.collection.TraversableOnce<A>> f, scala.reflect.api.TypeTags.TypeTag<A> evidence$4)
      Deprecated.
      use flatMap() or select() with functions.explode() instead. Since 2.0.0.
      (Scala-specific) Returns a new Dataset where each row has been expanded to zero or more rows by the provided function. This is similar to a LATERAL VIEW in HiveQL. The columns of the input row are implicitly joined with each row that is output by the function.

      Given that this is deprecated, as an alternative, you can explode columns either using functions.explode() or flatMap(). The following example uses these alternatives to count the number of books that contain a given word:

      
         case class Book(title: String, words: String)
         val ds: Dataset[Book]
      
         val allWords = ds.select($"title", explode(split($"words", " ")).as("word"))
      
         val bookCountPerWord = allWords.groupBy("word").agg(count_distinct("title"))
       

      Using flatMap() this can similarly be exploded as:

      
         ds.flatMap(_.words.split(" "))
       

      Parameters:
      input - (undocumented)
      f - (undocumented)
      evidence$4 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • explode

      public <A, B> Dataset<Row> explode(String inputColumn, String outputColumn, scala.Function1<A,scala.collection.TraversableOnce<B>> f, scala.reflect.api.TypeTags.TypeTag<B> evidence$5)
      Deprecated.
      use flatMap() or select() with functions.explode() instead. Since 2.0.0.
      (Scala-specific) Returns a new Dataset where a single column has been expanded to zero or more rows by the provided function. This is similar to a LATERAL VIEW in HiveQL. All columns of the input row are implicitly joined with each value that is output by the function.

      Given that this is deprecated, as an alternative, you can explode columns either using functions.explode():

      
         ds.select(explode(split($"words", " ")).as("word"))
       

      or flatMap():

      
         ds.flatMap(_.words.split(" "))
       

      Parameters:
      inputColumn - (undocumented)
      outputColumn - (undocumented)
      f - (undocumented)
      evidence$5 - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • withColumn

      public Dataset<Row> withColumn(String colName, Column col)
      Returns a new Dataset by adding a column or replacing the existing column that has the same name.

      column's expression must only refer to attributes supplied by this Dataset. It is an error to add a column that refers to some other Dataset.

      Parameters:
      colName - (undocumented)
      col - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      this method introduces a projection internally. Therefore, calling it multiple times, for instance, via loops in order to add multiple columns can generate big plans which can cause performance issues and even StackOverflowException. To avoid this, use select with the multiple columns at once.

    • withColumns

      public Dataset<Row> withColumns(scala.collection.immutable.Map<String,Column> colsMap)
      (Scala-specific) Returns a new Dataset by adding columns or replacing the existing columns that has the same names.

      colsMap is a map of column name and column, the column must only refer to attributes supplied by this Dataset. It is an error to add columns that refers to some other Dataset.

      Parameters:
      colsMap - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.3.0
    • withColumns

      public Dataset<Row> withColumns(Map<String,Column> colsMap)
      (Java-specific) Returns a new Dataset by adding columns or replacing the existing columns that has the same names.

      colsMap is a map of column name and column, the column must only refer to attribute supplied by this Dataset. It is an error to add columns that refers to some other Dataset.

      Parameters:
      colsMap - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.3.0
    • withColumnRenamed

      public Dataset<Row> withColumnRenamed(String existingName, String newName)
      Returns a new Dataset with a column renamed. This is a no-op if schema doesn't contain existingName.

      Parameters:
      existingName - (undocumented)
      newName - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • withColumnsRenamed

      public Dataset<Row> withColumnsRenamed(scala.collection.immutable.Map<String,String> colsMap) throws AnalysisException
      (Scala-specific) Returns a new Dataset with a columns renamed. This is a no-op if schema doesn't contain existingName.

      colsMap is a map of existing column name and new column name.

      Parameters:
      colsMap - (undocumented)
      Returns:
      (undocumented)
      Throws:
      AnalysisException - if there are duplicate names in resulting projection

      Since:
      3.4.0
    • withColumnsRenamed

      public Dataset<Row> withColumnsRenamed(Map<String,String> colsMap)
      (Java-specific) Returns a new Dataset with a columns renamed. This is a no-op if schema doesn't contain existingName.

      colsMap is a map of existing column name and new column name.

      Parameters:
      colsMap - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.4.0
    • withMetadata

      public Dataset<Row> withMetadata(String columnName, Metadata metadata)
      Returns a new Dataset by updating an existing column with metadata.

      Parameters:
      columnName - (undocumented)
      metadata - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.3.0
    • drop

      public Dataset<Row> drop(String colName)
      Returns a new Dataset with a column dropped. This is a no-op if schema doesn't contain column name.

      This method can only be used to drop top level columns. the colName string is treated literally without further interpretation.

      Note: drop(colName) has different semantic with drop(col(colName)), for example: 1, multi column have the same colName:

      
         val df1 = spark.range(0, 2).withColumn("key1", lit(1))
         val df2 = spark.range(0, 2).withColumn("key2", lit(2))
         val df3 = df1.join(df2)
      
         df3.show
         // +---+----+---+----+
         // | id|key1| id|key2|
         // +---+----+---+----+
         // |  0|   1|  0|   2|
         // |  0|   1|  1|   2|
         // |  1|   1|  0|   2|
         // |  1|   1|  1|   2|
         // +---+----+---+----+
      
         df3.drop("id").show()
         // output: the two 'id' columns are both dropped.
         // |key1|key2|
         // +----+----+
         // |   1|   2|
         // |   1|   2|
         // |   1|   2|
         // |   1|   2|
         // +----+----+
      
         df3.drop(col("id")).show()
         // ...AnalysisException: [AMBIGUOUS_REFERENCE] Reference `id` is ambiguous...
       

      2, colName contains special characters, like dot.

      
         val df = spark.range(0, 2).withColumn("a.b.c", lit(1))
      
         df.show()
         // +---+-----+
         // | id|a.b.c|
         // +---+-----+
         // |  0|    1|
         // |  1|    1|
         // +---+-----+
      
         df.drop("a.b.c").show()
         // +---+
         // | id|
         // +---+
         // |  0|
         // |  1|
         // +---+
      
         df.drop(col("a.b.c")).show()
         // no column match the expression 'a.b.c'
         // +---+-----+
         // | id|a.b.c|
         // +---+-----+
         // |  0|    1|
         // |  1|    1|
         // +---+-----+
       

      Parameters:
      colName - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • drop

      public Dataset<Row> drop(scala.collection.Seq<String> colNames)
      Returns a new Dataset with columns dropped. This is a no-op if schema doesn't contain column name(s).

      This method can only be used to drop top level columns. the colName string is treated literally without further interpretation.

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • drop

      public Dataset<Row> drop(Column col)
      Returns a new Dataset with column dropped.

      This method can only be used to drop top level column. This version of drop accepts a Column rather than a name. This is a no-op if the Dataset doesn't have a column with an equivalent expression.

      Note: drop(col(colName)) has different semantic with drop(colName), please refer to Dataset#drop(colName: String).

      Parameters:
      col - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • drop

      public Dataset<Row> drop(Column col, scala.collection.Seq<Column> cols)
      Returns a new Dataset with columns dropped.

      This method can only be used to drop top level columns. This is a no-op if the Dataset doesn't have a columns with an equivalent expression.

      Parameters:
      col - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.4.0
    • dropDuplicates

      public Dataset<T> dropDuplicates()
      Returns a new Dataset that contains only the unique rows from this Dataset. This is an alias for distinct.

      For a static batch Dataset, it just drops duplicate rows. For a streaming Dataset, it will keep all data across triggers as intermediate state to drop duplicates rows. You can use withWatermark(java.lang.String,java.lang.String) to limit how late the duplicate data can be and system will accordingly limit the state. In addition, too late data older than watermark will be dropped to avoid any possibility of duplicates.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • dropDuplicates

      public Dataset<T> dropDuplicates(scala.collection.Seq<String> colNames)
      (Scala-specific) Returns a new Dataset with duplicate rows removed, considering only the subset of columns.

      For a static batch Dataset, it just drops duplicate rows. For a streaming Dataset, it will keep all data across triggers as intermediate state to drop duplicates rows. You can use withWatermark(java.lang.String,java.lang.String) to limit how late the duplicate data can be and system will accordingly limit the state. In addition, too late data older than watermark will be dropped to avoid any possibility of duplicates.

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • dropDuplicates

      public Dataset<T> dropDuplicates(String[] colNames)
      Returns a new Dataset with duplicate rows removed, considering only the subset of columns.

      For a static batch Dataset, it just drops duplicate rows. For a streaming Dataset, it will keep all data across triggers as intermediate state to drop duplicates rows. You can use withWatermark(java.lang.String,java.lang.String) to limit how late the duplicate data can be and system will accordingly limit the state. In addition, too late data older than watermark will be dropped to avoid any possibility of duplicates.

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • dropDuplicates

      public Dataset<T> dropDuplicates(String col1, scala.collection.Seq<String> cols)
      Returns a new Dataset with duplicate rows removed, considering only the subset of columns.

      For a static batch Dataset, it just drops duplicate rows. For a streaming Dataset, it will keep all data across triggers as intermediate state to drop duplicates rows. You can use withWatermark(java.lang.String,java.lang.String) to limit how late the duplicate data can be and system will accordingly limit the state. In addition, too late data older than watermark will be dropped to avoid any possibility of duplicates.

      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • dropDuplicatesWithinWatermark

      public Dataset<T> dropDuplicatesWithinWatermark()
      Returns a new Dataset with duplicates rows removed, within watermark.

      This only works with streaming Dataset, and watermark for the input Dataset must be set via withWatermark(java.lang.String,java.lang.String).

      For a streaming Dataset, this will keep all data across triggers as intermediate state to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated as long as the time distance of earliest and latest events are smaller than the delay threshold of watermark." Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.

      Note: too late data older than watermark will be dropped.

      Returns:
      (undocumented)
      Since:
      3.5.0
    • dropDuplicatesWithinWatermark

      public Dataset<T> dropDuplicatesWithinWatermark(scala.collection.Seq<String> colNames)
      Returns a new Dataset with duplicates rows removed, considering only the subset of columns, within watermark.

      This only works with streaming Dataset, and watermark for the input Dataset must be set via withWatermark(java.lang.String,java.lang.String).

      For a streaming Dataset, this will keep all data across triggers as intermediate state to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated as long as the time distance of earliest and latest events are smaller than the delay threshold of watermark." Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.

      Note: too late data older than watermark will be dropped.

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.5.0
    • dropDuplicatesWithinWatermark

      public Dataset<T> dropDuplicatesWithinWatermark(String[] colNames)
      Returns a new Dataset with duplicates rows removed, considering only the subset of columns, within watermark.

      This only works with streaming Dataset, and watermark for the input Dataset must be set via withWatermark(java.lang.String,java.lang.String).

      For a streaming Dataset, this will keep all data across triggers as intermediate state to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated as long as the time distance of earliest and latest events are smaller than the delay threshold of watermark." Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.

      Note: too late data older than watermark will be dropped.

      Parameters:
      colNames - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.5.0
    • dropDuplicatesWithinWatermark

      public Dataset<T> dropDuplicatesWithinWatermark(String col1, scala.collection.Seq<String> cols)
      Returns a new Dataset with duplicates rows removed, considering only the subset of columns, within watermark.

      This only works with streaming Dataset, and watermark for the input Dataset must be set via withWatermark(java.lang.String,java.lang.String).

      For a streaming Dataset, this will keep all data across triggers as intermediate state to drop duplicated rows. The state will be kept to guarantee the semantic, "Events are deduplicated as long as the time distance of earliest and latest events are smaller than the delay threshold of watermark." Users are encouraged to set the delay threshold of watermark longer than max timestamp differences among duplicated events.

      Note: too late data older than watermark will be dropped.

      Parameters:
      col1 - (undocumented)
      cols - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.5.0
    • describe

      public Dataset<Row> describe(scala.collection.Seq<String> cols)
      Computes basic statistics for numeric and string columns, including count, mean, stddev, min, and max. If no columns are given, this function computes statistics for all numerical or string columns.

      This function is meant for exploratory data analysis, as we make no guarantee about the backward compatibility of the schema of the resulting Dataset. If you want to programmatically compute summary statistics, use the agg function instead.

      
         ds.describe("age", "height").show()
      
         // output:
         // summary age   height
         // count   10.0  10.0
         // mean    53.3  178.05
         // stddev  11.6  15.7
         // min     18.0  163.0
         // max     92.0  192.0
       

      Use summary(java.lang.String...) for expanded statistics and control over which statistics to compute.

      Parameters:
      cols - Columns to compute statistics on.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • summary

      public Dataset<Row> summary(scala.collection.Seq<String> statistics)
      Computes specified statistics for numeric and string columns. Available statistics are:
      • count
      • mean
      • stddev
      • min
      • max
      • arbitrary approximate percentiles specified as a percentage (e.g. 75%)
      • count_distinct
      • approx_count_distinct

      If no statistics are given, this function computes count, mean, stddev, min, approximate quartiles (percentiles at 25%, 50%, and 75%), and max.

      This function is meant for exploratory data analysis, as we make no guarantee about the backward compatibility of the schema of the resulting Dataset. If you want to programmatically compute summary statistics, use the agg function instead.

      
         ds.summary().show()
      
         // output:
         // summary age   height
         // count   10.0  10.0
         // mean    53.3  178.05
         // stddev  11.6  15.7
         // min     18.0  163.0
         // 25%     24.0  176.0
         // 50%     24.0  176.0
         // 75%     32.0  180.0
         // max     92.0  192.0
       

      
         ds.summary("count", "min", "25%", "75%", "max").show()
      
         // output:
         // summary age   height
         // count   10.0  10.0
         // min     18.0  163.0
         // 25%     24.0  176.0
         // 75%     32.0  180.0
         // max     92.0  192.0
       

      To do a summary for specific columns first select them:

      
         ds.select("age", "height").summary().show()
       

      Specify statistics to output custom summaries:

      
         ds.summary("count", "count_distinct").show()
       

      The distinct count isn't included by default.

      You can also run approximate distinct counts which are faster:

      
         ds.summary("count", "approx_count_distinct").show()
       

      See also describe(java.lang.String...) for basic statistics.

      Parameters:
      statistics - Statistics from above list to be computed.

      Returns:
      (undocumented)
      Since:
      2.3.0
    • head

      public Object head(int n)
      Returns the first n rows.

      Parameters:
      n - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
      Note:
      this method should only be used if the resulting array is expected to be small, as all the data is loaded into the driver's memory.

    • head

      public T head()
      Returns the first row.
      Returns:
      (undocumented)
      Since:
      1.6.0
    • first

      public T first()
      Returns the first row. Alias for head().
      Returns:
      (undocumented)
      Since:
      1.6.0
    • transform

      public <U> Dataset<U> transform(scala.Function1<Dataset<T>,Dataset<U>> t)
      Concise syntax for chaining custom transformations.
      
         def featurize(ds: Dataset[T]): Dataset[U] = ...
      
         ds
           .transform(featurize)
           .transform(...)
       

      Parameters:
      t - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • filter

      public Dataset<T> filter(scala.Function1<T,Object> func)
      (Scala-specific) Returns a new Dataset that only contains elements where func returns true.

      Parameters:
      func - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • filter

      public Dataset<T> filter(FilterFunction<T> func)
      (Java-specific) Returns a new Dataset that only contains elements where func returns true.

      Parameters:
      func - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • map

      public <U> Dataset<U> map(scala.Function1<T,U> func, Encoder<U> evidence$6)
      (Scala-specific) Returns a new Dataset that contains the result of applying func to each element.

      Parameters:
      func - (undocumented)
      evidence$6 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • map

      public <U> Dataset<U> map(MapFunction<T,U> func, Encoder<U> encoder)
      (Java-specific) Returns a new Dataset that contains the result of applying func to each element.

      Parameters:
      func - (undocumented)
      encoder - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • mapPartitions

      public <U> Dataset<U> mapPartitions(scala.Function1<scala.collection.Iterator<T>,scala.collection.Iterator<U>> func, Encoder<U> evidence$7)
      (Scala-specific) Returns a new Dataset that contains the result of applying func to each partition.

      Parameters:
      func - (undocumented)
      evidence$7 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • mapPartitions

      public <U> Dataset<U> mapPartitions(MapPartitionsFunction<T,U> f, Encoder<U> encoder)
      (Java-specific) Returns a new Dataset that contains the result of applying f to each partition.

      Parameters:
      f - (undocumented)
      encoder - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • flatMap

      public <U> Dataset<U> flatMap(scala.Function1<T,scala.collection.TraversableOnce<U>> func, Encoder<U> evidence$8)
      (Scala-specific) Returns a new Dataset by first applying a function to all elements of this Dataset, and then flattening the results.

      Parameters:
      func - (undocumented)
      evidence$8 - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • flatMap

      public <U> Dataset<U> flatMap(FlatMapFunction<T,U> f, Encoder<U> encoder)
      (Java-specific) Returns a new Dataset by first applying a function to all elements of this Dataset, and then flattening the results.

      Parameters:
      f - (undocumented)
      encoder - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • foreach

      public void foreach(scala.Function1<T,scala.runtime.BoxedUnit> f)
      Applies a function f to all rows.

      Parameters:
      f - (undocumented)
      Since:
      1.6.0
    • foreach

      public void foreach(ForeachFunction<T> func)
      (Java-specific) Runs func on each element of this Dataset.

      Parameters:
      func - (undocumented)
      Since:
      1.6.0
    • foreachPartition

      public void foreachPartition(scala.Function1<scala.collection.Iterator<T>,scala.runtime.BoxedUnit> f)
      Applies a function f to each partition of this Dataset.

      Parameters:
      f - (undocumented)
      Since:
      1.6.0
    • foreachPartition

      public void foreachPartition(ForeachPartitionFunction<T> func)
      (Java-specific) Runs func on each partition of this Dataset.

      Parameters:
      func - (undocumented)
      Since:
      1.6.0
    • take

      public Object take(int n)
      Returns the first n rows in the Dataset.

      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.

      Parameters:
      n - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • tail

      public Object tail(int n)
      Returns the last n rows in the Dataset.

      Running tail requires moving data into the application's driver process, and doing so with a very large n can crash the driver process with OutOfMemoryError.

      Parameters:
      n - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.0.0
    • takeAsList

      public List<T> takeAsList(int n)
      Returns the first n rows in the Dataset as a list.

      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.

      Parameters:
      n - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • collect

      public Object collect()
      Returns an array that contains all rows 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().

      Returns:
      (undocumented)
      Since:
      1.6.0
    • collectAsList

      public List<T> collectAsList()
      Returns a Java list that contains all rows 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.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • toLocalIterator

      public Iterator<T> toLocalIterator()
      Returns an iterator that contains all rows in this Dataset.

      The iterator will consume as much memory as the largest partition in this Dataset.

      Returns:
      (undocumented)
      Since:
      2.0.0
      Note:
      this results in multiple Spark jobs, and if the input Dataset is the result of a wide transformation (e.g. join with different partitioners), to avoid recomputing the input Dataset should be cached first.

    • count

      public long count()
      Returns the number of rows in the Dataset.
      Returns:
      (undocumented)
      Since:
      1.6.0
    • repartition

      public Dataset<T> repartition(int numPartitions)
      Returns a new Dataset that has exactly numPartitions partitions.

      Parameters:
      numPartitions - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • repartition

      public Dataset<T> repartition(int numPartitions, scala.collection.Seq<Column> partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions into numPartitions. The resulting Dataset is hash partitioned.

      This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL).

      Parameters:
      numPartitions - (undocumented)
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • repartition

      public Dataset<T> repartition(scala.collection.Seq<Column> partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions, using spark.sql.shuffle.partitions as number of partitions. The resulting Dataset is hash partitioned.

      This is the same operation as "DISTRIBUTE BY" in SQL (Hive QL).

      Parameters:
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.0.0
    • repartitionByRange

      public Dataset<T> repartitionByRange(int numPartitions, scala.collection.Seq<Column> partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions into numPartitions. The resulting Dataset is range partitioned.

      At least one partition-by expression must be specified. When no explicit sort order is specified, "ascending nulls first" is assumed. Note, the rows are not sorted in each partition of the resulting Dataset.

      Note that due to performance reasons this method uses sampling to estimate the ranges. Hence, the output may not be consistent, since sampling can return different values. The sample size can be controlled by the config spark.sql.execution.rangeExchange.sampleSizePerPartition.

      Parameters:
      numPartitions - (undocumented)
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • repartitionByRange

      public Dataset<T> repartitionByRange(scala.collection.Seq<Column> partitionExprs)
      Returns a new Dataset partitioned by the given partitioning expressions, using spark.sql.shuffle.partitions as number of partitions. The resulting Dataset is range partitioned.

      At least one partition-by expression must be specified. When no explicit sort order is specified, "ascending nulls first" is assumed. Note, the rows are not sorted in each partition of the resulting Dataset.

      Note that due to performance reasons this method uses sampling to estimate the ranges. Hence, the output may not be consistent, since sampling can return different values. The sample size can be controlled by the config spark.sql.execution.rangeExchange.sampleSizePerPartition.

      Parameters:
      partitionExprs - (undocumented)
      Returns:
      (undocumented)
      Since:
      2.3.0
    • coalesce

      public Dataset<T> coalesce(int numPartitions)
      Returns a new Dataset that has exactly numPartitions partitions, when the fewer partitions are requested. If a larger number of partitions is requested, it will stay at the current number of 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.

      However, if you're doing a drastic coalesce, e.g. to numPartitions = 1, this may result in your computation taking place on fewer nodes than you like (e.g. one node in the case of numPartitions = 1). To avoid this, you can call repartition. This will add a shuffle step, but means the current upstream partitions will be executed in parallel (per whatever the current partitioning is).

      Parameters:
      numPartitions - (undocumented)
      Returns:
      (undocumented)
      Since:
      1.6.0
    • distinct

      public Dataset<T> distinct()
      Returns a new Dataset that contains only the unique rows from this Dataset. This is an alias for dropDuplicates.

      Note that for a streaming Dataset, this method returns distinct rows only once regardless of the output mode, which the behavior may not be same with DISTINCT in SQL against streaming Dataset.

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

    • persist

      public Dataset<T> persist()
      Persist this Dataset with the default storage level (MEMORY_AND_DISK).

      Returns:
      (undocumented)
      Since:
      1.6.0
    • cache

      public Dataset<T> cache()
      Persist this Dataset with the default storage level (MEMORY_AND_DISK).

      Returns:
      (undocumented)
      Since:
      1.6.0
    • persist

      public Dataset<T> persist(StorageLevel newLevel)
      Persist this Dataset with the given storage level.
      Parameters:
      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.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • storageLevel

      public StorageLevel storageLevel()
      Get the Dataset's current storage level, or StorageLevel.NONE if not persisted.

      Returns:
      (undocumented)
      Since:
      2.1.0
    • unpersist

      public Dataset<T> unpersist(boolean blocking)
      Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. This will not un-persist any cached data that is built upon this Dataset.

      Parameters:
      blocking - Whether to block until all blocks are deleted.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • unpersist

      public Dataset<T> unpersist()
      Mark the Dataset as non-persistent, and remove all blocks for it from memory and disk. This will not un-persist any cached data that is built upon this Dataset.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • rdd

      public RDD<T> rdd()
    • toJavaRDD

      public JavaRDD<T> toJavaRDD()
      Returns the content of the Dataset as a JavaRDD of Ts.
      Returns:
      (undocumented)
      Since:
      1.6.0
    • javaRDD

      public JavaRDD<T> javaRDD()
      Returns the content of the Dataset as a JavaRDD of Ts.
      Returns:
      (undocumented)
      Since:
      1.6.0
    • registerTempTable

      public void registerTempTable(String tableName)
      Deprecated.
      Use createOrReplaceTempView(viewName) instead. Since 2.0.0.
      Registers this Dataset as a temporary table using the given name. The lifetime of this temporary table is tied to the SparkSession that was used to create this Dataset.

      Parameters:
      tableName - (undocumented)
      Since:
      1.6.0
    • createTempView

      public void createTempView(String viewName) throws AnalysisException
      Creates a local temporary view using the given name. The lifetime of this temporary view is tied to the SparkSession that was used to create this Dataset.

      Local temporary view is session-scoped. Its lifetime is the lifetime of the session that created it, i.e. it will be automatically dropped when the session terminates. It's not tied to any databases, i.e. we can't use db1.view1 to reference a local temporary view.

      Parameters:
      viewName - (undocumented)
      Throws:
      AnalysisException - if the view name is invalid or already exists

      Since:
      2.0.0
    • createOrReplaceTempView

      public void createOrReplaceTempView(String viewName)
      Creates a local temporary view using the given name. The lifetime of this temporary view is tied to the SparkSession that was used to create this Dataset.

      Parameters:
      viewName - (undocumented)
      Since:
      2.0.0
    • createGlobalTempView

      public void createGlobalTempView(String viewName) throws AnalysisException
      Creates a global temporary view using the given name. The lifetime of this temporary view is tied to this Spark application.

      Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database global_temp, and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

      Parameters:
      viewName - (undocumented)
      Throws:
      AnalysisException - if the view name is invalid or already exists

      Since:
      2.1.0
    • createOrReplaceGlobalTempView

      public void createOrReplaceGlobalTempView(String viewName)
      Creates or replaces a global temporary view using the given name. The lifetime of this temporary view is tied to this Spark application.

      Global temporary view is cross-session. Its lifetime is the lifetime of the Spark application, i.e. it will be automatically dropped when the application terminates. It's tied to a system preserved database global_temp, and we must use the qualified name to refer a global temp view, e.g. SELECT * FROM global_temp.view1.

      Parameters:
      viewName - (undocumented)
      Since:
      2.2.0
    • write

      public DataFrameWriter<T> write()
      Interface for saving the content of the non-streaming Dataset out into external storage.

      Returns:
      (undocumented)
      Since:
      1.6.0
    • writeTo

      public DataFrameWriterV2<T> writeTo(String table)
      Create a write configuration builder for v2 sources.

      This builder is used to configure and execute write operations. For example, to append to an existing table, run:

      
         df.writeTo("catalog.db.table").append()
       

      This can also be used to create or replace existing tables:

      
         df.writeTo("catalog.db.table").partitionedBy($"col").createOrReplace()
       

      Parameters:
      table - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.0.0
    • writeStream

      public DataStreamWriter<T> writeStream()
      Interface for saving the content of the streaming Dataset out into external storage.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • toJSON

      public Dataset<String> toJSON()
      Returns the content of the Dataset as a Dataset of JSON strings.
      Returns:
      (undocumented)
      Since:
      2.0.0
    • inputFiles

      public String[] inputFiles()
      Returns a best-effort snapshot of the files that compose this Dataset. This method simply asks each constituent BaseRelation for its respective files and takes the union of all results. Depending on the source relations, this may not find all input files. Duplicates are removed.

      Returns:
      (undocumented)
      Since:
      2.0.0
    • sameSemantics

      public boolean sameSemantics(Dataset<T> other)
      Returns true when the logical query plans inside both Datasets are equal and therefore return same results.

      Parameters:
      other - (undocumented)
      Returns:
      (undocumented)
      Since:
      3.1.0
      Note:
      The equality comparison here is simplified by tolerating the cosmetic differences such as attribute names., This API can compare both Datasets very fast but can still return false on the Dataset that return the same results, for instance, from different plans. Such false negative semantic can be useful when caching as an example.
    • semanticHash

      public int semanticHash()
      Returns a hashCode of the logical query plan against this Dataset.

      Returns:
      (undocumented)
      Since:
      3.1.0
      Note:
      Unlike the standard hashCode, the hash is calculated against the query plan simplified by tolerating the cosmetic differences such as attribute names.