/** * Sorts the {@link PCollection} using the natural ordering of its elements in * the order specified. * * @return a {@link PCollection} representing the sorted collection. */ public static <T> PCollection<T> sort(PCollection<T> collection, Order order) { PTypeFamily tf = collection.getTypeFamily(); PTableType<T, Void> type = tf.tableOf(collection.getPType(), tf.nulls()); Configuration conf = collection.getPipeline().getConfiguration(); GroupingOptions options = buildGroupingOptions(conf, tf, collection.getPType(), order); 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); PTable<T, Void> sortedPt = pt.groupByKey(options).ungroup(); return sortedPt.parallelDo("sort-post", new DoFn<Pair<T, Void>, T>() { @Override public void process(Pair<T, Void> input, Emitter<T> emitter) { emitter.emit(input.first()); } }, collection.getPType()); }
/** * Key a PCollection of Avro records by a String field name. This is less safe than writing a custom MapFn, but it * could significantly reduce code volume in cases that need a lot of disparate collections to be joined or processed * according to key values. * @param collection PCollection of Avro records to process * @param fieldPath The Avro schema field name of the field to key on. Use . separated names for nested records * @param fieldType PType of the field you wish to extract from the Avro record. * @param <T> record type * @return supplied collection keyed by the field named fieldName */ public static <T extends SpecificRecord, F> PTable<F, T> keyByAvroField(PCollection<T> collection, String fieldPath, PType<F> fieldType) { Class<T> recordType = collection.getPType().getTypeClass(); return collection.by(new AvroExtractMapFn<T, F>(recordType, fieldPath), fieldType); }
@Override public <T> void writeTextFile(PCollection<T> pcollection, String pathName) { pcollection.parallelDo("asText", new StringifyFn<T>(), Writables.strings()) .write(To.textFile(pathName)); }
public static <S> PCollection<S> sample(PCollection<S> input, long seed, double probability) { String stageName = String.format("sample(%.2f)", probability); return input.parallelDo(stageName, new SamplerFn<S>(seed, probability), input.getPType()); } }
private static <T> PTable<T, Boolean> toTable(PCollection<T> coll) { PTypeFamily typeFamily = coll.getTypeFamily(); return coll.parallelDo(new DoFn<T, Pair<T, Boolean>>() { @Override public void process(T input, Emitter<Pair<T, Boolean>> emitter) { emitter.emit(Pair.of(input, Boolean.TRUE)); } }, typeFamily.tableOf(coll.getPType(), typeFamily.booleans())); }
@Override public PCollection<S> union(PCollection<S>... collections) { Collection<S> output = Lists.newArrayList(); for (PCollection<S> pcollect : collections) { for (S s : pcollect.materialize()) { output.add(s); } } output.addAll(collect); return new MemCollection<S>(output, collections[0].getPType()); }
Configuration conf = cells.getPipeline().getConfiguration(); PTable<C, Void> t = cells.parallelDo( "Pre-partition", new MapFn<C, Pair<C, Void>>() { return Pair.of(input, (Void) null); }, tableOf(cells.getPType(), nulls())); splitPoints = getSplitPoints(table); Path partitionFile = new Path(((DistributedPipeline) cells.getPipeline()).createTempPath(), "partition"); writePartitionInfo(conf, partitionFile, splitPoints); GroupingOptions options = GroupingOptions.builder()
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()); }
/** * Splits a {@link PCollection} of any {@link Pair} of objects into a Pair of * PCollection}, to allow for the output of a DoFn to be handled using * separate channels. * * @param pCollection The {@code PCollection} to split * @param firstPType The {@code PType} for the first collection * @param secondPType The {@code PType} for the second collection * @return {@link Pair} of {@link PCollection} */ public static <T, U> Pair<PCollection<T>, PCollection<U>> split(PCollection<Pair<T, U>> pCollection, PType<T> firstPType, PType<U> secondPType) { PCollection<T> first = pCollection.parallelDo("Extract first value", new FirstEmittingDoFn<T, U>(), firstPType); PCollection<U> second = pCollection.parallelDo("Extract second value", new SecondEmittingDoFn<T, U>(), secondPType); return Pair.of(first, second); }
/** * 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(); }
@Override public <T> Iterable<T> materialize(PCollection<T> pcollection) { return pcollection.materialize(); }
PCollection<String> words = lines.parallelDo(new DoFn<String, String>() { public void process(String line, Emitter<String> emitter) { for (String word : line.split("\\s+")) { PTable<String, Long> counts = words.count();
PTypeFamily typeFamily = coll1.getTypeFamily(); PType<T> type = coll1.getPType(); return Cogroup.cogroup(toTable(coll1), toTable(coll2)).parallelDo( new DoFn<Pair<T, Pair<Collection<Boolean>, Collection<Boolean>>>, Tuple3<T, T, T>>() {
result.write(To.sequenceFile(cmdLine.getOptionValue("output"))); Iterator<ByteBuffer> iter = result.materialize().iterator(); x.start(); while (iter.hasNext()) {
table.getName().getNameAsString(), regionLocator.getAllRegionLocations()); Path regionLocationFilePath = new Path(((DistributedPipeline) cells.getPipeline()).createTempPath(), "regionLocations" + table.getName().getNameAsString()); writeRegionLocationTable(cells.getPipeline().getConfiguration(), regionLocationFilePath, regionLocationTable); hfileTarget.outputConf(RegionLocationTable.REGION_LOCATION_TABLE_PATH, regionLocationFilePath.toString()); partitioned .filter(new FilterByFamilyFn<C>(family)) .write(hfileTarget);
/** * 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(); }
/** * Get the {@link PType} representing how elements of this collection may be serialized. */ default PType<S> pType() { return underlying().getPType(); }
/** 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(); }
cells = cells.filter(new StartRowFilterFn<C>(scan.getStartRow())); cells = cells.filter(new StopRowFilterFn<C>(scan.getStopRow())); cells = cells.filter(new FamilyMapFilterFn<C>(scan.getFamilyMap())); cells = cells.filter(new TimeRangeFilterFn<C>(timeRange)); PTable<ByteBuffer, C> cellsByRow = cells.by(new ExtractRowFn<C>(), bytes()); final int versions = scan.getMaxVersions(); return cellsByRow.groupByKey().parallelDo("CombineKeyValueIntoRow",
/** * Writes out HFiles from the provided <code>cells</code> and <code>table</code>. <code>limitToAffectedRegions</code> * is used to indicate that the regions the <code>cells</code> will be loaded into should be identified prior to writing * HFiles. Identifying the regions ahead of time will reduce the number of reducers needed when writing. This is * beneficial if the data to be loaded only touches a small enough subset of the total regions in the table. If set to * false, the number of reducers will equal the number of regions in the table. * * @see <a href='https://issues.apache.org/jira/browse/CRUNCH-588'>CRUNCH-588</a> */ public static <C extends Cell> void writeToHFilesForIncrementalLoad( PCollection<C> cells, HTable table, Path outputPath, boolean limitToAffectedRegions) throws IOException { HColumnDescriptor[] families = table.getTableDescriptor().getColumnFamilies(); if (families.length == 0) { LOG.warn("{} has no column families", table); return; } PCollection<C> partitioned = sortAndPartition(cells, table, limitToAffectedRegions); for (HColumnDescriptor f : families) { byte[] family = f.getName(); partitioned .filter(new FilterByFamilyFn<C>(family)) .write(new HFileTarget(new Path(outputPath, Bytes.toString(family)), f)); } }