You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@storm.apache.org by vesense <gi...@git.apache.org> on 2015/09/18 08:12:03 UTC

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

GitHub user vesense opened a pull request:

    https://github.com/apache/storm/pull/743

    [STORM-1052] TridentKafkaState uses new Kafka Producer API

    

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/vesense/storm patch-6

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/storm/pull/743.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #743
    
----
commit df417e0aa1439c0619e905ee0e624aa0a9717f1e
Author: Xin Wang <be...@163.com>
Date:   2015-09-18T05:58:06Z

    use new kafka producer api

commit 5c38b91accb9b9f5adfeea412bac0e457508f492
Author: Xin Wang <be...@163.com>
Date:   2015-09-18T06:05:21Z

    use new kafka producer configs

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r41099035
  
    --- Diff: external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java ---
    @@ -88,11 +88,11 @@ public static void main(String[] args) throws Exception {
     
         private  static Config getConfig(String brokerConnectionString) {
             Config conf = new Config();
    -        Map config = new HashMap();
             Properties props = new Properties();
    -        props.put("metadata.broker.list", brokerConnectionString);
    -        props.put("request.required.acks", "1");
    -        props.put("serializer.class", "kafka.serializer.StringEncoder");
    +        props.put("bootstrap.servers", brokerConnectionString);
    +        props.put("acks", "1");
    +        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
    --- End diff --
    
    These should be configurable


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/storm/pull/743


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r43946675
  
    --- Diff: external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java ---
    @@ -83,8 +81,8 @@ public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
                     topic = topicSelector.getTopic(tuple);
     
                     if(topic != null) {
    -                    producer.send(new KeyedMessage(topic, mapper.getKeyFromTuple(tuple),
    -                            mapper.getMessageFromTuple(tuple)));
    +                    producer.send(new ProducerRecord(topic, mapper.getKeyFromTuple(tuple),
    --- End diff --
    
    we should be checking for exceptions in the response returned by the producer


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r41099133
  
    --- Diff: external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java ---
    @@ -72,27 +73,27 @@ public void prepare(Map stormConf) {
             Map configMap = (Map) stormConf.get(KAFKA_BROKER_PROPERTIES);
             Properties properties = new Properties();
             properties.putAll(configMap);
    -        ProducerConfig config = new ProducerConfig(properties);
    -        producer = new Producer(config);
    +        producer = new KafkaProducer(properties);
         }
     
         public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
    -        String topic = null;
    -        for (TridentTuple tuple : tuples) {
    -            try {
    -                topic = topicSelector.getTopic(tuple);
    -
    -                if(topic != null) {
    -                    producer.send(new KeyedMessage(topic, mapper.getKeyFromTuple(tuple),
    -                            mapper.getMessageFromTuple(tuple)));
    -                } else {
    -                    LOG.warn("skipping key = " + mapper.getKeyFromTuple(tuple) + ", topic selector returned null.");
    -                }
    -            } catch (Exception ex) {
    -                String errorMsg = "Could not send message with key = " + mapper.getKeyFromTuple(tuple)
    -                        + " to topic = " + topic;
    -                LOG.warn(errorMsg, ex);
    -                throw new FailedException(errorMsg, ex);
    +        for (final TridentTuple tuple : tuples) {
    +            final String topic = topicSelector.getTopic(tuple);
    +            if(topic != null) {
    +                producer.send(new ProducerRecord(topic, mapper.getKeyFromTuple(tuple),
    --- End diff --
    
    We are moving from sync behavior to async here. Any reason for this? or can't we just call .get() on the send method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by zhuoliu <gi...@git.apache.org>.
Github user zhuoliu commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r43939104
  
    --- Diff: external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java ---
    @@ -88,11 +88,11 @@ public static void main(String[] args) throws Exception {
     
         private  static Config getConfig(String brokerConnectionString) {
             Config conf = new Config();
    -        Map config = new HashMap();
             Properties props = new Properties();
    -        props.put("metadata.broker.list", brokerConnectionString);
    -        props.put("request.required.acks", "1");
    -        props.put("serializer.class", "kafka.serializer.StringEncoder");
    +        props.put("bootstrap.servers", brokerConnectionString);
    +        props.put("acks", "1");
    +        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
    --- End diff --
    
    Agreed, since it is a test, making it not configurable should be OK.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by revans2 <gi...@git.apache.org>.
Github user revans2 commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-154545489
  
    I just upmerged and ran the tests on this manually.  Everything looks good for me. I am +1 on merging this in @zhuoliu @harshach @HeartSaVioR do either of you have any more comments you would like to see addressed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r43970097
  
    --- Diff: external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java ---
    @@ -83,8 +81,8 @@ public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
                     topic = topicSelector.getTopic(tuple);
     
                     if(topic != null) {
    -                    producer.send(new KeyedMessage(topic, mapper.getKeyFromTuple(tuple),
    -                            mapper.getMessageFromTuple(tuple)));
    +                    producer.send(new ProducerRecord(topic, mapper.getKeyFromTuple(tuple),
    --- End diff --
    
    Good catch. I will fix it. Thx.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r43349321
  
    --- Diff: external/storm-kafka/src/jvm/storm/kafka/trident/TridentKafkaState.java ---
    @@ -72,27 +73,27 @@ public void prepare(Map stormConf) {
             Map configMap = (Map) stormConf.get(KAFKA_BROKER_PROPERTIES);
             Properties properties = new Properties();
             properties.putAll(configMap);
    -        ProducerConfig config = new ProducerConfig(properties);
    -        producer = new Producer(config);
    +        producer = new KafkaProducer(properties);
         }
     
         public void updateState(List<TridentTuple> tuples, TridentCollector collector) {
    -        String topic = null;
    -        for (TridentTuple tuple : tuples) {
    -            try {
    -                topic = topicSelector.getTopic(tuple);
    -
    -                if(topic != null) {
    -                    producer.send(new KeyedMessage(topic, mapper.getKeyFromTuple(tuple),
    -                            mapper.getMessageFromTuple(tuple)));
    -                } else {
    -                    LOG.warn("skipping key = " + mapper.getKeyFromTuple(tuple) + ", topic selector returned null.");
    -                }
    -            } catch (Exception ex) {
    -                String errorMsg = "Could not send message with key = " + mapper.getKeyFromTuple(tuple)
    -                        + " to topic = " + topic;
    -                LOG.warn(errorMsg, ex);
    -                throw new FailedException(errorMsg, ex);
    +        for (final TridentTuple tuple : tuples) {
    +            final String topic = topicSelector.getTopic(tuple);
    +            if(topic != null) {
    +                producer.send(new ProducerRecord(topic, mapper.getKeyFromTuple(tuple),
    --- End diff --
    
    Sure, we should just call get() method to send. I will fix this. Thx.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by zhuoliu <gi...@git.apache.org>.
Github user zhuoliu commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-154461421
  
    Hi @vesense The DisruptorQueue test bug has been fixed in another JIRA. Could you upmerge and trigger the CI again? This pull request looks good to me, I am +1 to merge it in as long as tests pass.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by harshach <gi...@git.apache.org>.
Github user harshach commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-153882896
  
    @zhuoliu for trident we are sending the tuples as a batch and we would like to know if any of those tuples failed or any other errors occurred rather than wait for the callback. I think using get here is fine.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by zhuoliu <gi...@git.apache.org>.
Github user zhuoliu commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-153884995
  
    Thanks @harshach for the explanation! That makes good sense.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on a diff in the pull request:

    https://github.com/apache/storm/pull/743#discussion_r43349319
  
    --- Diff: external/storm-kafka/src/test/storm/kafka/TridentKafkaTopology.java ---
    @@ -88,11 +88,11 @@ public static void main(String[] args) throws Exception {
     
         private  static Config getConfig(String brokerConnectionString) {
             Config conf = new Config();
    -        Map config = new HashMap();
             Properties props = new Properties();
    -        props.put("metadata.broker.list", brokerConnectionString);
    -        props.put("request.required.acks", "1");
    -        props.put("serializer.class", "kafka.serializer.StringEncoder");
    +        props.put("bootstrap.servers", brokerConnectionString);
    +        props.put("acks", "1");
    +        props.put("key.serializer", "org.apache.kafka.common.serialization.StringSerializer");
    --- End diff --
    
    Sorry I took so long to respond. I think it is unnecessary for UT.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-152083005
  
    @harshach Could you take a look?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by vesense <gi...@git.apache.org>.
Github user vesense commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-141853555
  
    @HeartSaVioR  Can you please review this. Thanks.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] storm pull request: [STORM-1052] TridentKafkaState uses new Kafka ...

Posted by zhuoliu <gi...@git.apache.org>.
Github user zhuoliu commented on the pull request:

    https://github.com/apache/storm/pull/743#issuecomment-154546742
  
    +1


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---