Github user HeartSaVioR commented on a diff in the pull request:
https://github.com/apache/storm/pull/1683#discussion_r78543845
--- Diff:
external/storm-kafka/src/jvm/org/apache/storm/kafka/trident/TridentKafkaState.java
---
@@ -73,30 +74,35 @@ public void prepare(Properties options) {
public void updateState(List<TridentTuple> tuples, TridentCollector
collector) {
String topic = null;
- for (TridentTuple tuple : tuples) {
- try {
+ try {
+ List<Future<RecordMetadata>> futures = new
ArrayList<>(tuples.size());
+ for (TridentTuple tuple : tuples) {
topic = topicSelector.getTopic(tuple);
if(topic != null) {
Future<RecordMetadata> result = producer.send(new
ProducerRecord(topic,
mapper.getKeyFromTuple(tuple),
mapper.getMessageFromTuple(tuple)));
- try {
- result.get();
- } catch (ExecutionException e) {
- String errorMsg = "Could not retrieve result for
message with key = "
- + mapper.getKeyFromTuple(tuple) + " from
topic = " + topic;
- LOG.error(errorMsg, e);
- throw new FailedException(errorMsg, e);
- }
+ futures.add(result);
} 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 (int i = 0 ; i < futures.size(); i++) {
+ Future<RecordMetadata> future = futures.get(i);
+ try {
+ future.get();
+ } catch (ExecutionException e) {
+ String errorMsg = "Could not retrieve result for
message with key = "
+ + mapper.getKeyFromTuple(tuples.get(i)) + "
from topic = " + topic;
+ LOG.error(errorMsg, e);
+ throw new FailedException(errorMsg, e);
--- End diff --
Since it already sends multiple requests, we need to log other errors as
well and throw FailedException with summarized message (containing all errors)
if any.
---
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 [email protected] or file a JIRA ticket
with INFRA.
---