Node addNode(Node child) { return addNode(node, child); }
Node addProcessorNode(Processor<?> processor, Fields outputFields) { return addNode(makeProcessorNode(processor, outputFields)); }
Node addProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) { return addNode(makeProcessorNode(processor, outputFields, preservesKey)); }
private void addSinkNode(SinkNode sinkNode, int parallelism) { String boltId = UniqueIdGen.getInstance().getUniqueBoltId(); sinkNode.setComponentId(boltId); sinkNode.setParallelism(parallelism); if (node instanceof SpoutNode) { addNode(node, sinkNode, Utils.DEFAULT_STREAM_ID, parallelism); } else { addNode(node, sinkNode, parallelism); } }
/** * Returns a new stream with the given value of parallelism. Further operations on this stream would execute at this level of * parallelism. * * @param parallelism the parallelism value * @return the new stream */ public Stream<T> repartition(int parallelism) { if (parallelism < 1) { throw new IllegalArgumentException("Parallelism should be >= 1"); } if (node.getParallelism() == parallelism) { LOG.debug("Node's current parallelism {}, new parallelism {}", node.getParallelism(), parallelism); return this; } Node partitionNode = addNode(node, new PartitionNode(stream, node.getOutputFields()), parallelism); return new Stream<>(streamBuilder, partitionNode); }
private Stream<T> global() { Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.global())); return new Stream<>(streamBuilder, partitionNode); }
protected Stream<T> partitionBy(Fields fields, int parallelism) { return new Stream<>( streamBuilder, addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism)); }
/** * Returns an array of streams by splitting the given stream into multiple branches based on the given predicates. The predicates are * applied in the given order to the values of this stream and the result is forwarded to the corresponding (index based) result stream * based on the (index of) predicate that matches. * <p> * <b>Note:</b> If none of the predicates match a value, that value is dropped. * </p> * * @param predicates the predicates * @return an array of result streams (branches) corresponding to the given predicates */ @SuppressWarnings("unchecked") public Stream<T>[] branch(Predicate<? super T>... predicates) { List<Stream<T>> childStreams = new ArrayList<>(); if (predicates.length > 0) { BranchProcessor<T> branchProcessor = new BranchProcessor<>(); Node branchNode = addProcessorNode(branchProcessor, VALUE); for (Predicate<? super T> predicate : predicates) { // create a child node (identity) per branch ProcessorNode child = makeProcessorNode(new MapProcessor<>(new IdentityFunction<>()), node.getOutputFields()); String branchStream = child.getOutputStreams().iterator().next() + "-branch"; // branchStream is the parent stream that connects branch processor to this child branchNode.addOutputStream(branchStream); addNode(branchNode, child, branchStream); childStreams.add(new Stream<>(streamBuilder, child)); branchProcessor.addPredicate(predicate, branchStream); } } return childStreams.toArray((Stream<T>[]) new Stream[childStreams.size()]); }
/** * Returns a new stream consisting of the elements that fall within the window as specified by the window parameter. The {@link Window} * specification could be used to specify sliding or tumbling windows based on time duration or event count. For example, * <pre> * // time duration based sliding window * stream.window(SlidingWindows.of(Duration.minutes(10), Duration.minutes(1)); * * // count based sliding window * stream.window(SlidingWindows.of(Count.(10), Count.of(2))); * * // time duration based tumbling window * stream.window(TumblingWindows.of(Duration.seconds(10)); * </p> * * @see org.apache.storm.streams.windowing.SlidingWindows * @see org.apache.storm.streams.windowing.TumblingWindows * @param window the window configuration * @return the new stream */ public Stream<T> window(Window<?, ?> window) { return new Stream<>(streamBuilder, addNode(new WindowNode(window, stream, node.getOutputFields()))); }