Github user arunmahadevan commented on a diff in the pull request:

    https://github.com/apache/storm/pull/1816#discussion_r92329373
  
    --- Diff: 
external/storm-pmml/src/main/java/org/apache/storm/pmml/PMMLPredictorBolt.java 
---
    @@ -0,0 +1,118 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one
    + *   or more contributor license agreements.  See the NOTICE file
    + *   distributed with this work for additional information
    + *   regarding copyright ownership.  The ASF licenses this file
    + *   to you under the Apache License, Version 2.0 (the
    + *   "License"); you may not use this file except in compliance
    + *   with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + *   Unless required by applicable law or agreed to in writing, software
    + *   distributed under the License is distributed on an "AS IS" BASIS,
    + *   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or 
implied.
    + *   See the License for the specific language governing permissions and
    + *   limitations under the License.
    + */
    +
    +package org.apache.storm.pmml;
    +
    +import org.apache.storm.pmml.model.ModelOutputFields;
    +import org.apache.storm.pmml.model.Stream;
    +import org.apache.storm.pmml.runner.ModelRunner;
    +import org.apache.storm.pmml.runner.ModelRunnerFactory;
    +import org.apache.storm.task.OutputCollector;
    +import org.apache.storm.task.TopologyContext;
    +import org.apache.storm.topology.OutputFieldsDeclarer;
    +import org.apache.storm.topology.base.BaseRichBolt;
    +import org.apache.storm.tuple.Fields;
    +import org.apache.storm.tuple.Tuple;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.List;
    +import java.util.Map;
    +
    +public class PMMLPredictorBolt extends BaseRichBolt {
    +    protected static final Logger LOG = 
LoggerFactory.getLogger(PMMLPredictorBolt.class);
    +
    +    private ModelOutputFields outFields;
    +    private ModelRunnerFactory runnerFactory;
    +    private ModelRunner runner;
    +    private OutputCollector collector;
    +
    +    /*
    +     * Passing a factory rather than the actual object to avoid enforcing 
the strong
    +     * requirement of having to have ModelRunner to be Serializable
    +     */
    +
    +    /**
    +     * Creates an instance of {@link PMMLPredictorBolt} that executes, for 
every tuple, the runner constructed with
    +     * the {@link ModelRunnerFactory} specified in the parameter
    +     * The {@link PMMLPredictorBolt} instantiated with this constructor 
has no declared output fields
    +     */
    +    public PMMLPredictorBolt(ModelRunnerFactory modelRunnerFactory) {
    +        this(modelRunnerFactory, null);
    +    }
    +
    +    /**
    +     * Creates an instance of {@link PMMLPredictorBolt} that executes, for 
every tuple, the runner constructed with
    +     * the {@link ModelRunnerFactory} specified in the parameter
    +     * The {@link PMMLPredictorBolt} instantiated with this constructor 
declares the output fields as specified
    +     * by the {@link ModelOutputFields} parameter
    +     */
    +    public PMMLPredictorBolt(ModelRunnerFactory modelRunnerFactory, 
ModelOutputFields modelOutputFields) {
    +        this.outFields = modelOutputFields;
    +        this.runnerFactory = modelRunnerFactory;
    +        LOG.info("Instantiated {}", this);
    +    }
    +
    +    @Override
    +    public void prepare(Map stormConf, TopologyContext context, 
OutputCollector collector) {
    +        this.runner = runnerFactory.newModelRunner();
    +        this.collector = collector;
    +    }
    +
    +    @Override
    +    public void execute(Tuple input) {
    +        try {
    +            final Map<Stream, List<Object>> scoresPerStream = 
runner.scoredTuplePerStream(input);
    +            LOG.debug("Input tuple [{}] generated predicted scores [{}]", 
input, scoresPerStream);
    +            if (scoresPerStream != null) {
    +                for (Stream stream : scoresPerStream.keySet()) {
    +                    collector.emit(stream.getId(), input, 
scoresPerStream.get(stream));
    +                }
    +            } else {
    +                LOG.debug("Input tuple [{}] generated NULL scores", input);
    +            }
    +        } catch(Exception e) {
    +            collector.reportError(e);
    +            collector.fail(input);
    +        }
    +    }
    +
    +    @Override
    +    public void declareOutputFields(OutputFieldsDeclarer declarer) {
    +        if (outFields != null) {
    --- End diff --
    
    In the `execute` it seems the results are always emitted, wont it fail when 
`outFields==null` ? 
    
    Also how are you ensuring that the streams declared in `declareStream` here 
are the ones returned by `runner.scoredTuplePerStream` ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to