/** * Group this table by key to yield a {@link LGroupedTable} */ default LGroupedTable<K, V> groupByKey(GroupingOptions opts) { return factory().wrap(underlying().groupByKey(opts)); }
/** * Group this table by key to yield a {@link LGroupedTable} */ default LGroupedTable<K, V> groupByKey() { return factory().wrap(underlying().groupByKey()); }
/** * Group this table by key to yield a {@link LGroupedTable} */ default LGroupedTable<K, V> groupByKey(int numReducers) { return factory().wrap(underlying().groupByKey(numReducers)); }
private static <E> PCollection<E> partition(PCollection<E> collection, int numReducers) { PType<E> type = collection.getPType(); PTableType<E, Void> tableType = Avros.tableOf(type, Avros.nulls()); PTable<E, Void> table = collection.parallelDo(new AsKeyTable<E>(), tableType); PGroupedTable<E, Void> grouped = numReducers > 0 ? table.groupByKey(numReducers) : table.groupByKey(); return grouped.ungroup().keys(); }
/** * Create a list of unique items in the input collection with their count, sorted descending by their frequency. * @param input input collection * @param <X> record type * @return global toplist */ public static <X> PTable<X, Long> globalToplist(PCollection<X> input) { return negateCounts(negateCounts(input.count()).groupByKey(1).ungroup()); }
/** * Sorts the {@code PTable} using the natural ordering of its keys in the * order specified with a client-specified number of reducers. * * @return a {@code PTable} representing the sorted collection. */ public static <K, V> PTable<K, V> sort(PTable<K, V> table, int numReducers, Order key) { Configuration conf = table.getPipeline().getConfiguration(); GroupingOptions options = buildGroupingOptions(table, conf, numReducers, key); return table.groupByKey(options).ungroup(); }
/** * Create a list of unique items in the input collection with their count, sorted descending by their frequency. * @param input input collection * @param <X> record type * @return global toplist */ public static <X> PTable<X, Long> globalToplist(PCollection<X> input) { return SPTables.negateCounts(SPTables.negateCounts(input.count()).groupByKey(1).ungroup()); } }
/** * Sorts the {@link PTable} using the natural ordering of its keys * in the order specified. * * @return a {@link PTable} representing the sorted collection. */ public static <K, V> PTable<K, V> sort(PTable<K, V> table, Order key) { PTypeFamily tf = table.getTypeFamily(); Configuration conf = table.getPipeline().getConfiguration(); GroupingOptions options = buildGroupingOptions(conf, tf, table.getKeyType(), key); return table.groupByKey(options).ungroup(); }
/** * Sorts the {@link PTable} using the natural ordering of its keys in the * order specified. * * @return a {@link PTable} representing the sorted collection. */ public static <K, V> PTable<K, V> sort(PTable<K, V> table, Order key) { PTypeFamily tf = table.getTypeFamily(); Configuration conf = table.getPipeline().getConfiguration(); GroupingOptions options = buildGroupingOptions(conf, tf, table.getKeyType(), key); return table.groupByKey(options).ungroup(); }
/** * Creates a {@code PCollection<T>} that has the same contents as its input argument but will * be written to a fixed number of output files. This is useful for map-only jobs that process * lots of input files but only write out a small amount of input per task. * * @param pc The {@code PCollection<T>} to rebalance * @param numPartitions The number of output partitions to create * @return A rebalanced {@code PCollection<T>} with the same contents as the input */ public static <T> PCollection<T> shard(PCollection<T> pc, int numPartitions) { return pc.by(new ShardFn<T>(), pc.getTypeFamily().ints()) .groupByKey(numPartitions) .ungroup() .values(); }
public static <S> PCollection<S> aggregate(PCollection<S> collect, Aggregator<S> aggregator) { PTypeFamily tf = collect.getTypeFamily(); return collect.parallelDo("Aggregate.aggregator", new MapFn<S, Pair<Boolean, S>>() { public Pair<Boolean, S> map(S input) { return Pair.of(false, input); } }, tf.tableOf(tf.booleans(), collect.getPType())) .groupByKey(1) .combineValues(aggregator) .values(); } }
/** * Returns a {@code PTable} that contains the unique elements of this collection mapped to a count * of their occurrences. */ public static <S> PTable<S, Long> count(PCollection<S> collect) { PTypeFamily tf = collect.getTypeFamily(); return collect.parallelDo("Aggregate.count", new MapFn<S, Pair<S, Long>>() { public Pair<S, Long> map(S input) { return Pair.of(input, 1L); } }, tf.tableOf(collect.getPType(), tf.longs())).groupByKey() .combineValues(Aggregators.SUM_LONGS()); }
private static <T> PTable<String, BloomFilter> createFilterTable(PCollection<T> collection, BloomFilterFn<T> filterFn) { PTypeFamily tf = collection.getTypeFamily(); PTable<String, BloomFilter> table = collection.parallelDo(filterFn, tf.tableOf(tf.strings(), Writables.writables(BloomFilter.class))); return table.groupByKey(1).combineValues(new BloomFilterAggregator()); }
/** * Returns a {@code PTable} that contains the unique elements of this * collection mapped to a count of their occurrences. */ public static <S> PTable<S, Long> count(PCollection<S> collect) { PTypeFamily tf = collect.getTypeFamily(); return collect.parallelDo("Aggregate.count", new MapFn<S, Pair<S, Long>>() { public Pair<S, Long> map(S input) { return Pair.of(input, 1L); } }, tf.tableOf(collect.getPType(), tf.longs())) .groupByKey() .combineValues(CombineFn.<S> SUM_LONGS()); }
public static <K, V> PTable<K, Collection<V>> collectValues(PTable<K, V> collect) { PTypeFamily tf = collect.getTypeFamily(); final PType<V> valueType = collect.getValueType(); return collect.groupByKey().parallelDo("collect", new MapValuesFn<K, Iterable<V>, Collection<V>>() { public Collection<V> map(Iterable<V> values) { List<V> collected = Lists.newArrayList(); for (V value : values) { collected.add(valueType.getDetachedValue(value)); } return collected; } }, tf.tableOf(collect.getKeyType(), tf.collections(collect.getValueType()))); } }
/** * Returns a {@code PTable} that contains the unique elements of this collection mapped to a count * of their occurrences. */ public static <S> PTable<S, Long> count(PCollection<S> collect, int numPartitions) { PTypeFamily tf = collect.getTypeFamily(); return collect.parallelDo("Aggregate.count", new MapFn<S, Pair<S, Long>>() { public Pair<S, Long> map(S input) { return Pair.of(input, 1L); } }, tf.tableOf(collect.getPType(), tf.longs())) .groupByKey(numPartitions) .combineValues(Aggregators.SUM_LONGS()); }
/** * Sorts the {@code PCollection} of {@link TupleN}s using the specified column * ordering and a client-specified number of reducers. * * @return a {@code PCollection} representing the sorted collection. */ public static <T extends Tuple> PCollection<T> sortTuples(PCollection<T> collection, int numReducers, ColumnOrder... columnOrders) { PType<T> pType = collection.getPType(); SortFns.KeyExtraction<T> ke = new SortFns.KeyExtraction<T>(pType, columnOrders); PTable<Object, T> pt = collection.by(ke.getByFn(), ke.getKeyType()); Configuration conf = collection.getPipeline().getConfiguration(); GroupingOptions options = buildGroupingOptions(pt, conf, numReducers, columnOrders); return pt.groupByKey(options).ungroup().values(); }
public static <K, V> PTable<K, V> top(PTable<K, V> ptable, int limit, boolean maximize) { PTypeFamily ptf = ptable.getTypeFamily(); PTableType<K, V> base = ptable.getPTableType(); PType<Pair<K, V>> pairType = ptf.pairs(base.getKeyType(), base.getValueType()); PTableType<Integer, Pair<K, V>> inter = ptf.tableOf(ptf.ints(), pairType); return ptable.parallelDo("top" + limit + "map", new TopKFn<K, V>(limit, maximize), inter) .groupByKey(1).combineValues(new TopKCombineFn<K, V>(limit, maximize)) .parallelDo("top" + limit + "reduce", new DoFn<Pair<Integer, Pair<K, V>>, Pair<K, V>>() { public void process(Pair<Integer, Pair<K, V>> input, Emitter<Pair<K, V>> emitter) { emitter.emit(input.second()); } }, base); }
public static <K, V> PTable<K, V> top(PTable<K, V> ptable, int limit, boolean maximize) { PTypeFamily ptf = ptable.getTypeFamily(); PTableType<K, V> base = ptable.getPTableType(); PType<Pair<K, V>> pairType = ptf.pairs(base.getKeyType(), base.getValueType()); PTableType<Integer, Pair<K, V>> inter = ptf.tableOf(ptf.ints(), pairType); return ptable.parallelDo("top" + limit + "map", new TopKFn<K, V>(limit, maximize), inter) .groupByKey(1) .combineValues(new TopKCombineFn<K, V>(limit, maximize)) .parallelDo("top" + limit + "reduce", new DoFn<Pair<Integer, Pair<K, V>>, Pair<K, V>>() { public void process(Pair<Integer, Pair<K, V>> input, Emitter<Pair<K, V>> emitter) { emitter.emit(input.second()); } }, base); }
private static <K, V1, V2> PGroupedTable<Pair<K, V1>, Pair<V1, V2>> prepare( PTable<K, Pair<V1, V2>> input) { PTypeFamily ptf = input.getTypeFamily(); PType<Pair<V1, V2>> valueType = input.getValueType(); PTableType<Pair<K, V1>, Pair<V1, V2>> inter = ptf.tableOf( ptf.pairs(input.getKeyType(), valueType.getSubTypes().get(0)), valueType); PTableType<K, Collection<Pair<V1, V2>>> out = ptf.tableOf(input.getKeyType(), ptf.collections(input.getValueType())); return input.parallelDo("SecondarySort.format", new SSFormatFn<K, V1, V2>(), inter) .groupByKey( GroupingOptions.builder() .groupingComparatorClass(JoinUtils.getGroupingComparator(ptf)) .partitionerClass(JoinUtils.getPartitionerClass(ptf)) .build()); }