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()))); } }
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); }
int numReducers, PTable<K, ?> first, PTable<K, ?>... rest) { PTypeFamily ptf = first.getTypeFamily(); PType[] ptypes = new PType[1 + rest.length]; ptypes[0] = first.getValueType(); for (int i = 0; i < rest.length; i++) { ptypes[i + 1] = rest[i].getValueType(); PTable<K, Union> firstInter = first.mapValues("coGroupTag1", new CogroupFn(0), itype); PTable<K, Union>[] inter = new PTable[rest.length]; for (int i = 0; i < rest.length; i++) { inter[i] = rest[i].mapValues("coGroupTag" + (i + 2), new CogroupFn(i + 1), itype); PTable<K, Union> union = firstInter.union(inter); PGroupedTable<K, Union> grouped; if (numReducers > 0) { grouped = union.groupByKey(numReducers); } else { grouped = union.groupByKey();
public static <K, V> PCollection<K> keys(PTable<K, V> ptable) { return ptable.parallelDo("PTables.keys", new DoFn<Pair<K, V>, K>() { @Override public void process(Pair<K, V> input, Emitter<K> emitter) { emitter.emit(input.first()); } }, ptable.getKeyType()); }
public static <K, V> PCollection<V> values(PTable<K, V> ptable) { return ptable.parallelDo("PTables.values", new DoFn<Pair<K, V>, V>() { @Override public void process(Pair<K, V> input, Emitter<V> emitter) { emitter.emit(input.second()); } }, ptable.getValueType()); }
/** * When creating toplists, it is often required to sort by count descending. As some sort operations don't support * order (such as SecondarySort), this method will negate counts so that a natural-ordered sort will produce a * descending order. * @param table PTable to process * @param <K> key type * @return PTable of the same format with the value negated */ public static <K> PTable<K, Long> negateCounts(PTable<K, Long> table) { return table.parallelDo(new MapFn<Pair<K, Long>, Pair<K, Long>>() { @Override public Pair<K, Long> map(Pair<K, Long> input) { return Pair.of(input.first(), -input.second()); } }, table.getPTableType()); } }
@Override public PTable<K, Pair<U, V>> join(PTable<K, U> left, PTable<K, V> right, JoinType joinType) { PType<BloomFilter> bloomFilterType = getBloomFilterType(left.getTypeFamily()); PCollection<BloomFilter> bloomFilters = left.keys().parallelDo( "Create bloom filters", new CreateBloomFilterFn<>(vectorSize, nbHash, left.getKeyType()), bloomFilterType); bloomData, vectorSize, nbHash, left.getKeyType()); right = right.parallelDo( "disable deep copy", new DeepCopyDisablerFn<Pair<K, V>>(), right.getPTableType()); PTable<K, V> filteredRightSide = right.parallelDo( "Filter right-side with BloomFilters", filterKeysFn, right.getPTableType(), options); return leftJoinedWithFilteredRight.union( right .parallelDo( "Negatively filter right-side with BloomFilters", FilterFns.not(filterKeysFn), right.getPTableType(), options) .mapValues( "Right outer join: attach null as left-value", new NullKeyFn<U, V>(), leftJoinedWithFilteredRight.getValueType()));
/** * 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(); }
/** * Group this table by key to yield a {@link LGroupedTable} */ default LGroupedTable<K, V> groupByKey() { return factory().wrap(underlying().groupByKey()); }
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(); }
/** * Calculate a set of quantiles for each key in a numerically-valued table. * * Quantiles are calculated on a per-key basis by counting, joining and sorting. This is highly scalable, but takes * 2 more map-reduce cycles than if you can guarantee that the value set will fit into memory. Use inMemory * if you have less than the order of 10M values per key. * * The quantile definition that we use here is the "nearest rank" defined here: * http://en.wikipedia.org/wiki/Percentile#Definition * * @param table numerically-valued PTable * @param p1 First quantile (in the range 0.0 - 1.0) * @param pn More quantiles (in the range 0.0 - 1.0) * @param <K> Key type of the table * @param <V> Value type of the table (must extends java.lang.Number) * @return PTable of each key with a collection of pairs of the quantile provided and it's result. */ public static <K, V extends Number> PTable<K, Result<V>> distributed(PTable<K, V> table, double p1, double... pn) { final List<Double> quantileList = createListFromVarargs(p1, pn); PTypeFamily ptf = table.getTypeFamily(); PTable<K, Long> totalCounts = table.keys().count(); PTable<K, Pair<Long, V>> countValuePairs = totalCounts.join(table); PTable<K, Pair<V, Long>> valueCountPairs = countValuePairs.mapValues(new SwapPairComponents<Long, V>(), ptf.pairs(table.getValueType(), ptf.longs())); return SecondarySort.sortAndApply( valueCountPairs, new DistributedQuantiles<K, V>(quantileList), ptf.tableOf(table.getKeyType(), Result.pType(table.getValueType()))); }
private PTable<K, Pair<U,V>> joinInternal(PTable<K, U> left, PTable<K, V> right, boolean includeUnmatchedLeftValues) { PTypeFamily tf = left.getTypeFamily(); ReadableData<Pair<K, V>> rightReadable = right.asReadable(materialize); MapsideJoinDoFn<K, U, V> mapJoinDoFn = new MapsideJoinDoFn<K, U, V>( rightReadable, right.getPTableType(), includeUnmatchedLeftValues); ParallelDoOptions options = ParallelDoOptions.builder() .sourceTargets(rightReadable.getSourceTargets()) .build(); return left.parallelDo("mapjoin", mapJoinDoFn, tf.tableOf(left.getKeyType(), tf.pairs(left.getValueType(), right.getValueType())), options); }
/** * 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(); }
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().mapValues("collect", new MapFn<Iterable<V>, Collection<V>>() { @Override public void initialize() { valueType.initialize(getConfiguration()); } public Collection<V> map(Iterable<V> values) { List<V> collected = Lists.newArrayList(); for (V value : values) { collected.add(valueType.getDetachedValue(value)); } return collected; } }, tf.collections(collect.getValueType())); }
public static <K, U, V> PTable<K, Pair<U, V>> join(PTable<K, U> left, PTable<K, V> right, JoinFn<K, U, V> joinFn) { PTypeFamily ptf = left.getTypeFamily(); PGroupedTable<Pair<K, Integer>, Pair<U, V>> grouped = preJoin(left, right); PTableType<K, Pair<U, V>> ret = ptf .tableOf(left.getKeyType(), ptf.pairs(left.getValueType(), right.getValueType())); return grouped.parallelDo(joinFn.getJoinType() + grouped.getName(), joinFn, ret); }
@Override public PTable<K, Pair<U, V>> join(PTable<K, U> left, PTable<K, V> right, JoinType joinType) { switch (joinType) { case INNER_JOIN: return join(left, right, new InnerJoinFn<K, U, V>(left.getKeyType(), left.getValueType())); case LEFT_OUTER_JOIN: return join(left, right, new LeftOuterJoinFn<K, U, V>(left.getKeyType(), left.getValueType())); case RIGHT_OUTER_JOIN: return join(left, right, new RightOuterJoinFn<K, U, V>(left.getKeyType(), left.getValueType())); case FULL_OUTER_JOIN: return join(left, right, new FullOuterJoinFn<K, U, V>(left.getKeyType(), left.getValueType())); default: throw new UnsupportedOperationException("Join type " + joinType + " is not supported"); } }
/** * Maps a {@code PTable<K1, V>} to a {@code PTable<K2, V>} using the given {@code MapFn<K1, K2>} on * the keys of the {@code PTable}. * * @param name The name of the transform * @param ptable The {@code PTable} to be mapped * @param mapFn The mapping function * @param ptype The PType for the returned keys * @return A new {@code PTable<K2, V>} instance */ public static <K1, K2, V> PTable<K2, V> mapKeys(String name, PTable<K1, V> ptable, MapFn<K1, K2> mapFn, PType<K2> ptype) { PTypeFamily ptf = ptable.getTypeFamily(); return ptable.parallelDo(name, new PairMapFn<K1, V, K2, V>(mapFn, IdentityFn.<V>getInstance()), ptf.tableOf(ptype, ptable.getValueType())); }
/** * Maps a {@code PTable<K, U>} to a {@code PTable<K, V>} using the given {@code MapFn<U, V>} on * the values of the {@code PTable}. * * @param name The name of the transform * @param ptable The {@code PTable} to be mapped * @param mapFn The mapping function * @param ptype The PType for the returned values * @return A new {@code PTable<K, V>} instance */ public static <K, U, V> PTable<K, V> mapValues(String name, PTable<K, U> ptable, MapFn<U, V> mapFn, PType<V> ptype) { PTypeFamily ptf = ptable.getTypeFamily(); return ptable.parallelDo(name, new PairMapFn<K, U, K, V>(IdentityFn.<K>getInstance(), mapFn), ptf.tableOf(ptable.getKeyType(), ptype)); }
/** * 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(); }
public static <K1, V1, K2 extends Writable, V2 extends Writable> PTable<K2, V2> map( PTable<K1, V1> input, Class<? extends Mapper<K1, V1, K2, V2>> mapperClass, Class<K2> keyClass, Class<V2> valueClass) { return input.parallelDo(new MapperFn<K1, V1, K2, V2>(mapperClass), tableOf(keyClass, valueClass)); }