[ 
https://issues.apache.org/jira/browse/STORM-405?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14075324#comment-14075324
 ] 

ASF GitHub Bot commented on STORM-405:
--------------------------------------

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

    https://github.com/apache/incubator-storm/pull/195#discussion_r15433274
  
    --- Diff: 
external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java ---
    @@ -0,0 +1,95 @@
    +/**
    + * 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 storm.kafka.trident;
    +
    +import backtype.storm.task.OutputCollector;
    +import backtype.storm.topology.FailedException;
    +import kafka.javaapi.producer.Producer;
    +import kafka.producer.KeyedMessage;
    +import kafka.producer.ProducerConfig;
    +import org.apache.commons.lang.Validate;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import storm.kafka.trident.mapper.TridentTupleToKafkaMapper;
    +import storm.kafka.trident.selector.KafkaTopicSelector;
    +import storm.trident.operation.TridentCollector;
    +import storm.trident.state.State;
    +import storm.trident.tuple.TridentTuple;
    +
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +
    +public class TridentKafkaState implements State {
    +    private static final Logger LOG = 
LoggerFactory.getLogger(TridentKafkaState.class);
    +
    +
    +    public static final String TOPIC = "topic";
    +    public static final String KAFKA_BROKER_PROPERTIES = 
"kafka.broker.properties";
    +
    +    private Producer producer;
    +    private OutputCollector collector;
    +
    +    private TridentTupleToKafkaMapper mapper;
    +    private KafkaTopicSelector topicSelector;
    +
    +    public TridentKafkaState 
withTridentTupleToKafkaMapper(TridentTupleToKafkaMapper mapper) {
    +        this.mapper = mapper;
    +        return this;
    +    }
    +
    +    public TridentKafkaState withKafkaTopicSelector(KafkaTopicSelector 
selector) {
    +        this.topicSelector = selector;
    +        return this;
    +    }
    +
    +    @Override
    +    public void beginCommit(Long txid) {
    +        LOG.debug("beginCommit is Noop.");
    +    }
    +
    +    @Override
    +    public void commit(Long txid) {
    +        LOG.debug("commit is Noop.");
    +    }
    +
    +    public void prepare(Map stormConf) {
    +        Validate.notNull(mapper, "mapper can not be null");
    +        Validate.notNull(topicSelector, "topicSelector can not be null");
    +        Map configMap = (Map) stormConf.get(KAFKA_BROKER_PROPERTIES);
    +        Properties properties = new Properties();
    +        properties.putAll(configMap);
    +        ProducerConfig config = new ProducerConfig(properties);
    +        producer = new Producer(config);
    +    }
    +
    +    public void updateState(List<TridentTuple> tuples, TridentCollector 
collector) {
    +        for (TridentTuple tuple : tuples) {
    +            try {
    +                for(String topic : topicSelector.getTopics(tuple)) {
    +                    producer.send(new KeyedMessage(topic, 
mapper.getKeyFromTuple(tuple),
    +                            mapper.getMessageFromTuple(tuple)));
    +                }
    +            } catch (Exception ex) {
    --- End diff --
    
    do we really need to be able to handle multiple topics here? if we have 2 
topics and we can successfully send the message to the first one but fail to 
send it to the second one we'd throw a FailedException resulting in 
inconsistencies between the topics. 
    Also the error message does not indicate which topic actually failed (and 
the single quote is not closed ;) )


> Add kafka trident state so messages can be sent to kafka topics
> ---------------------------------------------------------------
>
>                 Key: STORM-405
>                 URL: https://issues.apache.org/jira/browse/STORM-405
>             Project: Apache Storm (Incubating)
>          Issue Type: Improvement
>    Affects Versions: 0.9.3-incubating
>            Reporter: Parth Brahmbhatt
>            Priority: Minor
>             Fix For: 0.9.3-incubating
>
>
> Currently storm has a bolt for writing to kafka but we have no implementation 
> of trident state. We need a trident state implementation that allows wrting 
> tuples directly to kafka topics as part of trident topology. 



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to