-
Notifications
You must be signed in to change notification settings - Fork 334
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
SAMZA-168; add exponential backoff to kafka system admin
- Loading branch information
Showing
6 changed files
with
107 additions
and
114 deletions.
There are no files selected for viewing
45 changes: 45 additions & 0 deletions
45
samza-core/src/main/scala/org/apache/samza/util/ExponentialSleepStrategy.scala
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,45 @@ | ||
/* | ||
* | ||
* 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.samza.util | ||
|
||
class ExponentialSleepStrategy( | ||
backOffMultiplier: Double = 2.0, | ||
initialDelayMs: Long = 100, | ||
maximumDelayMs: Long = 10000) { | ||
|
||
require(backOffMultiplier > 1.0, "backOffMultiplier must be greater than 1") | ||
require(initialDelayMs > 0, "initialDelayMs must be positive") | ||
require(maximumDelayMs >= initialDelayMs, "maximumDelayMs must be >= initialDelayMs") | ||
|
||
var previousDelay = 0L | ||
|
||
def sleep() = { | ||
val nextDelay = getNextDelay(previousDelay) | ||
Thread.sleep(nextDelay) | ||
previousDelay = nextDelay | ||
} | ||
|
||
def getNextDelay(previousDelay: Long): Long = { | ||
val nextDelay = (previousDelay * backOffMultiplier).asInstanceOf[Long] | ||
math.min(math.max(initialDelayMs, nextDelay), maximumDelayMs) | ||
} | ||
} |
57 changes: 57 additions & 0 deletions
57
samza-core/src/test/scala/org/apache/samza/util/TestExponentialSleepStrategy.scala
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,57 @@ | ||
/* | ||
* | ||
* 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.samza.util | ||
|
||
import org.junit.Assert._ | ||
import org.junit.Test | ||
import org.apache.samza.util.ExponentialSleepStrategy | ||
|
||
class TestExponentialSleepStrategy { | ||
|
||
@Test def testGetNextDelayReturnsIncrementalDelay() = { | ||
val st = new ExponentialSleepStrategy | ||
var nextDelay = st.getNextDelay(0L) | ||
assertEquals(nextDelay, 100L) | ||
nextDelay = st.getNextDelay(nextDelay) | ||
assertEquals(nextDelay, 200L) | ||
nextDelay = st.getNextDelay(nextDelay) | ||
assertEquals(nextDelay, 400L) | ||
} | ||
|
||
@Test def testGetNextDelayReturnsMaximumDelayWhenDelayCapReached() = { | ||
val st = new ExponentialSleepStrategy | ||
var nextDelay = st.getNextDelay(6400L) | ||
assertEquals(nextDelay, 10000L) | ||
nextDelay = st.getNextDelay(nextDelay) | ||
assertEquals(nextDelay, 10000L) | ||
} | ||
|
||
@Test def testSleepStrategyIsConfigurable() = { | ||
val st = new ExponentialSleepStrategy(backOffMultiplier = 3.0, initialDelayMs = 10) | ||
var nextDelay = st.getNextDelay(0L) | ||
assertEquals(nextDelay, 10L) | ||
nextDelay = st.getNextDelay(nextDelay) | ||
assertEquals(nextDelay, 30L) | ||
nextDelay = st.getNextDelay(nextDelay) | ||
assertEquals(nextDelay, 90L) | ||
} | ||
} |
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
65 changes: 0 additions & 65 deletions
65
samza-kafka/src/main/scala/org/apache/samza/util/ExponentialThreadSleepStrategy.scala
This file was deleted.
Oops, something went wrong.
47 changes: 0 additions & 47 deletions
47
samza-kafka/src/test/scala/org/apache/samza/util/TestExponentialThreadSleepStrategy.scala
This file was deleted.
Oops, something went wrong.