Github user hmcl commented on a diff in the pull request: https://github.com/apache/storm/pull/1924#discussion_r99869978 --- Diff: external/storm-kafka-client/src/main/java/org/apache/storm/kafka/spout/KafkaSpoutRetryExponentialBackoff.java --- @@ -54,59 +56,92 @@ private static class RetryEntryTimeStampComparator implements Serializable, Comparator<RetrySchedule> { @Override public int compare(RetrySchedule entry1, RetrySchedule entry2) { - return Long.valueOf(entry1.nextRetryTimeNanos()).compareTo(entry2.nextRetryTimeNanos()); + int result = Long.valueOf(entry1.nextRetryTimeMs()).compareTo(entry2.nextRetryTimeMs()); + + if(result == 0) { + //TreeSet uses compareTo instead of equals() for the Set contract + //Ensure that we can save two retry schedules with the same timestamp + result = entry1.hashCode() - entry2.hashCode(); + } + return result; } } private class RetrySchedule { private final KafkaSpoutMessageId msgId; - private long nextRetryTimeNanos; + private long nextRetryTimeMs; public RetrySchedule(KafkaSpoutMessageId msgId, long nextRetryTime) { this.msgId = msgId; - this.nextRetryTimeNanos = nextRetryTime; + this.nextRetryTimeMs = nextRetryTime; LOG.debug("Created {}", this); } public void setNextRetryTime() { - nextRetryTimeNanos = nextTime(msgId); + nextRetryTimeMs = nextTime(msgId); LOG.debug("Updated {}", this); } - public boolean retry(long currentTimeNanos) { - return nextRetryTimeNanos <= currentTimeNanos; + public boolean retry(long currentTimeMs) { + return nextRetryTimeMs <= currentTimeMs; } @Override public String toString() { return "RetrySchedule{" + "msgId=" + msgId + - ", nextRetryTime=" + nextRetryTimeNanos + + ", nextRetryTimeMs=" + nextRetryTimeMs + '}'; } + @Override + public int hashCode() { + int hash = 5; + hash = 29 * hash + Objects.hashCode(this.msgId); + return hash; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + final RetrySchedule other = (RetrySchedule) obj; + if (!Objects.equals(this.msgId, other.msgId)) { + return false; + } + return true; + } + public KafkaSpoutMessageId msgId() { return msgId; } - public long nextRetryTimeNanos() { - return nextRetryTimeNanos; + public long nextRetryTimeMs() { + return nextRetryTimeMs; } } public static class TimeInterval implements Serializable { - private final long lengthNanos; - private final long length; - private final TimeUnit timeUnit; + private final long lengthMs; /** * @param length length of the time interval in the units specified by {@link TimeUnit} - * @param timeUnit unit used to specify a time interval on which to specify a time unit + * @param timeUnit unit used to specify a time interval on which to specify a time unit. Smallest supported unit is milliseconds */ public TimeInterval(long length, TimeUnit timeUnit) { - this.length = length; - this.timeUnit = timeUnit; - this.lengthNanos = timeUnit.toNanos(length); + + if(timeUnit == TimeUnit.MICROSECONDS || timeUnit == TimeUnit.NANOSECONDS) { + throw new IllegalArgumentException("TimeInterval does not support time units smaller than milliseconds"); + } --- End diff -- same comment as above
--- 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. ---