zookeeper-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "ASF GitHub Bot (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (ZOOKEEPER-2849) Quorum port binding needs exponential back-off retry
Date Mon, 20 Nov 2017 21:46:00 GMT

    [ https://issues.apache.org/jira/browse/ZOOKEEPER-2849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16259906#comment-16259906
] 

ASF GitHub Bot commented on ZOOKEEPER-2849:
-------------------------------------------

Github user brian-lininger commented on a diff in the pull request:

    https://github.com/apache/zookeeper/pull/419#discussion_r152119989
  
    --- Diff: src/java/main/org/apache/zookeeper/server/quorum/ExponentialBackoffStrategy.java
---
    @@ -0,0 +1,192 @@
    +package org.apache.zookeeper.server.quorum;
    +
    +/**
    + * A {@link BackoffStrategy} that increases the wait time between each
    + * interval up to the configured maximum wait time.
    + */
    +public class ExponentialBackoffStrategy implements BackoffStrategy {
    +
    +    // Sensible default values to use if not set by the user
    +    private static final long DEFAULT_INITIAL_BACKOFF_MILLIS = 500L;  // 0.5s
    +    private static final long DEFAULT_MAX_BACKOFF_MILLIS = 30_000L;  // 30s
    +    private static final long DEFAULT_MAX_ELAPSED_MILLIS = 5 * 60_000L; // 10m
    +    private static final double DEFAULT_BACKOFF_MULTIPLE = 1.5;
    +
    +    // internal values per instance
    +    private final long initialBackoffMillis;
    +    private final long maxBackoffMillis;
    +    private final long maxElapsedMillis;
    +    private final double backoffMultiple;
    +
    +    // internal state
    +    private long nextWait;
    +    private long totalElapsed;
    +    private final boolean limitBackoffMillis;
    +    private final boolean checkElapsedTime;
    +
    +    /**
    +     * Construct a new instance.
    +     * @param builder the Builder to use for configuring this BackoffStrategy
    +     */
    +    private ExponentialBackoffStrategy(Builder builder) {
    +        this.initialBackoffMillis = builder.initialBackoffMillis;
    +        this.maxBackoffMillis = builder.maxBackoffMillis;
    +        this.maxElapsedMillis = builder.maxElapsedMillis;
    +        this.backoffMultiple = builder.backoffMultiple;
    +
    +        if(maxBackoffMillis == -1) {
    +            limitBackoffMillis = false;
    +        } else {
    +            limitBackoffMillis = true;
    +        }
    +
    +        if(maxElapsedMillis == -1) {
    +            checkElapsedTime = false;
    +        } else {
    +            checkElapsedTime = true;
    +        }
    +
    +        reset();
    +    }
    +
    +
    +    @Override
    +    public long nextWaitMillis() throws IllegalStateException {
    +        // check if we have exceeded the allowed maximum elapsed time
    +        if(checkElapsedTime && totalElapsed > maxElapsedMillis) {
    +            return BackoffStrategy.STOP;
    +        }
    +
    +        long waitMillis = nextWait;
    +
    +        // calculate the next wait milliseconds
    +        nextWait = Math.round(nextWait * backoffMultiple);
    +
    +        // don't exceed the allowed maximum wait milliseconds
    +        // if a maximum was configured
    +        if(limitBackoffMillis && nextWait > maxBackoffMillis) {
    +            nextWait = maxBackoffMillis;
    +        }
    +
    +        // track total elapsed time, even if we don't wait we have to assume
    +        // that some amount of time passed outside of the wait or we'll never
    +        // hit the elapsed time limit
    +        totalElapsed += waitMillis != 0 ? waitMillis : 1L;
    +        return waitMillis;
    +    }
    +
    +    @Override
    +    public void reset() {
    +        nextWait = this.initialBackoffMillis;
    +        totalElapsed = 0;
    +    }
    +
    +    /**
    +     *
    +     * @return a new {@link Builder} instance.
    +     */
    +    public static Builder builder() {
    +        return new Builder();
    +    }
    +
    +    /**
    +     * Builder for instances of {@link ExponentialBackoffStrategy}.
    +     */
    +    public static final class Builder {
    --- End diff --
    
    That should be doable.


> Quorum port binding needs exponential back-off retry
> ----------------------------------------------------
>
>                 Key: ZOOKEEPER-2849
>                 URL: https://issues.apache.org/jira/browse/ZOOKEEPER-2849
>             Project: ZooKeeper
>          Issue Type: Improvement
>          Components: quorum
>    Affects Versions: 3.4.6, 3.5.3
>            Reporter: Brian Lininger
>            Assignee: Brian Lininger
>            Priority: Minor
>
> Recently we upgraded the AWS instance type we use for running out ZooKeeper nodes, and
by doing so we're intermittently hitting an issue where ZooKeeper cannot bind to the server
election port because the IP is incorrect.  This is due to name resolution in Route53 not
being in sync when ZooKeeper starts on the more powerful EC2 instances.  Currently in QuorumCnxManager.Listener,
we only attempt to bind 3 times with a 1s sleep between retries, which is not long enough.
 
> I'm proposing to change this to follow an exponential back-off type strategy where each
failed attempt causes a longer sleep between retry attempts.  This would allow for Zookeeper
to gracefully recover when the host is misconfigured, and subsequently corrected, without
requiring the process to be restarted while also minimizing the impact to the running instance.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Mime
View raw message