@Override public final void run(final SourceContext<OUT> ctx) throws Exception { final GlobalJobParameters config = super.getRuntimeContext().getExecutionConfig().getGlobalJobParameters(); StormConfig stormConfig = new StormConfig(); if (config != null) { if (config instanceof StormConfig) { stormConfig = (StormConfig) config; } else { stormConfig.putAll(config.toMap()); } } final TopologyContext stormTopologyContext = WrapperSetupHelper.createTopologyContext( (StreamingRuntimeContext) super.getRuntimeContext(), this.spout, this.name, this.stormTopology, stormConfig); SpoutCollector<OUT> collector = new SpoutCollector<OUT>(this.numberOfAttributes, stormTopologyContext.getThisTaskId(), ctx); this.spout.open(stormConfig, stormTopologyContext, new SpoutOutputCollector(collector)); this.spout.activate(); if (numberOfInvocations == null) { if (this.spout instanceof FiniteSpout) { final FiniteSpout finiteSpout = (FiniteSpout) this.spout; while (this.isRunning && !finiteSpout.reachedEnd()) { finiteSpout.nextTuple(); } } else { while (this.isRunning) { this.spout.nextTuple(); } } } else { int counter = this.numberOfInvocations; if (counter >= 0) { while ((--counter >= 0) && this.isRunning) { this.spout.nextTuple(); } } else { do { collector.tupleEmitted = false; this.spout.nextTuple(); } while (collector.tupleEmitted && this.isRunning); } } }
/** * Instantiates a new {@link SpoutWrapper} that calls the {@link IRichSpout#nextTuple() * nextTuple()} method of the given {@link IRichSpout spout} a finite number of times. The output * type can be any type if parameter {@code rawOutput} is {@code true} and the spout's number of * declared output tuples is 1. If {@code rawOutput} is {@code false} the output type will be one * of {@link Tuple0} to {@link Tuple25} depending on the spout's declared number of attributes. * * @param spout The {@link IRichSpout spout} to be used. * @param name The name of the spout. * @param rawOutputs Contains stream names if a single attribute output stream, should not be of * type {@link Tuple1} but be of a raw type. (Can be {@code null}.) * @param numberOfInvocations The number of calls to {@link IRichSpout#nextTuple()}. If value is * negative, {@link SpoutWrapper} terminates if no tuple was emitted for the first time. If * value is {@code null}, finite invocation is disabled. * @throws IllegalArgumentException If {@code rawOuput} is {@code true} and the number of declared * output attributes is not 1 or if {@code rawOuput} is {@code false} and the number of * declared output attributes is not with range [0;25]. */ public SpoutWrapper( final IRichSpout spout, final String name, final Collection<String> rawOutputs, final Integer numberOfInvocations) throws IllegalArgumentException { this.spout = spout; this.name = name; this.numberOfAttributes = WrapperSetupHelper.getNumberOfAttributes(spout, rawOutputs); this.numberOfInvocations = numberOfInvocations; }