/** * Applies {@code ApproximateUnique(sampleSize)} verifying that the estimation error falls within * the maximum allowed error of {@code 2/sqrt(sampleSize)}. */ private void runApproximateUniquePipeline(int sampleSize) { Pipeline p = TestPipeline.create(); PCollection<String> collection = readPCollection(p); final PCollectionView<Long> exact = collection .apply(RemoveDuplicates.<String>create()) .apply(Combine.globally(new CountElements<String>())) .apply(View.<Long>asSingleton()); PCollection<Long> approximate = collection.apply(ApproximateUnique.<String>globally(sampleSize)); PCollection<KV<Long, Long>> approximateAndExact = approximate.apply( ParDo.of( new DoFn<Long, KV<Long, Long>>() { @Override public void processElement(ProcessContext c) { c.output(KV.of(c.element(), c.sideInput(exact))); } }) .withSideInputs(exact)); DataflowAssert.that(approximateAndExact).satisfies(new VerifyEstimatePerKeyFn(sampleSize)); p.run(); }
@Override public PCollection<T> apply(PCollection<T> in) { WithKeys<IdT, T> withKeys = WithKeys.of(fn); if (representativeType != null) { withKeys = withKeys.withKeyType(representativeType); } return in.apply(withKeys) .apply( Combine.<IdT, T, T>perKey( new Combine.BinaryCombineFn<T>() { @Override public T apply(T left, T right) { return left; } })) .apply(Values.<T>create()); }
@Override public PCollection<T> apply(PCollection<T> in) { return in.apply( ParDo.named("CreateIndex") .of( new DoFn<T, KV<T, Void>>() { @Override public void processElement(ProcessContext c) { c.output(KV.of(c.element(), (Void) null)); } })) .apply( Combine.<T, Void>perKey( new SerializableFunction<Iterable<Void>, Void>() { @Override public Void apply(Iterable<Void> iter) { return null; // ignore input } })) .apply(Keys.<T>create()); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<KV<K, N>>} and returns a * {@code PCollection<KV<K, Double>>} that contains an output element mapping each distinct key in * the input {@code PCollection} to the mean of the values associated with that key in the input * {@code PCollection}. * * <p>See {@link Combine.PerKey} for how this affects timestamps and bucketing. * * @param <K> the type of the keys * @param <NumT> the type of the {@code Number}s being combined */ public static <K, NumT extends Number> Combine.PerKey<K, NumT, Double> perKey() { return Combine.<K, NumT, Double>perKey(new MeanFn<>()).named("Mean.PerKey"); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<NumT>} and returns a {@code * PCollection<Double>} whose contents is the mean of the input {@code PCollection}'s elements, or * {@code 0} if there are no elements. * * @param <NumT> the type of the {@code Number}s being combined */ public static <NumT extends Number> Combine.Globally<NumT, Double> globally() { return Combine.<NumT, Double>globally(new MeanFn<>()).named("Mean.Globally"); }
@Override public PCollectionView<List<T>> apply(PCollection<T> input) { return input.apply(Combine.globally(new Concatenate<T>()).asSingletonView()); }
@Override public String getPostprocessCode() { return original.getPostprocessCode(); }
@Override public void combine(String type, String target, String prefix) { original.combine(type, target, prefix); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<T>} and returns a {@code * PCollection<List<T>>} with a single element containing the largest {@code count} elements of * the input {@code PCollection<T>}, in decreasing order, sorted using the given {@code * Comparator<T>}. The {@code Comparator<T>} must also be {@code Serializable}. * * <p>If {@code count} {@code <} the number of elements in the input {@code PCollection}, then all * the elements of the input {@code PCollection} will be in the resulting {@code List}, albeit in * sorted order. * * <p>All the elements of the result's {@code List} must fit into the memory of a single machine. * * <p>Example of use: * * <pre>{@code * PCollection<Student> students = ...; * PCollection<List<Student>> top10Students = * students.apply(Top.of(10, new CompareStudentsByAvgGrade())); * } * </pre> * * <p>By default, the {@code Coder} of the output {@code PCollection} is a {@code ListCoder} of * the {@code Coder} of the elements of the input {@code PCollection}. * * <p>If the input {@code PCollection} is windowed into {@link GlobalWindows}, an empty {@code * List<T>} in the {@link GlobalWindow} will be output if the input {@code PCollection} is empty. * To use this with inputs with other windowing, either {@link Combine.Globally#withoutDefaults * withoutDefaults} or {@link Combine.Globally#asSingletonView asSingletonView} must be called. * * <p>See also {@link #smallest} and {@link #largest}, which sort {@code Comparable} elements * using their natural ordering. * * <p>See also {@link #perKey}, {@link #smallestPerKey}, and {@link #largestPerKey}, which take a * {@code PCollection} of {@code KV}s and return the top values associated with each key. */ public static <T, ComparatorT extends Comparator<T> & Serializable> Combine.Globally<T, List<T>> of(int count, ComparatorT compareFn) { return Combine.globally(new TopCombineFn<>(count, compareFn)).named("Top.Globally"); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<KV<K, V>>} and returns a * {@code PCollection<KV<K, List<V>>>} that contains an output element mapping each distinct key * in the input {@code PCollection} to the largest {@code count} values associated with that key * in the input {@code PCollection<KV<K, V>>}, in decreasing order, sorted according to their * natural order. * * <p>If there are fewer than {@code count} values associated with a particular key, then all * those values will be in the result mapping for that key, albeit in sorted order. * * <p>All the values associated with a single key must fit into the memory of a single machine, * but there can be many more {@code KV}s in the resulting {@code PCollection} than can fit into * the memory of a single machine. * * <p>Example of use: * * <pre>{@code * PCollection<KV<String, Integer>> keyedValues = ...; * PCollection<KV<String, List<Integer>>> largest10ValuesPerKey = * keyedValues.apply(Top.largestPerKey(10)); * } * </pre> * * <p>By default, the {@code Coder} of the keys of the output {@code PCollection} is the same as * that of the keys of the input {@code PCollection}, and the {@code Coder} of the values of the * output {@code PCollection} is a {@code ListCoder} of the {@code Coder} of the values of the * input {@code PCollection}. * * <p>See also {@link #smallestPerKey}. * * <p>See also {@link #perKey}, which sorts values using a user-specified {@code Comparator} * function. * * <p>See also {@link #of}, {@link #smallest}, and {@link #largest}, which take a {@code * PCollection} and return the top elements. */ public static <K, V extends Comparable<V>> PerKey<K, V, List<V>> largestPerKey(int count) { return Combine.perKey(new TopCombineFn<>(count, new Largest<V>()).<K>asKeyedFn()) .named("Largest.PerKey"); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<KV<K, V>>} and returns a * {@code PCollection<KV<K, List<V>>>} that contains an output element mapping each distinct key * in the input {@code PCollection} to the smallest {@code count} values associated with that key * in the input {@code PCollection<KV<K, V>>}, in increasing order, sorted according to their * natural order. * * <p>If there are fewer than {@code count} values associated with a particular key, then all * those values will be in the result mapping for that key, albeit in sorted order. * * <p>All the values associated with a single key must fit into the memory of a single machine, * but there can be many more {@code KV}s in the resulting {@code PCollection} than can fit into * the memory of a single machine. * * <p>Example of use: * * <pre>{@code * PCollection<KV<String, Integer>> keyedValues = ...; * PCollection<KV<String, List<Integer>>> smallest10ValuesPerKey = * keyedValues.apply(Top.smallestPerKey(10)); * } * </pre> * * <p>By default, the {@code Coder} of the keys of the output {@code PCollection} is the same as * that of the keys of the input {@code PCollection}, and the {@code Coder} of the values of the * output {@code PCollection} is a {@code ListCoder} of the {@code Coder} of the values of the * input {@code PCollection}. * * <p>See also {@link #largestPerKey}. * * <p>See also {@link #perKey}, which sorts values using a user-specified {@code Comparator} * function. * * <p>See also {@link #of}, {@link #smallest}, and {@link #largest}, which take a {@code * PCollection} and return the top elements. */ public static <K, V extends Comparable<V>> PTransform<PCollection<KV<K, V>>, PCollection<KV<K, List<V>>>> smallestPerKey(int count) { return Combine.perKey(new TopCombineFn<>(count, new Smallest<V>()).<K>asKeyedFn()) .named("Smallest.PerKey"); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<KV<K, V>>} and returns a * {@code PCollection<KV<K, List<V>>>} that contains an output element mapping each distinct key * in the input {@code PCollection} to the largest {@code count} values associated with that key * in the input {@code PCollection<KV<K, V>>}, in decreasing order, sorted using the given {@code * Comparator<V>}. The {@code Comparator<V>} must also be {@code Serializable}. * * <p>If there are fewer than {@code count} values associated with a particular key, then all * those values will be in the result mapping for that key, albeit in sorted order. * * <p>All the values associated with a single key must fit into the memory of a single machine, * but there can be many more {@code KV}s in the resulting {@code PCollection} than can fit into * the memory of a single machine. * * <p>Example of use: * * <pre>{@code * PCollection<KV<School, Student>> studentsBySchool = ...; * PCollection<KV<School, List<Student>>> top10StudentsBySchool = * studentsBySchool.apply( * Top.perKey(10, new CompareStudentsByAvgGrade())); * } * </pre> * * <p>By default, the {@code Coder} of the keys of the output {@code PCollection} is the same as * that of the keys of the input {@code PCollection}, and the {@code Coder} of the values of the * output {@code PCollection} is a {@code ListCoder} of the {@code Coder} of the values of the * input {@code PCollection}. * * <p>See also {@link #smallestPerKey} and {@link #largestPerKey}, which sort {@code * Comparable<V>} values using their natural ordering. * * <p>See also {@link #of}, {@link #smallest}, and {@link #largest}, which take a {@code * PCollection} and return the top elements. */ public static <K, V, ComparatorT extends Comparator<V> & Serializable> PTransform<PCollection<KV<K, V>>, PCollection<KV<K, List<V>>>> perKey( int count, ComparatorT compareFn) { return Combine.perKey(new TopCombineFn<>(count, compareFn).<K>asKeyedFn()).named("Top.PerKey"); }
/** * Returns a {@code PTransform} that takes an input {@code PCollection<T>} and returns a {@code * PCollection<List<T>>} with a single element containing the largest {@code count} elements of * the input {@code PCollection<T>}, in decreasing order, sorted according to their natural order. * * <p>If {@code count} {@code <} the number of elements in the input {@code PCollection}, then all * the elements of the input {@code PCollection} will be in the resulting {@code PCollection}'s * {@code List}, albeit in sorted order. * * <p>All the elements of the result's {@code List} must fit into the memory of a single machine. * * <p>Example of use: * * <pre>{@code * PCollection<Integer> values = ...; * PCollection<List<Integer>> largest10Values = values.apply(Top.largest(10)); * } * </pre> * * <p>By default, the {@code Coder} of the output {@code PCollection} is a {@code ListCoder} of * the {@code Coder} of the elements of the input {@code PCollection}. * * <p>If the input {@code PCollection} is windowed into {@link GlobalWindows}, an empty {@code * List<T>} in the {@link GlobalWindow} will be output if the input {@code PCollection} is empty. * To use this with inputs with other windowing, either {@link Combine.Globally#withoutDefaults * withoutDefaults} or {@link Combine.Globally#asSingletonView asSingletonView} must be called. * * <p>See also {@link #smallest}. * * <p>See also {@link #of}, which sorts using a user-specified {@code Comparator} function. * * <p>See also {@link #perKey}, {@link #smallestPerKey}, and {@link #largestPerKey}, which take a * {@code PCollection} of {@code KV}s and return the top values associated with each key. */ public static <T extends Comparable<T>> Combine.Globally<T, List<T>> largest(int count) { return Combine.globally(new TopCombineFn<>(count, new Largest<T>())).named("Largest.Globally"); }