private boolean shouldPartition() { return node.getParallelism() > 1; }
Node addNode(Node parent, Node child, String parentStreamId) { return addNode(parent, child, parentStreamId, parent.getParallelism()); }
Node addNode(Node parent, Node child) { return addNode(parent, child, parent.getOutputStreams().iterator().next(), parent.getParallelism()); }
/** * 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); }
/** * 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); }
private <R, V1> PairStream<K, R> coGroupByKeyPartition(PairStream<K, V1> otherStream) { String firstStream = stream; String secondStream = otherStream.stream; Node coGroupNode = addProcessorNode( new CoGroupByKeyProcessor<>(firstStream, secondStream), KEY_VALUE, true); addNode(otherStream.getNode(), coGroupNode, coGroupNode.getParallelism()); return new PairStream<>(streamBuilder, coGroupNode); }
private <R, V1> PairStream<K, R> joinPartition(PairStream<K, V1> otherStream, ValueJoiner<? super V, ? super V1, ? extends R> valueJoiner, JoinProcessor.JoinType leftType, JoinProcessor.JoinType rightType) { String leftStream = stream; String rightStream = otherStream.stream; Node joinNode = addProcessorNode( new JoinProcessor<>(leftStream, rightStream, valueJoiner, leftType, rightType), KEY_VALUE, true); addNode(otherStream.getNode(), joinNode, joinNode.getParallelism()); return new PairStream<>(streamBuilder, joinNode); }
private boolean shouldPartitionByKey() { if (node.getParallelism() == 1) { return false; } /* * if the current processor preserves the key and is * already partitioned on key, skip the re-partition. */ if (node instanceof ProcessorNode) { ProcessorNode pn = (ProcessorNode) node; Fields fields = pn.getGroupingInfo() == null ? null : pn.getGroupingInfo().getFields(); if (pn.isPreservesKey() && fields != null && fields.equals(KEY)) { return false; } } return true; }