/** * Activates the {@link RoutingPolicy#PARTITIONED PARTITIONED} routing * policy and applies the {@link Partitioner#defaultPartitioner() default} * Hazelcast partitioning strategy. The strategy is applied to the result of * the {@code extractKeyFn} function. */ @Nonnull public <T> Edge partitioned(@Nonnull DistributedFunction<T, ?> extractKeyFn) { return partitioned(extractKeyFn, defaultPartitioner()); }
/** * Activates a special-cased {@link RoutingPolicy#PARTITIONED PARTITIONED} * routing policy where all items will be assigned the same, randomly * chosen partition ID. Therefore all items will be directed to the same * processor. */ @Nonnull public Edge allToOne() { return partitioned(wholeItem(), new Single()); }
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 addSessionWindow(Planner p, SessionWindowDef wDef) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), aggregateToSessionWindowP( wDef.sessionTimeout(), nCopies(keyFns.size(), (DistributedToLongFunction<JetEvent>) JetEvent::timestamp), keyFns, aggrOp, mapToOutputFn )); p.addEdges(this, pv.v, (e, ord) -> e.distributed().partitioned(keyFns.get(ord))); } }
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 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())); } }
@Override public void addToDag(Planner p) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), processorSupplier); p.addEdges(this, pv.v, e -> e.partitioned(partitionKeyFn).distributed()); } }
@Override public void addToDag(Planner p) { String vertexName = p.uniqueVertexName(this.name()); Vertex v1 = p.dag.newVertex(vertexName + FIRST_STAGE_VERTEX_NAME_SUFFIX, distinctP(keyFn)) .localParallelism(localParallelism()); PlannerVertex pv2 = p.addVertex(this, vertexName, localParallelism(), distinctP(keyFn)); p.addEdges(this, v1, (e, ord) -> e.partitioned(keyFn, HASH_CODE)); p.dag.edge(between(v1, pv2.v).distributed().partitioned(keyFn)); }
@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)); }
@Override public void addToDag(Planner p) { PlannerVertex pv = p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), rollingAggregateP(keyFn, aggrOp, mapToOutputFn)); p.addEdges(this, pv.v, edge -> edge.partitioned(keyFn).distributed()); } }
.partitioned(Trade::getTicker, HASH_CODE)) .edge(between(slidingStage1, slidingStage2) .partitioned(entryKey(), HASH_CODE) .distributed()) .edge(between(slidingStage2, formatOutput)
.edge(between(docLines.localParallelism(1), tokenize)) .edge(between(tokenize, accumulate) .partitioned(wholeItem(), HASH_CODE)) .edge(between(accumulate, combine) .distributed() .partitioned(entryKey())) .edge(between(combine, sink));
.edge(from(docSource, 1).to(docLines)) .edge(from(docLines).to(tokenize, 1)) .edge(between(tokenize, tf).partitioned(wholeItem(), HASH_CODE)) .edge(between(docCount, tfidf).broadcast().priority(-1)) .edge(from(tf).to(tfidf, 1).distributed().partitioned(byWord, HASH_CODE)) .edge(between(tfidf, sink));
.partitioned(Trade::getTicker, HASH_CODE) .distributed()) .edge(between(slidingWindow, formatOutput).isolated())
.priority(-1)) .edge(from(tradesSource).to(hashJoin, 1) .partitioned(Trade::getTicker)) .edge(between(hashJoin, sink));