/** * Get an {@link LCollection} containing just the keys from this table */ default LCollection<K> keys() { return factory().wrap(underlying().keys()); }
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(); }
private static <K, V> void configureReducers(GroupingOptions.Builder builder, PTable<K, V> ptable, Configuration conf, int numReducers) { if (numReducers <= 0) { numReducers = PartitionUtils.getRecommendedPartitions(ptable, conf); if (numReducers < 5) { // Not worth the overhead, force it to 1 numReducers = 1; } } builder.numReducers(numReducers); if (numReducers > 1) { Iterable<K> iter = Sample.reservoirSample(ptable.keys(), numReducers - 1).materialize(); MaterializableIterable<K> mi = (MaterializableIterable<K>) iter; if (mi.isSourceTarget()) { builder.sourceTargets((SourceTarget) mi.getSource()); } builder.partitionerClass(TotalOrderPartitioner.class); builder.conf(TotalOrderPartitioner.PARTITIONER_PATH, mi.getPath().toString()); //TODO: distcache handling } }
/** * Sorts the {@code PCollection} using the natural ordering of its elements in * the order specified using the given number of reducers. * * @return a {@code PCollection} representing the sorted collection. */ public static <T> PCollection<T> sort(PCollection<T> collection, int numReducers, Order order) { PTypeFamily tf = collection.getTypeFamily(); PTableType<T, Void> type = tf.tableOf(collection.getPType(), tf.nulls()); Configuration conf = collection.getPipeline().getConfiguration(); PTable<T, Void> pt = collection.parallelDo("sort-pre", new DoFn<T, Pair<T, Void>>() { @Override public void process(T input, Emitter<Pair<T, Void>> emitter) { emitter.emit(Pair.of(input, (Void) null)); } }, type); GroupingOptions options = buildGroupingOptions(pt, conf, numReducers, order); return pt.groupByKey(options).ungroup().keys(); }
.numReducers(splitPoints.size() + 1) .build(); return t.groupByKey(options).ungroup().keys();
.numReducers(splitPoints.size() + 1) .build(); return t.groupByKey(options).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()))); }
/** * Calculate a set of percentiles for each key in a numerically-valued table. * * Percentiles 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 percentile 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 percentile (in the range 0.0 - 1.0) * @param pn More percentiles (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 percentile 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> percentileList = 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 DistributedPercentiles<K, V>(percentileList), ptf.tableOf(table.getKeyType(), Result.pType(table.getValueType()))); }
PCollection<BloomFilter> bloomFilters = left.keys().parallelDo( "Create bloom filters", new CreateBloomFilterFn<>(vectorSize, nbHash, left.getKeyType()),