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; }
public static void generateRandomizedIntegerSequence( StreamExecutionEnvironment env, String brokerConnection, String topic, final int numPartitions, final int numElements, final boolean randomizeOrder) throws Exception { env.setParallelism(numPartitions); env.getConfig().disableSysoutLogging(); env.setNumberOfExecutionRetries(0); DataStream<Integer> stream = env.addSource( new RichParallelSourceFunction<Integer>() { private volatile boolean running = true; @Override public void run(SourceContext<Integer> ctx) { // create a sequence int[] elements = new int[numElements]; for (int i = 0, val = getRuntimeContext().getIndexOfThisSubtask(); i < numElements; i++, val += getRuntimeContext().getNumberOfParallelSubtasks()) { elements[i] = val; } // scramble the sequence if (randomizeOrder) { Random rnd = new Random(); for (int i = 0; i < elements.length; i++) { int otherPos = rnd.nextInt(elements.length); int tmp = elements[i]; elements[i] = elements[otherPos]; elements[otherPos] = tmp; } } // emit the sequence int pos = 0; while (running && pos < elements.length) { ctx.collect(elements[pos++]); } } @Override public void cancel() { running = false; } }); stream .rebalance() .addSink( new FlinkKafkaProducer<>( topic, new TypeInformationSerializationSchema<>( BasicTypeInfo.INT_TYPE_INFO, env.getConfig()), FlinkKafkaProducer.getPropertiesFromBrokerList(brokerConnection), new KafkaPartitioner() { @Override public int partition(Object key, int numPartitions) { return ((Integer) key) % numPartitions; } })); env.execute("Scrambles int sequence generator"); }