public PlannerVertex addVertex(Transform transform, String name, int localParallelism, ProcessorMetaSupplier metaSupplier) { PlannerVertex pv = new PlannerVertex(dag.newVertex(name, metaSupplier)); pv.v.localParallelism(localParallelism); xform2vertex.put(transform, pv); return pv; }
@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())); }
Vertex collector = p.dag.newVertex(collectorName + collectorOrdinal, () -> new HashJoinCollectP(getKeyFn, projectFn)); collector.localParallelism(1); p.dag.edge(from(fromPv.v, fromPv.nextAvailableOrdinal()) .to(collector, 0)
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 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()); }
@Override @SuppressWarnings("unchecked") public void addToDag(Planner p) { if (emitsWatermarks || eventTimePolicy == null) { // Reached when the source either emits both JetEvents and watermarks // or neither. In these cases we don't have to insert watermarks. p.addVertex(this, p.uniqueVertexName(name()), localParallelism(), metaSupplierFn.apply(eventTimePolicy != null ? eventTimePolicy : noEventTime()) ); } else { // ------------ // | sourceP | // ------------ // | // isolated // v // ------------- // | insertWmP | // ------------- String v1name = p.uniqueVertexName(name()); Vertex v1 = p.dag.newVertex(v1name, metaSupplierFn.apply(eventTimePolicy)) .localParallelism(localParallelism()); PlannerVertex pv2 = p.addVertex( this, v1name + "-add-timestamps", localParallelism(), insertWatermarksP(eventTimePolicy) ); p.dag.edge(between(v1, pv2.v).isolated()); } }
private void addToDagTwoStage(Planner p) { String vertexName = p.uniqueVertexName(name()); Vertex v1 = p.dag.newVertex(vertexName + FIRST_STAGE_VERTEX_NAME_SUFFIX, accumulateP(aggrOp)) .localParallelism(localParallelism()); PlannerVertex pv2 = p.addVertex(this, vertexName, 1, combineP(aggrOp)); p.addEdges(this, v1); p.dag.edge(between(v1, pv2.v).distributed().allToOne()); } }
return dag.edge(between(source.localParallelism(1), docLines)) .edge(between(docLines.localParallelism(1), tokenize)) .edge(between(tokenize, accumulate) .partitioned(wholeItem(), HASH_CODE))
Vertex sink = dag.newVertex("sink", SinkProcessors.writeMapP(INVERTED_INDEX)); stopwordSource.localParallelism(1); docSource.localParallelism(1); docCount.localParallelism(1); docLines.localParallelism(1);
SinkProcessors.writeFileP(OUTPUT_DIR_NAME, Object::toString, StandardCharsets.UTF_8, false)); tradeSource.localParallelism(1);
Vertex sink = dag.newVertex("sink", writeLoggerP(o -> Arrays.toString((Object[]) o))); tradesSource.localParallelism(1); collectToMap.localParallelism(1); sink.localParallelism(1);
@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)); }
writeFileP(OUTPUT_DIR_NAME, Object::toString, StandardCharsets.UTF_8, false)); streamTrades.localParallelism(1);
public static void main(String[] args) { System.setProperty("hazelcast.logging.type", "log4j"); JetInstance instance = Jet.newJetInstance(); Jet.newJetInstance(); try { TickerInfo.populateMap(instance.getHazelcastInstance().getReplicatedMap("tickersInfo")); DAG dag = new DAG(); Vertex tradesSource = dag.newVertex("tradesSource", GenerateTradesP::new); Vertex enrichment = dag.newVertex("enrichment", mapUsingContextP(replicatedMapContext("tickersInfo"), (ReplicatedMap<String, TickerInfo> map, Trade item) -> tuple2(item, map.get(item.getTicker())))); Vertex sink = dag.newVertex("sink", DiagnosticProcessors.writeLoggerP()); tradesSource.localParallelism(1); dag .edge(between(tradesSource, enrichment)) .edge(between(enrichment, sink)); instance.newJob(dag).join(); } finally { Jet.shutdownAll(); } } }