/** * Get an {@link LCollection} containing just the values from this table */ default LCollection<V> values() { return factory().wrap(underlying().values()); }
public static <T> PObject<BloomFilter> createFilter(PCollection<T> collection, BloomFilterFn<T> filterFn) { collection.getPipeline().getConfiguration().set(BloomFilterFn.CRUNCH_FILTER_NAME, collection.getName()); return new FirstElementPObject<BloomFilter>(createFilterTable(collection, filterFn).values()); }
/** Randomizes the order of the items in the collection via a MapReduce job */ private static <T> PCollection<T> randomize(PCollection<T> items) { PTable<Long, T> table = items.by("randomize", new RandomizeFn<T>(), Writables.longs()); table = Sort.sort(table, Sort.Order.ASCENDING); return table.values(); }
/** * 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(); } }
return pipeline.read(source).values();
/** * 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(); }
/** * Returns the number of elements in the provided PCollection. * * @param collect The PCollection whose elements should be counted. * @param <S> The type of the PCollection. * @return A {@code PObject} containing the number of elements in the {@code PCollection}. */ public static <S> PObject<Long> length(PCollection<S> collect) { PTypeFamily tf = collect.getTypeFamily(); PTable<Integer, Long> countTable = collect .parallelDo("Aggregate.count", new MapFn<S, Pair<Integer, Long>>() { public Pair<Integer, Long> map(S input) { return Pair.of(1, 1L); } }, tf.tableOf(tf.ints(), tf.longs())) .groupByKey(GroupingOptions.builder().numReducers(1).build()) .combineValues(Aggregators.SUM_LONGS()); PCollection<Long> count = countTable.values(); return new FirstElementPObject<Long>(count); }
/** * Returns the number of elements in the provided PCollection. * * @param collect The PCollection whose elements should be counted. * @param <S> The type of the PCollection. * @return A {@code PObject} containing the number of elements in the {@code PCollection}. */ public static <S> PObject<Long> length(PCollection<S> collect) { PTypeFamily tf = collect.getTypeFamily(); PTable<Integer, Long> countTable = collect .parallelDo("Aggregate.count", new MapFn<S, Pair<Integer, Long>>() { public Pair<Integer, Long> map(S input) { return Pair.of(1, 1L); } public void cleanup(Emitter<Pair<Integer, Long>> e) { e.emit(Pair.of(1, 0L)); } }, tf.tableOf(tf.ints(), tf.longs())) .groupByKey(GroupingOptions.builder().numReducers(1).build()) .combineValues(Aggregators.SUM_LONGS()); PCollection<Long> count = countTable.values(); return new FirstElementPObject<Long>(count, 0L); }
PGroupedTable<Pair<GenericData.Record, Integer>, E> grouped = numWriters > 0 ? table.groupByKey(numWriters) : table.groupByKey(); return grouped.ungroup().values(); } else { return partition(collection, numWriters);