public static <C, T, R> ProcessorTransform flatMapUsingContextTransform( @Nonnull Transform upstream, @Nonnull ContextFactory<C> contextFactory, @Nonnull DistributedBiFunction<? super C, ? super T, ? extends Traverser<? extends R>> flatMapFn ) { return new ProcessorTransform("flatMapUsingContext", upstream, flatMapUsingContextP(contextFactory, flatMapFn)); }
private static DistributedSupplier<Processor> formatOutput() { return () -> { // If DateTimeFormatter was serializable, it could be created in // buildDag() and simply captured by the serializable lambda below. Since // it isn't, we need this long-hand approach that explicitly creates the // formatter at the use site instead of having it implicitly deserialized. DateTimeFormatter timeFormat = DateTimeFormatter.ofPattern("HH:mm:ss.SSS"); return Processors.mapP((TimestampedEntry<String, Long> f) -> String.format("%s %5s %4d", timeFormat.format(Instant.ofEpochMilli(f.getTimestamp()).atZone(ZoneId.systemDefault())), f.getKey(), f.getValue())).get(); }; } }
public static <C, T, R> ProcessorTransform mapUsingContextTransform( @Nonnull Transform upstream, @Nonnull ContextFactory<C> contextFactory, @Nonnull DistributedBiFunction<? super C, ? super T, ? extends R> mapFn ) { return new ProcessorTransform("mapUsingContext", upstream, mapUsingContextP(contextFactory, mapFn)); }
flatMapUsingContextP( ContextFactory.withCreateFn(jet -> null).nonCooperative(), (Object ctx, Entry<?, String> e) -> traverseStream(docLines(e.getValue()))) flatMapP((String line) -> traverseArray(delimiter.split(line.toLowerCase())) .filter(word -> !word.isEmpty())) ); Vertex accumulate = dag.newVertex("accumulate", accumulateByKeyP(singletonList(wholeItem()), counting())); Vertex combine = dag.newVertex("combine", combineByKeyP(counting(), Util::entry));
Vertex docCount = dag.newVertex("doc-count", Processors.aggregateP(counting())); flatMapUsingContextP( ContextFactory.withCreateFn(jet -> null).nonCooperative(), (Object ctx, Entry<Long, String> e) -> Vertex tf = dag.newVertex("tf", aggregateByKeyP(singletonList(wholeItem()), counting(), Util::entry));
))); Vertex slidingStage1 = dag.newVertex("sliding-stage-1", Processors.accumulateByFrameP( singletonList(keyFn), singletonList(timestampFn), TimestampKind.EVENT, winPolicy, counting() )); Vertex slidingStage2 = dag.newVertex("sliding-stage-2", Processors.combineToSlidingWindowP(winPolicy, counting(), TimestampedEntry::fromWindowResult)); Vertex formatOutput = dag.newVertex("format-output", mapUsingContextP( ContextFactory.withCreateFn(x -> DateTimeFormatter.ofPattern("HH:mm:ss.SSS")), (DateTimeFormatter timeFormat, TimestampedEntry<String, Long> tse) ->
@Nonnull private static DAG buildDag(Map<String, Long> counts) { final Pattern delimiter = Pattern.compile("\\W+"); DAG dag = new DAG(); Vertex source = dag.newVertex("source", DocLinesP::new); Vertex tokenize = dag.newVertex("tokenize", flatMapP((String line) -> traverseArray(delimiter.split(line.toLowerCase())) .filter(word -> !word.isEmpty())) ); Vertex aggregate = dag.newVertex("aggregate", aggregateByKeyP(singletonList(wholeItem()), counting(), Util::entry)); Vertex sink = dag.newVertex("sink", () -> new MapSinkP(counts)); return dag.edge(between(source.localParallelism(1), tokenize)) .edge(between(tokenize, aggregate).partitioned(wholeItem(), HASH_CODE)) .edge(between(aggregate, sink)); }
private void addSlidingWindowTwoStage(Planner p, SlidingWindowDef wDef) { String vertexName = p.uniqueVertexName(name()); SlidingWindowPolicy winPolicy = wDef.toSlidingWindowPolicy(); Vertex v1 = p.dag.newVertex(vertexName + FIRST_STAGE_VERTEX_NAME_SUFFIX, accumulateByFrameP( keyFns, nCopies(keyFns.size(), (DistributedToLongFunction<JetEvent>) JetEvent::timestamp), TimestampKind.EVENT, winPolicy, aggrOp)); v1.localParallelism(localParallelism()); PlannerVertex pv2 = p.addVertex(this, vertexName, localParallelism(), combineToSlidingWindowP(winPolicy, aggrOp, mapToOutputFn)); p.addEdges(this, v1, (e, ord) -> e.partitioned(keyFns.get(ord), HASH_CODE)); p.dag.edge(between(v1, pv2.v).distributed().partitioned(entryKey())); }
private void addToDagTwoStage(Planner p) { List<DistributedFunction<?, ? extends K>> groupKeyFns = this.groupKeyFns; String vertexName = p.uniqueVertexName(this.name()); Vertex v1 = p.dag.newVertex(vertexName + FIRST_STAGE_VERTEX_NAME_SUFFIX, accumulateByKeyP(groupKeyFns, aggrOp)) .localParallelism(localParallelism()); PlannerVertex pv2 = p.addVertex(this, vertexName, localParallelism(), combineByKeyP(aggrOp, mapToOutputFn)); p.addEdges(this, v1, (e, ord) -> e.partitioned(groupKeyFns.get(ord), HASH_CODE)); p.dag.edge(between(v1, pv2.v).distributed().partitioned(entryKey())); } }
private void addToDagSingleStage(Planner p) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), 1, aggregateP(aggrOp)); p.addEdges(this, pv.v, edge -> edge.distributed().allToOne()); }
private void addToDagSingleStage(Planner p) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), aggregateByKeyP(groupKeyFns, aggrOp, mapToOutputFn)); p.addEdges(this, pv.v, (e, ord) -> e.distributed().partitioned(groupKeyFns.get(ord))); }
private void addSlidingWindowSingleStage(Planner p, SlidingWindowDef wDef) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), aggregateToSlidingWindowP( keyFns, nCopies(keyFns.size(), (DistributedToLongFunction<JetEvent>) JetEvent::timestamp), TimestampKind.EVENT, wDef.toSlidingWindowPolicy(), aggrOp, mapToOutputFn )); p.addEdges(this, pv.v, (e, ord) -> e.distributed().partitioned(keyFns.get(ord))); }
private void addSlidingWindowTwoStage(Planner p, SlidingWindowDef wDef) { String vertexName = p.uniqueVertexName(name()); SlidingWindowPolicy winPolicy = wDef.toSlidingWindowPolicy(); Vertex v1 = p.dag.newVertex(vertexName + FIRST_STAGE_VERTEX_NAME_SUFFIX, accumulateByFrameP( nCopies(aggrOp.arity(), constantKey()), nCopies(aggrOp.arity(), (DistributedToLongFunction<JetEvent>) JetEvent::timestamp), TimestampKind.EVENT, winPolicy, aggrOp )); v1.localParallelism(localParallelism()); PlannerVertex pv2 = p.addVertex(this, vertexName, 1, combineToSlidingWindowP(winPolicy, aggrOp, mapToOutputFn.toKeyedWindowResultFn())); p.addEdges(this, v1); p.dag.edge(between(v1, pv2.v).distributed().allToOne()); }
Vertex readTickerInfoMap = dag.newVertex("readTickerInfoMap", readMapP(TICKER_INFO_MAP_NAME)); Vertex collectToMap = dag.newVertex("collectToMap", Processors.aggregateP(AggregateOperations.toMap(entryKey(), entryValue()))); Vertex hashJoin = dag.newVertex("hashJoin", () -> new HashJoinP<>(Trade::getTicker)); Vertex sink = dag.newVertex("sink", writeLoggerP(o -> Arrays.toString((Object[]) o)));
private void addSlidingWindowSingleStage(Planner p, SlidingWindowDef wDef) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), 1, aggregateToSlidingWindowP( nCopies(aggrOp.arity(), constantKey()), nCopies(aggrOp.arity(), (DistributedToLongFunction<JetEvent>) JetEvent::timestamp), TimestampKind.EVENT, wDef.toSlidingWindowPolicy(), aggrOp, mapToOutputFn.toKeyedWindowResultFn() )); p.addEdges(this, pv.v, edge -> edge.distributed().allToOne()); }
@Override public void addToDag(Planner p) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), mapP(identity())); p.addEdges(this, pv.v); } }
public static <C, T, K, R> PartitionedProcessorTransform<T, K> mapUsingContextPartitionedTransform( @Nonnull Transform upstream, @Nonnull ContextFactory<C> contextFactory, @Nonnull DistributedBiFunction<? super C, ? super T, ? extends R> mapFn, @Nonnull DistributedFunction<? super T, ? extends K> partitionKeyFn ) { return new PartitionedProcessorTransform<>("mapUsingPartitionedContext", upstream, mapUsingContextP(contextFactory, mapFn), partitionKeyFn); }
public static <C, T, K, R> PartitionedProcessorTransform<T, K> flatMapUsingPartitionedContextTransform( @Nonnull Transform upstream, @Nonnull ContextFactory<C> contextFactory, @Nonnull DistributedBiFunction<? super C, ? super T, ? extends Traverser<? extends R>> flatMapFn, @Nonnull DistributedFunction<? super T, ? extends K> partitionKeyFn ) { return new PartitionedProcessorTransform<>("flatMapUsingPartitionedContext", upstream, flatMapUsingContextP(contextFactory, flatMapFn), partitionKeyFn); }
))); Vertex slidingWindow = dag.newVertex("aggregate-to-sliding-win", aggregateToSlidingWindowP( singletonList((DistributedFunction<Trade, String>) Trade::getTicker), singletonList((DistributedToLongFunction<Trade>) Trade::getTime),
@Override public void addToDag(Planner p) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), mapP(mapFn())); p.addEdges(this, pv.v); } }