-
Notifications
You must be signed in to change notification settings - Fork 3.6k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[PIP-130] Apply redelivery backoff policy for ack timeout (#13707)
### Motivation This pull request is the specific implementation plan of PIP-130 ### Modifications *Apply the message redelivery policy for the ack timeout.* *Alert NegativeAckBackoff interface to RedeliveryBackoff.* *Expose AckTimeoutRedeliveryBackoff in ConsumerBulider.*
- Loading branch information
1 parent
5bb4b75
commit 74b5d50
Showing
21 changed files
with
946 additions
and
314 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
250 changes: 207 additions & 43 deletions
250
...broker/src/test/java/org/apache/pulsar/client/impl/UnAcknowledgedMessagesTimeoutTest.java
Large diffs are not rendered by default.
Oops, something went wrong.
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
94 changes: 94 additions & 0 deletions
94
pulsar-client/src/main/java/org/apache/pulsar/client/impl/MultiplierRedeliveryBackoff.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,94 @@ | ||
/** | ||
* 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 org.apache.pulsar.client.impl; | ||
|
||
import static com.google.common.base.Preconditions.checkArgument; | ||
import org.apache.pulsar.client.api.RedeliveryBackoff; | ||
|
||
/** | ||
* MultiplierRedeliveryBackoff. | ||
*/ | ||
public class MultiplierRedeliveryBackoff implements RedeliveryBackoff { | ||
|
||
private final long minDelayMs; | ||
private final long maxDelayMs; | ||
private final double multiplier; | ||
private final int maxMultiplierPow; | ||
|
||
private MultiplierRedeliveryBackoff(long minDelayMs, long maxDelayMs, double multiplier) { | ||
this.minDelayMs = minDelayMs; | ||
this.maxDelayMs = maxDelayMs; | ||
this.multiplier = multiplier; | ||
maxMultiplierPow = (int) (Math.log((double) maxDelayMs / minDelayMs) / Math.log(multiplier)) + 1; | ||
} | ||
|
||
public static MultiplierRedeliveryBackoff.MultiplierRedeliveryBackoffBuilder builder() { | ||
return new MultiplierRedeliveryBackoff.MultiplierRedeliveryBackoffBuilder(); | ||
} | ||
|
||
public long getMinDelayMs() { | ||
return this.minDelayMs; | ||
} | ||
|
||
public long getMaxDelayMs() { | ||
return this.maxDelayMs; | ||
} | ||
|
||
@Override | ||
public long next(int redeliveryCount) { | ||
if (redeliveryCount <= 0 || minDelayMs <= 0) { | ||
return this.minDelayMs; | ||
} | ||
if (redeliveryCount > maxMultiplierPow) { | ||
return this.maxDelayMs; | ||
} | ||
return Math.min((long) (minDelayMs * Math.pow(multiplier, redeliveryCount)), this.maxDelayMs); | ||
} | ||
|
||
/** | ||
* Builder of MultiplierRedeliveryBackoff. | ||
*/ | ||
public static class MultiplierRedeliveryBackoffBuilder { | ||
private long minDelayMs = 1000 * 10; | ||
private long maxDelayMs = 1000 * 60 * 10; | ||
private double multiplier = 2.0; | ||
|
||
public MultiplierRedeliveryBackoffBuilder minDelayMs(long minDelayMs) { | ||
this.minDelayMs = minDelayMs; | ||
return this; | ||
} | ||
|
||
public MultiplierRedeliveryBackoffBuilder maxDelayMs(long maxDelayMs) { | ||
this.maxDelayMs = maxDelayMs; | ||
return this; | ||
} | ||
|
||
public MultiplierRedeliveryBackoffBuilder multiplier(double multiplier) { | ||
this.multiplier = multiplier; | ||
return this; | ||
} | ||
|
||
public MultiplierRedeliveryBackoff build() { | ||
checkArgument(minDelayMs >= 0, "min delay time must be >= 0"); | ||
checkArgument(maxDelayMs >= minDelayMs, "maxDelayMs must be >= minDelayMs"); | ||
checkArgument(multiplier > 1, "multiplier must be > 1"); | ||
return new MultiplierRedeliveryBackoff(minDelayMs, maxDelayMs, multiplier); | ||
} | ||
} | ||
} |
Oops, something went wrong.