Class Top
PTransforms for finding the largest (or smallest) set of elements in a
PCollection, or the largest (or smallest) set of values associated with each key in a
PCollection of KVs.-
Nested Class Summary
Nested ClassesModifier and TypeClassDescriptionstatic classTop.Largest<T extends Comparable<? super T>>Deprecated.static classTop.Natural<T extends Comparable<? super T>>ASerializableComparatorthat that uses the compared elements' natural ordering.static classTop.Reversed<T extends Comparable<? super T>>SerializableComparatorthat that uses the reverse of the compared elements' natural ordering.static classTop.Smallest<T extends Comparable<? super T>>Deprecated.useTop.Reversedinsteadstatic classTop.TopCombineFn<T,ComparatorT extends Comparator<T> & Serializable> CombineFnforToptransforms that combines a bunch ofTs into a singlecount-longList<T>, usingcompareFnto choose the largestTs. -
Method Summary
Modifier and TypeMethodDescriptionstatic <T extends Comparable<T>>
Combine.Globally<T, List<T>> largest(int count) Returns aPTransformthat takes an inputPCollection<T>and returns aPCollection<List<T>>with a single element containing the largestcountelements of the inputPCollection<T>, in decreasing order, sorted according to their natural order.static Top.TopCombineFn<Double, Top.Natural<Double>> largestDoublesFn(int count) Returns aTop.TopCombineFnthat aggregates the largest count double values.static <T extends Comparable<T>>
Top.TopCombineFn<T, Top.Natural<T>> largestFn(int count) Returns aTop.TopCombineFnthat aggregates the largest count values.static Top.TopCombineFn<Integer, Top.Natural<Integer>> largestIntsFn(int count) Returns aTop.TopCombineFnthat aggregates the largest count int values.static Top.TopCombineFn<Long, Top.Natural<Long>> largestLongsFn(int count) Returns aTop.TopCombineFnthat aggregates the largest count long values.static <K,V extends Comparable<V>>
Combine.PerKey<K, V, List<V>> largestPerKey(int count) Returns aPTransformthat takes an inputPCollection<KV<K, V>>and returns aPCollection<KV<K, List<V>>>that contains an output element mapping each distinct key in the inputPCollectionto the largestcountvalues associated with that key in the inputPCollection<KV<K, V>>, in decreasing order, sorted according to their natural order.static <T,ComparatorT extends Comparator<T> & Serializable>
Combine.Globally<T, List<T>> of(int count, ComparatorT compareFn) Returns aPTransformthat takes an inputPCollection<T>and returns aPCollection<List<T>>with a single element containing the largestcountelements of the inputPCollection<T>, in decreasing order, sorted using the givenComparator<T>.static <K,V, ComparatorT extends Comparator<V> & Serializable>
PTransform<PCollection<KV<K, V>>, PCollection<KV<K, List<V>>>> perKey(int count, ComparatorT compareFn) Returns aPTransformthat takes an inputPCollection<KV<K, V>>and returns aPCollection<KV<K, List<V>>>that contains an output element mapping each distinct key in the inputPCollectionto the largestcountvalues associated with that key in the inputPCollection<KV<K, V>>, in decreasing order, sorted using the givenComparator<V>.static <T extends Comparable<T>>
Combine.Globally<T, List<T>> smallest(int count) Returns aPTransformthat takes an inputPCollection<T>and returns aPCollection<List<T>>with a single element containing the smallestcountelements of the inputPCollection<T>, in increasing order, sorted according to their natural order.static Top.TopCombineFn<Double, Top.Reversed<Double>> smallestDoublesFn(int count) Returns aTop.TopCombineFnthat aggregates the smallest count double values.static <T extends Comparable<T>>
Top.TopCombineFn<T, Top.Reversed<T>> smallestFn(int count) Returns aTop.TopCombineFnthat aggregates the smallest count values.static Top.TopCombineFn<Integer, Top.Reversed<Integer>> smallestIntsFn(int count) Returns aTop.TopCombineFnthat aggregates the smallest count int values.static Top.TopCombineFn<Long, Top.Reversed<Long>> smallestLongsFn(int count) Returns aTop.TopCombineFnthat aggregates the smallest count long values.static <K,V extends Comparable<V>>
PTransform<PCollection<KV<K, V>>, PCollection<KV<K, List<V>>>> smallestPerKey(int count) Returns aPTransformthat takes an inputPCollection<KV<K, V>>and returns aPCollection<KV<K, List<V>>>that contains an output element mapping each distinct key in the inputPCollectionto the smallestcountvalues associated with that key in the inputPCollection<KV<K, V>>, in increasing order, sorted according to their natural order.
-
Method Details
-
of
public static <T,ComparatorT extends Comparator<T> & Serializable> Combine.Globally<T,List<T>> of(int count, ComparatorT compareFn) Returns aPTransformthat takes an inputPCollection<T>and returns aPCollection<List<T>>with a single element containing the largestcountelements of the inputPCollection<T>, in decreasing order, sorted using the givenComparator<T>. TheComparator<T>must also beSerializable.If
count>the number of elements in the inputPCollection, then all the elements of the inputPCollectionwill be in the resultingList, albeit in sorted order.All the elements of the result's
Listmust fit into the memory of a single machine.Example of use:
PCollection<Student> students = ...; PCollection<List<Student>> top10Students = students.apply(Top.of(10, new CompareStudentsByAvgGrade()));By default, the
Coderof the outputPCollectionis aListCoderof theCoderof the elements of the inputPCollection.If the input
PCollectionis windowed intoGlobalWindows, an emptyList<T>in theGlobalWindowwill be output if the inputPCollectionis empty. To use this with inputs with other windowing, eitherwithoutDefaultsorasSingletonViewmust be called.See also
smallest(int)andlargest(int), which sortComparableelements using their natural ordering.See also
perKey(int, ComparatorT),smallestPerKey(int), andlargestPerKey(int), which take aPCollectionofKVs and return the top values associated with each key. -
smallest
Returns aPTransformthat takes an inputPCollection<T>and returns aPCollection<List<T>>with a single element containing the smallestcountelements of the inputPCollection<T>, in increasing order, sorted according to their natural order.If
count>the number of elements in the inputPCollection, then all the elements of the inputPCollectionwill be in the resultingPCollection'sList, albeit in sorted order.All the elements of the result
Listmust fit into the memory of a single machine.Example of use:
PCollection<Integer> values = ...; PCollection<List<Integer>> smallest10Values = values.apply(Top.smallest(10));By default, the
Coderof the outputPCollectionis aListCoderof theCoderof the elements of the inputPCollection.If the input
PCollectionis windowed intoGlobalWindows, an emptyList<T>in theGlobalWindowwill be output if the inputPCollectionis empty. To use this with inputs with other windowing, eitherwithoutDefaultsorasSingletonViewmust be called.See also
largest(int).See also
of(int, ComparatorT), which sorts using a user-specifiedComparatorfunction.See also
perKey(int, ComparatorT),smallestPerKey(int), andlargestPerKey(int), which take aPCollectionofKVs and return the top values associated with each key. -
largest
Returns aPTransformthat takes an inputPCollection<T>and returns aPCollection<List<T>>with a single element containing the largestcountelements of the inputPCollection<T>, in decreasing order, sorted according to their natural order.If
count>the number of elements in the inputPCollection, then all the elements of the inputPCollectionwill be in the resultingPCollection'sList, albeit in sorted order.All the elements of the result's
Listmust fit into the memory of a single machine.Example of use:
PCollection<Integer> values = ...; PCollection<List<Integer>> largest10Values = values.apply(Top.largest(10));By default, the
Coderof the outputPCollectionis aListCoderof theCoderof the elements of the inputPCollection.If the input
PCollectionis windowed intoGlobalWindows, an emptyList<T>in theGlobalWindowwill be output if the inputPCollectionis empty. To use this with inputs with other windowing, eitherwithoutDefaultsorasSingletonViewmust be called.See also
smallest(int).See also
of(int, ComparatorT), which sorts using a user-specifiedComparatorfunction.See also
perKey(int, ComparatorT),smallestPerKey(int), andlargestPerKey(int), which take aPCollectionofKVs and return the top values associated with each key. -
largestFn
Returns aTop.TopCombineFnthat aggregates the largest count values. -
largestLongsFn
Returns aTop.TopCombineFnthat aggregates the largest count long values. -
largestIntsFn
Returns aTop.TopCombineFnthat aggregates the largest count int values. -
largestDoublesFn
Returns aTop.TopCombineFnthat aggregates the largest count double values. -
smallestFn
Returns aTop.TopCombineFnthat aggregates the smallest count values. -
smallestLongsFn
Returns aTop.TopCombineFnthat aggregates the smallest count long values. -
smallestIntsFn
Returns aTop.TopCombineFnthat aggregates the smallest count int values. -
smallestDoublesFn
Returns aTop.TopCombineFnthat aggregates the smallest count double values. -
perKey
public static <K,V, PTransform<PCollection<KV<K,ComparatorT extends Comparator<V> & Serializable> V>>, perKeyPCollection<KV<K, List<V>>>> (int count, ComparatorT compareFn) Returns aPTransformthat takes an inputPCollection<KV<K, V>>and returns aPCollection<KV<K, List<V>>>that contains an output element mapping each distinct key in the inputPCollectionto the largestcountvalues associated with that key in the inputPCollection<KV<K, V>>, in decreasing order, sorted using the givenComparator<V>. TheComparator<V>must also beSerializable.If there are fewer than
countvalues associated with a particular key, then all those values will be in the result mapping for that key, albeit in sorted order.All the values associated with a single key must fit into the memory of a single machine, but there can be many more
KVs in the resultingPCollectionthan can fit into the memory of a single machine.Example of use:
PCollection<KV<School, Student>> studentsBySchool = ...; PCollection<KV<School, List<Student>>> top10StudentsBySchool = studentsBySchool.apply( Top.perKey(10, new CompareStudentsByAvgGrade()));By default, the
Coderof the keys of the outputPCollectionis the same as that of the keys of the inputPCollection, and theCoderof the values of the outputPCollectionis aListCoderof theCoderof the values of the inputPCollection.See also
smallestPerKey(int)andlargestPerKey(int), which sortComparable<V>values using their natural ordering.See also
of(int, ComparatorT),smallest(int), andlargest(int), which take aPCollectionand return the top elements. -
smallestPerKey
public static <K,V extends Comparable<V>> PTransform<PCollection<KV<K,V>>, smallestPerKeyPCollection<KV<K, List<V>>>> (int count) Returns aPTransformthat takes an inputPCollection<KV<K, V>>and returns aPCollection<KV<K, List<V>>>that contains an output element mapping each distinct key in the inputPCollectionto the smallestcountvalues associated with that key in the inputPCollection<KV<K, V>>, in increasing order, sorted according to their natural order.If there are fewer than
countvalues associated with a particular key, then all those values will be in the result mapping for that key, albeit in sorted order.All the values associated with a single key must fit into the memory of a single machine, but there can be many more
KVs in the resultingPCollectionthan can fit into the memory of a single machine.Example of use:
PCollection<KV<String, Integer>> keyedValues = ...; PCollection<KV<String, List<Integer>>> smallest10ValuesPerKey = keyedValues.apply(Top.smallestPerKey(10));By default, the
Coderof the keys of the outputPCollectionis the same as that of the keys of the inputPCollection, and theCoderof the values of the outputPCollectionis aListCoderof theCoderof the values of the inputPCollection.See also
largestPerKey(int).See also
perKey(int, ComparatorT), which sorts values using a user-specifiedComparatorfunction.See also
of(int, ComparatorT),smallest(int), andlargest(int), which take aPCollectionand return the top elements. -
largestPerKey
Returns aPTransformthat takes an inputPCollection<KV<K, V>>and returns aPCollection<KV<K, List<V>>>that contains an output element mapping each distinct key in the inputPCollectionto the largestcountvalues associated with that key in the inputPCollection<KV<K, V>>, in decreasing order, sorted according to their natural order.If there are fewer than
countvalues associated with a particular key, then all those values will be in the result mapping for that key, albeit in sorted order.All the values associated with a single key must fit into the memory of a single machine, but there can be many more
KVs in the resultingPCollectionthan can fit into the memory of a single machine.Example of use:
PCollection<KV<String, Integer>> keyedValues = ...; PCollection<KV<String, List<Integer>>> largest10ValuesPerKey = keyedValues.apply(Top.largestPerKey(10));By default, the
Coderof the keys of the outputPCollectionis the same as that of the keys of the inputPCollection, and theCoderof the values of the outputPCollectionis aListCoderof theCoderof the values of the inputPCollection.See also
smallestPerKey(int).See also
perKey(int, ComparatorT), which sorts values using a user-specifiedComparatorfunction.See also
of(int, ComparatorT),smallest(int), andlargest(int), which take aPCollectionand return the top elements.
-
Top.Naturalinstead