org.apache.flink.api.scala

DataSet

class DataSet[T] extends AnyRef

The DataSet, the basic abstraction of Flink. This represents a collection of elements of a specific type T. The operations in this class can be used to create new DataSets and to combine two DataSets. The methods of ExecutionEnvironment can be used to create a DataSet from an external source, such as files in HDFS. The write* methods can be used to write the elements to storage.

All operations accept either a lambda function or an operation-specific function object for specifying the operation. For example, using a lambda:

val input: DataSet[String] = ...
val mapped = input flatMap { _.split(" ") }

And using a MapFunction:

val input: DataSet[String] = ...
val mapped = input flatMap { new FlatMapFunction[String, String] {
  def flatMap(in: String, out: Collector[String]): Unit = {
    in.split(" ") foreach { out.collect(_) }
  }
}

A rich function can be used when more control is required, for example for accessing the RuntimeContext. The rich function for flatMap is RichFlatMapFunction, all other functions are named similarly. All functions are available in package org.apache.flink.api.common.functions.

The elements are partitioned depending on the parallelism of the ExecutionEnvironment or of one specific DataSet.

Most of the operations have an implicit TypeInformation parameter. This is supplied by an implicit conversion in the flink.api.scala Package. For this to work, createTypeInformation needs to be imported. This is normally achieved with a

import org.apache.flink.api.scala._
T

The type of the DataSet, i.e., the type of the elements of the DataSet.

Annotations
@Public()
Linear Supertypes
AnyRef, Any
Known Subclasses
Ordering
  1. Alphabetic
  2. By inheritance
Inherited
  1. DataSet
  2. AnyRef
  3. Any
  1. Hide All
  2. Show all
Learn more about member selection
Visibility
  1. Public
  2. All

Instance Constructors

  1. new DataSet(set: java.DataSet[T])(implicit arg0: ClassTag[T])

Value Members

  1. final def !=(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  2. final def !=(arg0: Any): Boolean

    Definition Classes
    Any
  3. final def ##(): Int

    Definition Classes
    AnyRef → Any
  4. final def ==(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  5. final def ==(arg0: Any): Boolean

    Definition Classes
    Any
  6. def aggregate(agg: Aggregations, field: String): AggregateDataSet[T]

    Creates a new DataSet by aggregating the specified field using the given aggregation function.

    Creates a new DataSet by aggregating the specified field using the given aggregation function. Since this is not a keyed DataSet the aggregation will be performed on the whole collection of elements.

    This only works on CaseClass DataSets.

  7. def aggregate(agg: Aggregations, field: Int): AggregateDataSet[T]

    Creates a new DataSet by aggregating the specified tuple field using the given aggregation function.

    Creates a new DataSet by aggregating the specified tuple field using the given aggregation function. Since this is not a keyed DataSet the aggregation will be performed on the whole collection of elements.

    This only works on Tuple DataSets.

  8. final def asInstanceOf[T0]: T0

    Definition Classes
    Any
  9. def clone(): AnyRef

    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  10. def coGroup[O](other: DataSet[O])(implicit arg0: ClassTag[O]): UnfinishedCoGroupOperation[T, O]

    For each key in this DataSet and the other DataSet, create a tuple containing a list of elements for that key from both DataSets.

    For each key in this DataSet and the other DataSet, create a tuple containing a list of elements for that key from both DataSets. To specify the join keys the where and isEqualTo methods must be used. For example:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val coGrouped = left.coGroup(right).where(0).isEqualTo(1)

    A custom coGroup function can be used if more control over the result is required. This can either be given as a lambda or a custom CoGroupFunction. For example:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val coGrouped = left.coGroup(right).where(0).isEqualTo(1) { (l, r) =>
      // l and r are of type Iterator
      (l.min, r.max)
    }

    A coGroup function with a Collector can be used to implement a filter directly in the coGroup or to output more than one values. This type of coGroup function does not return a value, instead values are emitted using the collector

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val coGrouped = left.coGroup(right).where(0).isEqualTo(1) {
      (l, r, out: Collector[(String, Int)]) =>
        out.collect((l.min, r.max))
        out.collect(l.max, r.min))
      }
  11. def collect(): Seq[T]

    Convenience method to get the elements of a DataSet as a List As DataSet can contain a lot of data, this method should be used with caution.

    Convenience method to get the elements of a DataSet as a List As DataSet can contain a lot of data, this method should be used with caution.

    returns

    A Seq containing the elements of the DataSet

    Annotations
    @throws( classOf[Exception] )
    See also

    org.apache.flink.api.java.Utils.CollectHelper

  12. def combineGroup[R](fun: (Iterator[T], Collector[R]) ⇒ Unit)(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Applies a GroupCombineFunction on a grouped DataSet.

    Applies a GroupCombineFunction on a grouped DataSet. A GroupCombineFunction is similar to a GroupReduceFunction but does not perform a full data exchange. Instead, the GroupCombineFunction calls the combine method once per partition for combining a group of results. This operator is suitable for combining values into an intermediate format before doing a proper groupReduce where the data is shuffled across the node for further reduction. The GroupReduce operator can also be supplied with a combiner by implementing the RichGroupReduce function. The combine method of the RichGroupReduce function demands input and output type to be the same. The GroupCombineFunction, on the other side, can have an arbitrary output type.

  13. def combineGroup[R](combiner: GroupCombineFunction[T, R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Applies a GroupCombineFunction on a grouped DataSet.

    Applies a GroupCombineFunction on a grouped DataSet. A GroupCombineFunction is similar to a GroupReduceFunction but does not perform a full data exchange. Instead, the GroupCombineFunction calls the combine method once per partition for combining a group of results. This operator is suitable for combining values into an intermediate format before doing a proper groupReduce where the data is shuffled across the node for further reduction. The GroupReduce operator can also be supplied with a combiner by implementing the RichGroupReduce function. The combine method of the RichGroupReduce function demands input and output type to be the same. The GroupCombineFunction, on the other side, can have an arbitrary output type.

  14. def count(): Long

    Convenience method to get the count (number of elements) of a DataSet

    Convenience method to get the count (number of elements) of a DataSet

    returns

    A long integer that represents the number of elements in the set

    Annotations
    @throws( classOf[Exception] )
    See also

    org.apache.flink.api.java.Utils.CountHelper

  15. def cross[O](other: DataSet[O]): CrossDataSet[T, O]

    Creates a new DataSet by forming the cartesian product of this DataSet and the other DataSet.

    Creates a new DataSet by forming the cartesian product of this DataSet and the other DataSet.

    The default cross result is a DataSet with 2-Tuples of the combined values. A custom cross function can be used if more control over the result is required. This can either be given as a lambda or a custom CrossFunction. For example:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val product = left.cross(right) { (l, r) => (l._2, r._3) }
    }
  16. def crossWithHuge[O](other: DataSet[O]): CrossDataSet[T, O]

    Special cross operation for explicitly telling the system that the left side is assumed to be a lot smaller than the right side of the cartesian product.

  17. def crossWithTiny[O](other: DataSet[O]): CrossDataSet[T, O]

    Special cross operation for explicitly telling the system that the right side is assumed to be a lot smaller than the left side of the cartesian product.

  18. def distinct(firstField: String, otherFields: String*): DataSet[T]

    Returns a distinct set of this DataSet using expression keys.

    Returns a distinct set of this DataSet using expression keys.

    The field position keys specify the fields of Tuples or Pojos on which the decision is made if two elements are distinct or not.

    The field expression keys specify the fields of a org.apache.flink.api.common.typeutils.CompositeType (e.g., Tuple or Pojo type) on which the decision is made if two elements are distinct or not. In case of a org.apache.flink.api.common.typeinfo.AtomicType, only the wildcard expression ("_") is valid.

    firstField

    First field position on which the distinction of the DataSet is decided

    otherFields

    Zero or more field positions on which the distinction of the DataSet is decided

  19. def distinct(fields: Int*): DataSet[T]

    Returns a distinct set of a tuple DataSet using field position keys.

    Returns a distinct set of a tuple DataSet using field position keys.

    The field position keys specify the fields of Tuples on which the decision is made if two Tuples are distinct or not.

    Note: Field position keys can only be specified for Tuple DataSets.

    fields

    One or more field positions on which the distinction of the DataSet is decided.

  20. def distinct(): DataSet[T]

    Returns a distinct set of this DataSet.

    Returns a distinct set of this DataSet.

    If the input is a composite type (Tuple or Pojo type), distinct is performed on all fields and each field must be a key type.

  21. def distinct[K](fun: (T) ⇒ K)(implicit arg0: TypeInformation[K]): DataSet[T]

    Creates a new DataSet containing the distinct elements of this DataSet.

    Creates a new DataSet containing the distinct elements of this DataSet. The decision whether two elements are distinct or not is made using the return value of the given function.

    fun

    The function which extracts the key values from the DataSet on which the distinction of the DataSet is decided.

  22. final def eq(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  23. def equals(arg0: Any): Boolean

    Definition Classes
    AnyRef → Any
  24. def filter(fun: (T) ⇒ Boolean): DataSet[T]

    Creates a new DataSet that contains only the elements satisfying the given filter predicate.

  25. def filter(filter: FilterFunction[T]): DataSet[T]

    Creates a new DataSet that contains only the elements satisfying the given filter predicate.

  26. def finalize(): Unit

    Attributes
    protected[java.lang]
    Definition Classes
    AnyRef
    Annotations
    @throws( classOf[java.lang.Throwable] )
  27. def first(n: Int): DataSet[T]

    Creates a new DataSet containing the first n elements of this DataSet.

  28. def flatMap[R](fun: (T) ⇒ TraversableOnce[R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to every element and flattening the results.

  29. def flatMap[R](fun: (T, Collector[R]) ⇒ Unit)(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to every element and flattening the results.

  30. def flatMap[R](flatMapper: FlatMapFunction[T, R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to every element and flattening the results.

  31. def fullOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O]

    Special fullOuterJoin operation for explicitly telling the system what join strategy to use.

    Special fullOuterJoin operation for explicitly telling the system what join strategy to use. If null is given as the join strategy, then the optimizer will pick the strategy.

  32. def fullOuterJoin[O](other: DataSet[O]): UnfinishedOuterJoinOperation[T, O]

    Creates a new DataSet by performing a full outer join of this DataSet with the other DataSet, by combining two elements of two DataSets on key equality.

    Creates a new DataSet by performing a full outer join of this DataSet with the other DataSet, by combining two elements of two DataSets on key equality. Elements of both DataSets that do not have a matching element on the opposing side are joined with null and emitted to the resulting DataSet.

    To specify the join keys the where and equalTo methods must be used. For example:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val joined = left.fullOuterJoin(right).where(0).equalTo(1)

    When using an outer join you are required to specify a join function. For example:

    val joined = left.fullOuterJoin(right).where(0).equalTo(1) {
    (left, right) =>
      val a = if (left == null) null else left._1
      val b = if (right == null) null else right._3
      (a, b)
    }
  33. final def getClass(): Class[_]

    Definition Classes
    AnyRef → Any
  34. def getExecutionEnvironment: ExecutionEnvironment

    Returns the execution environment associated with the current DataSet.

    Returns the execution environment associated with the current DataSet.

    returns

    associated execution environment

  35. def getParallelism: Int

    Returns the parallelism of this operation.

  36. def getType(): TypeInformation[T]

    Returns the TypeInformation for the elements of this DataSet.

  37. def groupBy(firstField: String, otherFields: String*): GroupedDataSet[T]

    Creates a GroupedDataSet which provides operations on groups of elements.

    Creates a GroupedDataSet which provides operations on groups of elements. Elements are grouped based on the given fields.

    This will not create a new DataSet, it will just attach the field names which will be used for grouping when executing a grouped operation.

  38. def groupBy(fields: Int*): GroupedDataSet[T]

    Creates a GroupedDataSet which provides operations on groups of elements.

    Creates a GroupedDataSet which provides operations on groups of elements. Elements are grouped based on the given tuple fields.

    This will not create a new DataSet, it will just attach the tuple field positions which will be used for grouping when executing a grouped operation.

    This only works on Tuple DataSets.

  39. def groupBy[K](fun: (T) ⇒ K)(implicit arg0: TypeInformation[K]): GroupedDataSet[T]

    Creates a GroupedDataSet which provides operations on groups of elements.

    Creates a GroupedDataSet which provides operations on groups of elements. Elements are grouped based on the value returned by the given function.

    This will not create a new DataSet, it will just attach the key function which will be used for grouping when executing a grouped operation.

  40. def hashCode(): Int

    Definition Classes
    AnyRef → Any
  41. final def isInstanceOf[T0]: Boolean

    Definition Classes
    Any
  42. def iterate(maxIterations: Int)(stepFunction: (DataSet[T]) ⇒ DataSet[T]): DataSet[T]

    Creates a new DataSet by performing bulk iterations using the given step function.

    Creates a new DataSet by performing bulk iterations using the given step function. The iterations terminate when maxIterations iterations have been performed.

    For example:

    val input: DataSet[(String, Int)] = ...
    val iterated = input.iterate(5) { previous =>
      val next = previous.map { x => (x._1, x._2 + 1) }
      next
    }

    This example will simply increase the second field of the tuple by 5.

  43. def iterateDelta[R](workset: DataSet[R], maxIterations: Int, keyFields: Array[String], solutionSetUnManaged: Boolean)(stepFunction: (DataSet[T], DataSet[R]) ⇒ (DataSet[T], DataSet[R]))(implicit arg0: ClassTag[R]): DataSet[T]

    Creates a new DataSet by performing delta (or workset) iterations using the given step function.

    Creates a new DataSet by performing delta (or workset) iterations using the given step function. At the beginning this DataSet is the solution set and workset is the Workset. The iteration step function gets the current solution set and workset and must output the delta for the solution set and the workset for the next iteration.

    Note: The syntax of delta iterations are very likely going to change soon.

  44. def iterateDelta[R](workset: DataSet[R], maxIterations: Int, keyFields: Array[String])(stepFunction: (DataSet[T], DataSet[R]) ⇒ (DataSet[T], DataSet[R]))(implicit arg0: ClassTag[R]): DataSet[T]

    Creates a new DataSet by performing delta (or workset) iterations using the given step function.

    Creates a new DataSet by performing delta (or workset) iterations using the given step function. At the beginning this DataSet is the solution set and workset is the Workset. The iteration step function gets the current solution set and workset and must output the delta for the solution set and the workset for the next iteration.

    Note: The syntax of delta iterations are very likely going to change soon.

  45. def iterateDelta[R](workset: DataSet[R], maxIterations: Int, keyFields: Array[Int], solutionSetUnManaged: Boolean)(stepFunction: (DataSet[T], DataSet[R]) ⇒ (DataSet[T], DataSet[R]))(implicit arg0: ClassTag[R]): DataSet[T]

    Creates a new DataSet by performing delta (or workset) iterations using the given step function.

    Creates a new DataSet by performing delta (or workset) iterations using the given step function. At the beginning this DataSet is the solution set and workset is the Workset. The iteration step function gets the current solution set and workset and must output the delta for the solution set and the workset for the next iteration.

    Note: The syntax of delta iterations are very likely going to change soon.

  46. def iterateDelta[R](workset: DataSet[R], maxIterations: Int, keyFields: Array[Int])(stepFunction: (DataSet[T], DataSet[R]) ⇒ (DataSet[T], DataSet[R]))(implicit arg0: ClassTag[R]): DataSet[T]

    Creates a new DataSet by performing delta (or workset) iterations using the given step function.

    Creates a new DataSet by performing delta (or workset) iterations using the given step function. At the beginning this DataSet is the solution set and workset is the Workset. The iteration step function gets the current solution set and workset and must output the delta for the solution set and the workset for the next iteration.

    Note: The syntax of delta iterations are very likely going to change soon.

  47. def iterateWithTermination(maxIterations: Int)(stepFunction: (DataSet[T]) ⇒ (DataSet[T], DataSet[_])): DataSet[T]

    Creates a new DataSet by performing bulk iterations using the given step function.

    Creates a new DataSet by performing bulk iterations using the given step function. The first DataSet the step function returns is the input for the next iteration, the second DataSet is the termination criterion. The iterations terminate when either the termination criterion DataSet contains no elements or when maxIterations iterations have been performed.

    For example:

    val input: DataSet[(String, Int)] = ...
    val iterated = input.iterateWithTermination(5) { previous =>
      val next = previous.map { x => (x._1, x._2 + 1) }
      val term = next.filter { _._2 <  3 }
      (next, term)
    }

    This example will simply increase the second field of the Tuples until they are no longer smaller than 3.

  48. def join[O](other: DataSet[O], strategy: JoinHint): UnfinishedJoinOperation[T, O]

    Special join operation for explicitly telling the system what join strategy to use.

    Special join operation for explicitly telling the system what join strategy to use. If null is given as the join strategy, then the optimizer will pick the strategy.

  49. def join[O](other: DataSet[O]): UnfinishedJoinOperation[T, O]

    Creates a new DataSet by joining this DataSet with the other DataSet.

    Creates a new DataSet by joining this DataSet with the other DataSet. To specify the join keys the where and equalTo methods must be used. For example:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val joined = left.join(right).where(0).equalTo(1)

    The default join result is a DataSet with 2-Tuples of the joined values. In the above example that would be ((String, Int, Int), (Int, String, Int)). A custom join function can be used if more control over the result is required. This can either be given as a lambda or a custom JoinFunction. For example:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val joined = left.join(right).where(0).equalTo(1) { (l, r) =>
      (l._1, r._2)
    }

    A join function with a Collector can be used to implement a filter directly in the join or to output more than one values. This type of join function does not return a value, instead values are emitted using the collector:

    val left: DataSet[(String, Int, Int)] = ...
    val right: DataSet[(Int, String, Int)] = ...
    val joined = left.join(right).where(0).equalTo(1) {
      (l, r, out: Collector[(String, Int)]) =>
        if (l._2 > 4) {
          out.collect((l._1, r._3))
          out.collect((l._1, r._1))
        } else {
          None
        }
      }
  50. def joinWithHuge[O](other: DataSet[O]): UnfinishedJoinOperation[T, O]

    Special join operation for explicitly telling the system that the left side is assumed to be a lot smaller than the right side of the join.

  51. def joinWithTiny[O](other: DataSet[O]): UnfinishedJoinOperation[T, O]

    Special join operation for explicitly telling the system that the right side is assumed to be a lot smaller than the left side of the join.

  52. def leftOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O]

    An outer join on the left side.

    An outer join on the left side.

    Elements of the left side (i.e. this) that do not have a matching element on the other side are joined with null and emitted to the resulting DataSet.

    other

    The other DataSet with which this DataSet is joined.

    strategy

    The strategy that should be used execute the join. If { @code null} is given, then the optimizer will pick the join strategy.

    returns

    An UnfinishedJoinOperation to continue with the definition of the join transformation

    See also

    #fullOuterJoin

  53. def leftOuterJoin[O](other: DataSet[O]): UnfinishedOuterJoinOperation[T, O]

    An outer join on the left side.

    An outer join on the left side.

    Elements of the left side (i.e. this) that do not have a matching element on the other side are joined with null and emitted to the resulting DataSet.

    other

    The other DataSet with which this DataSet is joined.

    returns

    An UnfinishedJoinOperation to continue with the definition of the join transformation

    See also

    #fullOuterJoin

  54. def map[R](fun: (T) ⇒ R)(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to every element of this DataSet.

  55. def map[R](mapper: MapFunction[T, R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to every element of this DataSet.

  56. def mapPartition[R](fun: (Iterator[T]) ⇒ TraversableOnce[R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to each parallel partition of the DataSet.

    Creates a new DataSet by applying the given function to each parallel partition of the DataSet.

    This function is intended for operations that cannot transform individual elements and requires no grouping of elements. To transform individual elements, the use of map and flatMap is preferable.

  57. def mapPartition[R](fun: (Iterator[T], Collector[R]) ⇒ Unit)(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to each parallel partition of the DataSet.

    Creates a new DataSet by applying the given function to each parallel partition of the DataSet.

    This function is intended for operations that cannot transform individual elements and requires no grouping of elements. To transform individual elements, the use of map and flatMap is preferable.

  58. def mapPartition[R](partitionMapper: MapPartitionFunction[T, R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by applying the given function to each parallel partition of the DataSet.

    Creates a new DataSet by applying the given function to each parallel partition of the DataSet.

    This function is intended for operations that cannot transform individual elements and requires no grouping of elements. To transform individual elements, the use of map and flatMap is preferable.

  59. def max(field: String): AggregateDataSet[T]

    Syntactic sugar for aggregate with MAX

  60. def max(field: Int): AggregateDataSet[T]

    Syntactic sugar for aggregate with MAX

  61. def maxBy(fields: Int*): DataSet[T]

    Selects an element with maximum value.

    Selects an element with maximum value.

    The maximum is computed over the specified fields in lexicographical order.

    Example 1: Given a data set with elements [0, 1], [1, 0], the results will be:

    maxBy(0)[1, 0]
    maxBy(1)[0, 1]

    Example 2: Given a data set with elements [0, 0], [0, 1], the results will be:

    maxBy(0, 1)[0, 1]

    If multiple values with maximum value at the specified fields exist, a random one will be picked Internally, this operation is implemented as a ReduceFunction.

  62. def min(field: String): AggregateDataSet[T]

    Syntactic sugar for aggregate with MIN

  63. def min(field: Int): AggregateDataSet[T]

    Syntactic sugar for aggregate with MIN

  64. def minBy(fields: Int*): DataSet[T]

    Selects an element with minimum value.

    Selects an element with minimum value.

    The minimum is computed over the specified fields in lexicographical order.

    Example 1: Given a data set with elements [0, 1], [1, 0], the results will be:

    minBy(0)[0, 1]
    minBy(1)[1, 0]

    Example 2: Given a data set with elements [0, 0], [0, 1], the results will be:

    minBy(0, 1)[0, 0]

    If multiple values with minimum value at the specified fields exist, a random one will be picked. Internally, this operation is implemented as a ReduceFunction

  65. def name(name: String): DataSet[T]

    Sets the name of the DataSet.

    Sets the name of the DataSet. This will appear in logs and graphical representations of the execution graph.

  66. final def ne(arg0: AnyRef): Boolean

    Definition Classes
    AnyRef
  67. final def notify(): Unit

    Definition Classes
    AnyRef
  68. final def notifyAll(): Unit

    Definition Classes
    AnyRef
  69. def output(outputFormat: OutputFormat[T]): DataSink[T]

    Emits this DataSet using a custom org.apache.flink.api.common.io.OutputFormat.

  70. def partitionByHash[K](fun: (T) ⇒ K)(implicit arg0: TypeInformation[K]): DataSet[T]

    Partitions a DataSet using the specified key selector function.

    Partitions a DataSet using the specified key selector function.

    Important:This operation shuffles the whole DataSet over the network and can take significant amount of time.

  71. def partitionByHash(firstField: String, otherFields: String*): DataSet[T]

    Hash-partitions a DataSet on the specified fields.

    Hash-partitions a DataSet on the specified fields.

    important: This operation shuffles the whole DataSet over the network and can take significant amount of time.

  72. def partitionByHash(fields: Int*): DataSet[T]

    Hash-partitions a DataSet on the specified tuple field positions.

    Hash-partitions a DataSet on the specified tuple field positions.

    important: This operation shuffles the whole DataSet over the network and can take significant amount of time.

  73. def partitionByRange[K](fun: (T) ⇒ K)(implicit arg0: TypeInformation[K]): DataSet[T]

    Range-partitions a DataSet using the specified key selector function.

    Range-partitions a DataSet using the specified key selector function.

    important: This operation requires an extra pass over the DataSet to compute the range boundaries and shuffles the whole DataSet over the network. This can take significant amount of time.

  74. def partitionByRange(firstField: String, otherFields: String*): DataSet[T]

    Range-partitions a DataSet on the specified fields.

    Range-partitions a DataSet on the specified fields.

    important: This operation requires an extra pass over the DataSet to compute the range boundaries and shuffles the whole DataSet over the network. This can take significant amount of time.

  75. def partitionByRange(fields: Int*): DataSet[T]

    Range-partitions a DataSet on the specified tuple field positions.

    Range-partitions a DataSet on the specified tuple field positions.

    important: This operation requires an extra pass over the DataSet to compute the range boundaries and shuffles the whole DataSet over the network. This can take significant amount of time.

  76. def partitionCustom[K](partitioner: Partitioner[K], fun: (T) ⇒ K)(implicit arg0: TypeInformation[K]): DataSet[T]

    Partitions a DataSet on the key returned by the selector, using a custom partitioner.

    Partitions a DataSet on the key returned by the selector, using a custom partitioner. This method takes the key selector to get the key to partition on, and a partitioner that accepts the key type.

    Note: This method works only on single field keys, i.e. the selector cannot return tuples of fields.

  77. def partitionCustom[K](partitioner: Partitioner[K], field: String)(implicit arg0: TypeInformation[K]): DataSet[T]

    Partitions a POJO DataSet on the specified key fields using a custom partitioner.

    Partitions a POJO DataSet on the specified key fields using a custom partitioner. This method takes the key expression to partition on, and a partitioner that accepts the key type.

    Note: This method works only on single field keys.

  78. def partitionCustom[K](partitioner: Partitioner[K], field: Int)(implicit arg0: TypeInformation[K]): DataSet[T]

    Partitions a tuple DataSet on the specified key fields using a custom partitioner.

    Partitions a tuple DataSet on the specified key fields using a custom partitioner. This method takes the key position to partition on, and a partitioner that accepts the key type.

    Note: This method works only on single field keys.

  79. def print(): Unit

    Prints the elements in a DataSet to the standard output stream System.out of the JVM that calls the print() method.

    Prints the elements in a DataSet to the standard output stream System.out of the JVM that calls the print() method. For programs that are executed in a cluster, this method needs to gather the contents of the DataSet back to the client, to print it there.

    The string written for each element is defined by the AnyRef.toString method.

    This method immediately triggers the program execution, similar to the collect() and count() methods.

  80. def printOnTaskManager(prefix: String): DataSink[T]

    Writes a DataSet to the standard output streams (stdout) of the TaskManagers that execute the program (or more specifically, the data sink operators).

    Writes a DataSet to the standard output streams (stdout) of the TaskManagers that execute the program (or more specifically, the data sink operators). On a typical cluster setup, the data will appear in the TaskManagers' .out files.

    To print the data to the console or stdout stream of the client process instead, use the print() method.

    For each element of the DataSet the result of AnyRef.toString() is written.

    prefix

    The string to prefix each line of the output with. This helps identifying outputs from different printing sinks.

    returns

    The DataSink operator that writes the DataSet.

  81. def printToErr(): Unit

    Prints the elements in a DataSet to the standard error stream System.err of the JVM that calls the print() method.

    Prints the elements in a DataSet to the standard error stream System.err of the JVM that calls the print() method. For programs that are executed in a cluster, this method needs to gather the contents of the DataSet back to the client, to print it there.

    The string written for each element is defined by the AnyRef.toString method.

    This method immediately triggers the program execution, similar to the collect() and count() methods.

  82. def rebalance(): DataSet[T]

    Enforces a re-balancing of the DataSet, i.

    Enforces a re-balancing of the DataSet, i.e., the DataSet is evenly distributed over all parallel instances of the following task. This can help to improve performance in case of heavy data skew and compute intensive operations.

    Important: This operation shuffles the whole DataSet over the network and can take significant amount of time.

    returns

    The rebalanced DataSet.

  83. def reduce(fun: (T, T) ⇒ T): DataSet[T]

    Creates a new DataSet by merging the elements of this DataSet using an associative reduce function.

  84. def reduce(reducer: ReduceFunction[T]): DataSet[T]

    Creates a new DataSet by merging the elements of this DataSet using an associative reduce function.

  85. def reduceGroup[R](fun: (Iterator[T]) ⇒ R)(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by passing all elements in this DataSet to the group reduce function.

  86. def reduceGroup[R](fun: (Iterator[T], Collector[R]) ⇒ Unit)(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by passing all elements in this DataSet to the group reduce function.

    Creates a new DataSet by passing all elements in this DataSet to the group reduce function. The function can output zero or more elements using the Collector. The concatenation of the emitted values will form the resulting DataSet.

  87. def reduceGroup[R](reducer: GroupReduceFunction[T, R])(implicit arg0: TypeInformation[R], arg1: ClassTag[R]): DataSet[R]

    Creates a new DataSet by passing all elements in this DataSet to the group reduce function.

    Creates a new DataSet by passing all elements in this DataSet to the group reduce function. The function can output zero or more elements using the Collector. The concatenation of the emitted values will form the resulting DataSet.

  88. def registerAggregator(name: String, aggregator: Aggregator[_]): DataSet[T]

    Registers an org.apache.flink.api.common.aggregators.Aggregator for the iteration.

    Registers an org.apache.flink.api.common.aggregators.Aggregator for the iteration. Aggregators can be used to maintain simple statistics during the iteration, such as number of elements processed. The aggregators compute global aggregates: After each iteration step, the values are globally aggregated to produce one aggregate that represents statistics across all parallel instances. The value of an aggregator can be accessed in the next iteration.

    Aggregators can be accessed inside a function via org.apache.flink.api.common.functions.AbstractRichFunction#getIterationRuntimeContext.

    name

    The name under which the aggregator is registered.

    aggregator

    The aggregator class.

    Annotations
    @PublicEvolving()
  89. def rightOuterJoin[O](other: DataSet[O], strategy: JoinHint): UnfinishedOuterJoinOperation[T, O]

    An outer join on the right side.

    An outer join on the right side.

    Elements of the right side (i.e. other) that do not have a matching element on this side are joined with null and emitted to the resulting DataSet.

    other

    The other DataSet with which this DataSet is joined.

    strategy

    The strategy that should be used execute the join. If { @code null} is given, then the optimizer will pick the join strategy.

    returns

    An UnfinishedJoinOperation to continue with the definition of the join transformation

    See also

    #fullOuterJoin

  90. def rightOuterJoin[O](other: DataSet[O]): UnfinishedOuterJoinOperation[T, O]

    An outer join on the right side.

    An outer join on the right side.

    Elements of the right side (i.e. other) that do not have a matching element on this side are joined with null and emitted to the resulting DataSet.

    other

    The other DataSet with which this DataSet is joined.

    returns

    An UnfinishedJoinOperation to continue with the definition of the join transformation

    See also

    #fullOuterJoin

  91. def setParallelism(parallelism: Int): DataSet[T]

    Sets the parallelism of this operation.

    Sets the parallelism of this operation. This must be greater than 1.

  92. def sortPartition[K](fun: (T) ⇒ K, order: Order)(implicit arg0: TypeInformation[K]): DataSet[T]

    Locally sorts the partitions of the DataSet on the extracted key in the specified order.

    Locally sorts the partitions of the DataSet on the extracted key in the specified order. The DataSet can be sorted on multiple values by returning a tuple from the KeySelector.

    Note that no additional sort keys can be appended to a KeySelector sort keys. To sort the partitions by multiple values using KeySelector, the KeySelector must return a tuple consisting of the values.

  93. def sortPartition(field: String, order: Order): DataSet[T]

    Locally sorts the partitions of the DataSet on the specified field in the specified order.

    Locally sorts the partitions of the DataSet on the specified field in the specified order. The DataSet can be sorted on multiple fields by chaining sortPartition() calls.

  94. def sortPartition(field: Int, order: Order): DataSet[T]

    Locally sorts the partitions of the DataSet on the specified field in the specified order.

    Locally sorts the partitions of the DataSet on the specified field in the specified order. The DataSet can be sorted on multiple fields by chaining sortPartition() calls.

  95. def sum(field: String): AggregateDataSet[T]

    Syntactic sugar for aggregate with SUM

  96. def sum(field: Int): AggregateDataSet[T]

    Syntactic sugar for aggregate with SUM

  97. final def synchronized[T0](arg0: ⇒ T0): T0

    Definition Classes
    AnyRef
  98. def toString(): String

    Definition Classes
    AnyRef → Any
  99. def union(other: DataSet[T]): DataSet[T]

    Creates a new DataSet containing the elements from both this DataSet and the other DataSet.

  100. final def wait(): Unit

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  101. final def wait(arg0: Long, arg1: Int): Unit

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  102. final def wait(arg0: Long): Unit

    Definition Classes
    AnyRef
    Annotations
    @throws( ... )
  103. def withBroadcastSet(data: DataSet[_], name: String): DataSet[T]

    Adds a certain data set as a broadcast set to this operator.

    Adds a certain data set as a broadcast set to this operator. Broadcast data sets are available at all parallel instances of this operator. A broadcast data set is registered under a certain name, and can be retrieved under that name from the operators runtime context via org.apache.flink.api.common.functions.RuntimeContext.getBroadCastVariable(String)

    The runtime context itself is available in all UDFs via org.apache.flink.api.common.functions.AbstractRichFunction#getRuntimeContext()

    data

    The data set to be broadcasted.

    name

    The name under which the broadcast data set retrieved.

    returns

    The operator itself, to allow chaining function calls.

  104. def withForwardedFields(forwardedFields: String*): DataSet[T]

  105. def withForwardedFieldsFirst(forwardedFields: String*): DataSet[T]

  106. def withForwardedFieldsSecond(forwardedFields: String*): DataSet[T]

  107. def withParameters(parameters: Configuration): DataSet[T]

  108. def write(outputFormat: FileOutputFormat[T], filePath: String, writeMode: WriteMode = null): DataSink[T]

    Writes this DataSet to the specified location using a custom org.apache.flink.api.common.io.FileOutputFormat.

  109. def writeAsCsv(filePath: String, rowDelimiter: String = ..., fieldDelimiter: String = ..., writeMode: WriteMode = null): DataSink[T]

    Writes this DataSet to the specified location as CSV file(s).

    Writes this DataSet to the specified location as CSV file(s).

    This only works on Tuple DataSets. For individual tuple fields AnyRef.toString is used.

    See also

    org.apache.flink.api.java.DataSet#writeAsText(String)

  110. def writeAsText(filePath: String, writeMode: WriteMode = null): DataSink[T]

    Writes this DataSet to the specified location.

    Writes this DataSet to the specified location. This uses AnyRef.toString on each element.

    See also

    org.apache.flink.api.java.DataSet#writeAsText(String)

Deprecated Value Members

  1. def print(sinkIdentifier: String): DataSink[T]

    * Writes a DataSet to the standard output stream (stdout) with a sink identifier prefixed.

    * Writes a DataSet to the standard output stream (stdout) with a sink identifier prefixed. This uses AnyRef.toString on each element.

    sinkIdentifier

    The string to prefix the output with.

    Annotations
    @deprecated @PublicEvolving()
    Deprecated
  2. def printToErr(sinkIdentifier: String): DataSink[T]

    Writes a DataSet to the standard error stream (stderr) with a sink identifier prefixed.

    Writes a DataSet to the standard error stream (stderr) with a sink identifier prefixed. This uses AnyRef.toString on each element.

    sinkIdentifier

    The string to prefix the output with.

    Annotations
    @deprecated @PublicEvolving()
    Deprecated

Inherited from AnyRef

Inherited from Any

Ungrouped