protected Stream<T> partitionBy(Fields fields, int parallelism) { return new Stream<>( streamBuilder, addNode(node, new PartitionNode(stream, node.getOutputFields(), GroupingInfo.fields(fields)), parallelism)); }
/** * Creates a new {@link Stream} of values from the given {@link IRichSpout} by extracting field(s) from tuples via the supplied {@link * TupleValueMapper}. * * @param spout the spout * @param valueMapper the value mapper * @param <T> the type of values in the resultant stream * @return the new stream */ public <T> Stream<T> newStream(IRichSpout spout, TupleValueMapper<T> valueMapper) { return newStream(spout).map(valueMapper); }
/** * {@inheritDoc} */ public PairStream<K, V> filter(Predicate<? super Pair<K, V>> predicate) { return toPairStream(super.filter(predicate)); }
void processAndAck(Tuple input) { RefCountedTuple refCountedTuple = new RefCountedTuple(input); setAnchor(refCountedTuple); if (isEventTimestamp()) { setEventTimestamp(input.getLongByField(getTimestampField())); } Pair<Object, String> valueAndStream = getValueAndStream(input); process(valueAndStream.getFirst(), valueAndStream.getSecond()); ack(refCountedTuple); }
/** * Creates a new {@link PairStream} of key-value pairs from the given {@link IRichSpout} by extracting key and value from tuples via the * supplied {@link PairValueMapper}. * * @param spout the spout * @param pairValueMapper the pair value mapper * @param <K> the key type * @param <V> the value type * @return the new stream of key-value pairs */ public <K, V> PairStream<K, V> newStream(IRichSpout spout, PairValueMapper<K, V> pairValueMapper) { return newStream(spout).mapToPair(pairValueMapper); }
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 where the values are grouped by keys and the given window. The values that arrive within a window having the * same key will be merged together and returned as an Iterable of values mapped to the key. * * @param window the window configuration * @return the new stream */ public PairStream<K, Iterable<V>> groupByKeyAndWindow(Window<?, ?> window) { return partitionByKey().window(window).aggregatePartition(new MergeValues<>()); }
private Stream<T> global() { Node partitionNode = addNode(new PartitionNode(stream, node.getOutputFields(), GroupingInfo.global())); return new Stream<>(streamBuilder, partitionNode); }
/** * Queries the given stream state with the values in this stream as the keys. * * @param streamState the stream state * @param <V> the value type * @return the result stream */ public <V> PairStream<T, V> stateQuery(StreamState<T, V> streamState) { // need field grouping for state query so that the query is routed to the correct task Node newNode = partitionBy(VALUE, node.getParallelism()).addProcessorNode(new StateQueryProcessor<>(streamState), KEY_VALUE); return new PairStream<>(streamBuilder, newNode); }
ProcessorNode makeProcessorNode(Processor<?> processor, Fields outputFields, boolean preservesKey) { return new ProcessorNode(processor, UniqueIdGen.getInstance().getUniqueStreamId(), outputFields, preservesKey); }
/** * Returns a new stream where the values are grouped by the keys. * * @return the new stream */ public PairStream<K, Iterable<V>> groupByKey() { return partitionByKey().aggregatePartition(new MergeValues<>()); }
/** * {@inheritDoc} */ @Override public PairStream<K, V> window(Window<?, ?> window) { return toPairStream(super.window(window)); }
/** * {@inheritDoc} */ @Override public PairStream<K, V> repartition(int parallelism) { return toPairStream(super.repartition(parallelism)); }
Node insert(Node parent, Node child) { Node newChild = addNode(parent, child); for (Edge edge : graph.outgoingEdgesOf(parent)) { Node oldChild = edge.getTarget(); graph.removeEdge(parent, oldChild); oldChild.removeParentStreams(parent); addNode(newChild, oldChild); } return newChild; }
/** * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as sinks in the stream, for e.g. a {@code * RedisStoreBolt}. * <p> * <b>Note:</b> This would provide guarantees only based on what the bolt provides. * </p> * * @param bolt the bolt * @param parallelism the parallelism of the bolt */ public void to(IBasicBolt bolt, int parallelism) { addSinkNode(new SinkNode(bolt), parallelism); }
WindowedProcessorBolt(String id, DirectedGraph<Node, Edge> graph, List<ProcessorNode> nodes, Window<?, ?> window) { delegate = new ProcessorBoltDelegate(id, graph, nodes); this.window = window; setWindowConfig(); }
/** * {@inheritDoc} */ @Override public PairStream<K, V> peek(Consumer<? super Pair<K, V>> action) { return toPairStream(super.peek(action)); }
StatefulProcessorBolt(String boltId, DirectedGraph<Node, Edge> graph, List<ProcessorNode> nodes) { delegate = new ProcessorBoltDelegate(boltId, graph, nodes); statefulProcessors = getStatefulProcessors(nodes); }
/** * Creates a new {@link Stream} of values from the given {@link IRichSpout} by extracting field(s) from tuples via the supplied {@link * TupleValueMapper} with the given parallelism. * * @param spout the spout * @param valueMapper the value mapper * @param parallelism the parallelism of the stream * @param <T> the type of values in the resultant stream * @return the new stream */ public <T> Stream<T> newStream(IRichSpout spout, TupleValueMapper<T> valueMapper, int parallelism) { return newStream(spout, parallelism).map(valueMapper); }
/** * Sends the elements of this stream to a bolt. This could be used to plug in existing bolts as sinks in the stream, for e.g. a {@code * RedisStoreBolt}. * <p> * <b>Note:</b> This would provide guarantees only based on what the bolt provides. * </p> * * @param bolt the bolt * @param parallelism the parallelism of the bolt */ public void to(IRichBolt bolt, int parallelism) { addSinkNode(new SinkNode(bolt), parallelism); }