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

    https://github.com/apache/storm/pull/254#discussion_r22678403
  
    --- Diff: 
external/storm-kafka/src/jvm/storm/kafka/ExponentialBackoffMsgRetryManager.java 
---
    @@ -0,0 +1,154 @@
    +/**
    + * 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;
    +
    +import java.util.PriorityQueue;
    +import java.util.Queue;
    +import java.util.SortedMap;
    +import java.util.TreeMap;
    +
    +public class ExponentialBackoffMsgRetryManager implements 
FailedMsgRetryManager {
    +
    +    private final long retryInitialDelayMs;
    +    private final double retryDelayMultiplier;
    +    private final long retryDelayMaxMs;
    +
    +    private Queue<MessageRetryRecord> waiting = new 
PriorityQueue<MessageRetryRecord>();
    +    private SortedMap<Long,MessageRetryRecord> records = new 
TreeMap<Long,MessageRetryRecord>();
    +
    +    public ExponentialBackoffMsgRetryManager(long retryInitialDelayMs, 
double retryDelayMultiplier, long retryDelayMaxMs) {
    +        this.retryInitialDelayMs = retryInitialDelayMs;
    +        this.retryDelayMultiplier = retryDelayMultiplier;
    +        this.retryDelayMaxMs = retryDelayMaxMs;
    +    }
    +
    +    @Override
    +    public void failed(Long offset) {
    +        MessageRetryRecord oldRecord = this.records.get(offset);
    +        MessageRetryRecord newRecord = oldRecord == null ?
    +                                       new MessageRetryRecord(offset) :
    +                                       oldRecord.createNextRetryRecord();
    +        this.records.put(offset, newRecord);
    +        this.waiting.add(newRecord);
    +    }
    +
    +    @Override
    +    public void acked(Long offset) {
    +        MessageRetryRecord record = this.records.remove(offset);
    +        if (record != null) {
    +            this.waiting.remove(record);
    +        }
    +    }
    +
    +    @Override
    +    public void retryStarted(Long offset) {
    +        MessageRetryRecord record = this.records.get(offset);
    +        if (record == null || !this.waiting.contains(record)) {
    +            throw new IllegalStateException("cannot retry a message that 
has not failed");
    +        } else {
    +            this.waiting.remove(record);
    +        }
    +    }
    +
    +    @Override
    +    public Long nextFailedMessageToRetry() {
    +        if (this.waiting.size() > 0) {
    +            MessageRetryRecord first = this.waiting.peek();
    +            if (System.currentTimeMillis() >= first.retryTimeUTC) {
    +                if (this.records.containsKey(first.offset)) {
    +                    return first.offset;
    +                } else {
    +                    // defensive programming - should be impossible
    +                    this.waiting.remove(first);
    +                    return nextFailedMessageToRetry();
    +                }
    +            }
    +        }
    +        return null;
    +    }
    +
    +    @Override
    +    public boolean shouldRetryMsg(Long offset) {
    +        MessageRetryRecord record = this.records.get(offset);
    +        return record != null &&
    +                this.waiting.contains(record) &&
    +                System.currentTimeMillis() >= record.retryTimeUTC;
    +    }
    +
    +    /**
    +     * A MessageRetryRecord holds the data of how many times a message has
    +     * failed and been retried, and when the last failure occurred.  It can
    +     * determine whether it is ready to be retried by employing an 
exponential
    +     * back-off calculation using config values stored in SpoutConfig:
    +     * <ul>
    +     *  <li>retryInitialDelayMs - time to delay before the first retry</li>
    +     *  <li>retryDelayMultiplier - multiplier by which to increase the 
delay for each subsequent retry</li>
    +     *  <li>retryDelayMaxMs - maximum retry delay (once this delay time is 
reached, subsequent retries will
    +     *                        delay for this amount of time every time)
    +     *  </li>
    +     * </ul>
    +     */
    +    class MessageRetryRecord implements Comparable<MessageRetryRecord> {
    +        private final long offset;
    +        private final int retryNum;
    +        private final long retryTimeUTC;
    +
    +        public MessageRetryRecord(long offset) {
    +            this(offset, 1);
    +        }
    +
    +        private MessageRetryRecord(long offset, int retryNum) {
    +            this.offset = offset;
    +            this.retryNum = retryNum;
    +            this.retryTimeUTC = System.currentTimeMillis() + 
calculateRetryDelay();
    +        }
    +
    +        /**
    +         * Create a MessageRetryRecord for the next retry that should 
occur after this one.
    +         * @return MessageRetryRecord with the next retry time, or null to 
indicate that another
    +         *         retry should not be performed.  The latter case can 
happen if we are about to
    +         *         run into the 
backtype.storm.Config.TOPOLOGY_MESSAGE_TIMEOUT_SECS in the Storm
    +         *         configuration.
    +         */
    +        public MessageRetryRecord createNextRetryRecord() {
    +            return new MessageRetryRecord(this.offset, this.retryNum + 1);
    +        }
    +
    +        private long calculateRetryDelay() {
    +            double delayMultiplier = Math.pow(retryDelayMultiplier, 
this.retryNum - 1);
    +            long delayThisRetryMs = (long) (retryInitialDelayMs * 
delayMultiplier);
    +            return Math.min(delayThisRetryMs, retryDelayMaxMs);
    +        }
    +
    +        @Override
    +        public boolean equals(Object other) {
    +            return (other instanceof MessageRetryRecord
    +                    && this.offset == ((MessageRetryRecord) other).offset);
    +        }
    +
    +        @Override
    +        public int hashCode() {
    +            return Long.valueOf(this.offset).hashCode();
    +        }
    +
    +        @Override
    +        public int compareTo(MessageRetryRecord other) {
    --- End diff --
    
    I know that technically this code is not incorrect, but it seems really 
confusing to me where you can have two MessageRetryRecords that are not equal 
to each other with the equals or hadCode methods but compareTo says that they 
are equal.  If we need to sort them for the priority queue, I would prefer to 
pass in a separate comparator instead of having the object be inconsistent with 
itself. But I don't see this as being big enough of an issue to block this from 
going in, as the class is package private.


---
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