private DataStream<Tuple> processInput( String boltId, IRichBolt userBolt, GlobalStreamId streamId, Grouping grouping, Map<String, DataStream<Tuple>> producer) { assert (userBolt != null); assert (boltId != null); assert (streamId != null); assert (grouping != null); assert (producer != null); final String producerId = streamId.get_componentId(); final String inputStreamId = streamId.get_streamId(); DataStream<Tuple> inputStream = producer.get(inputStreamId); final FlinkOutputFieldsDeclarer declarer = new FlinkOutputFieldsDeclarer(); declarers.put(boltId, declarer); userBolt.declareOutputFields(declarer); this.outputStreams.put(boltId, declarer.outputStreams); // if producer was processed already if (grouping.is_set_shuffle()) { // Storm uses a round-robin shuffle strategy inputStream = inputStream.rebalance(); } else if (grouping.is_set_fields()) { // global grouping is emulated in Storm via an empty fields grouping list final List<String> fields = grouping.get_fields(); if (fields.size() > 0) { FlinkOutputFieldsDeclarer prodDeclarer = this.declarers.get(producerId); inputStream = inputStream.keyBy( prodDeclarer.getGroupingFieldIndexes(inputStreamId, grouping.get_fields())); } else { inputStream = inputStream.global(); } } else if (grouping.is_set_all()) { inputStream = inputStream.broadcast(); } else if (!grouping.is_set_local_or_shuffle()) { throw new UnsupportedOperationException( "Flink only supports (local-or-)shuffle, fields, all, and global grouping"); } return inputStream; }
/** * KeyBy operation for connected data stream. Assigns keys to the elements of input1 and input2 * using keySelector1 and keySelector2. * * @param keySelector1 The {@link KeySelector} used for grouping the first input * @param keySelector2 The {@link KeySelector} used for grouping the second input * @return The partitioned {@link ConnectedStreams} */ public ConnectedStreams<IN1, IN2> keyBy( KeySelector<IN1, ?> keySelector1, KeySelector<IN2, ?> keySelector2) { return new ConnectedStreams<>( environment, inputStream1.keyBy(keySelector1), inputStream2.keyBy(keySelector2)); }
/** * KeyBy operation for connected data stream using key expressions. the elements of input1 and * input2 according to fields1 and fields2. A field expression is either the name of a public * field or a getter method with parentheses of the {@link DataStream}S underlying type. A dot can * be used to drill down into objects, as in {@code "field1.getInnerField2()" } . * * @param fields1 The grouping expressions for the first input * @param fields2 The grouping expressions for the second input * @return The grouped {@link ConnectedStreams} */ public ConnectedStreams<IN1, IN2> keyBy(String[] fields1, String[] fields2) { return new ConnectedStreams<>( environment, inputStream1.keyBy(fields1), inputStream2.keyBy(fields2)); }
/** * KeyBy operation for connected data stream. Assigns keys to the elements of input1 and input2 * according to keyPositions1 and keyPositions2. * * @param keyPositions1 The fields used to group the first input stream. * @param keyPositions2 The fields used to group the second input stream. * @return The grouped {@link ConnectedStreams} */ public ConnectedStreams<IN1, IN2> keyBy(int[] keyPositions1, int[] keyPositions2) { return new ConnectedStreams<>( environment, inputStream1.keyBy(keyPositions1), inputStream2.keyBy(keyPositions2)); }
/** * KeyBy operation for connected data stream. Assigns keys to the elements of input1 and input2 * according to keyPosition1 and keyPosition2. * * @param keyPosition1 The field used to compute the hashcode of the elements in the first input * stream. * @param keyPosition2 The field used to compute the hashcode of the elements in the second input * stream. * @return The grouped {@link ConnectedStreams} */ public ConnectedStreams<IN1, IN2> keyBy(int keyPosition1, int keyPosition2) { return new ConnectedStreams<>( this.environment, inputStream1.keyBy(keyPosition1), inputStream2.keyBy(keyPosition2)); }