You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@pulsar.apache.org by GitBox <gi...@apache.org> on 2021/12/15 03:56:04 UTC

[GitHub] [pulsar] fantapsody commented on a change in pull request #13316: PIP-121: Pulsar cluster level auto failover [WIP]

fantapsody commented on a change in pull request #13316:
URL: https://github.com/apache/pulsar/pull/13316#discussion_r769203656



##########
File path: pulsar-client-api/src/main/java/org/apache/pulsar/client/api/ServiceUrlProvider.java
##########
@@ -51,4 +51,11 @@
      */
     String getServiceUrl();
 
+    /**
+     * Close the resource that the provider allocated.
+     *
+     */
+    default void close() {

Review comment:
       Would it be better to make the interface inherit `AutoCloseable`?

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java
##########
@@ -0,0 +1,118 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.net.URLConnection;
+import java.nio.charset.StandardCharsets;
+import java.util.Timer;
+import java.util.TimerTask;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.commons.io.IOUtils;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.ServiceUrlProvider;
+
+@Slf4j
+public class ControlledClusterFailover implements ServiceUrlProvider {
+    private PulsarClient pulsarClient;
+    private volatile String currentPulsarServiceUrl;
+    private final String defaultServiceUrl;
+    private final URL pulsarUrlProvider;
+    private final Timer timer;
+
+    private ControlledClusterFailover(String defaultServiceUrl, String urlProvider) throws IOException {
+        this.defaultServiceUrl = defaultServiceUrl;
+        this.currentPulsarServiceUrl = defaultServiceUrl;
+        this.pulsarUrlProvider = new URL(urlProvider);
+        this.timer = new Timer("pulsar-service-provider");
+    }
+
+    @Override
+    public void initialize(PulsarClient client) {
+        this.pulsarClient = client;
+
+        // start to check service url every 30 seconds
+        this.timer.scheduleAtFixedRate(new TimerTask() {
+            @Override
+            public void run() {
+                String newPulsarUrl = null;
+                try {
+                    newPulsarUrl = fetchServiceUrl();
+                    if (!currentPulsarServiceUrl.equals(newPulsarUrl)) {
+                        log.info("Switch Pulsar service url from {} to {}", currentPulsarServiceUrl, newPulsarUrl);
+                        pulsarClient.updateServiceUrl(newPulsarUrl);
+                        currentPulsarServiceUrl = newPulsarUrl;
+                    }
+                } catch (IOException e) {
+                    log.error("Failed to switch new Pulsar URL, current: {}, new: {}",
+                            currentPulsarServiceUrl, newPulsarUrl, e);
+                }
+            }
+        }, 30_000, 30_000);
+    }
+
+    private String fetchServiceUrl() throws IOException {
+        // call the service to get service URL
+        InputStream inputStream = null;
+        try {
+            URLConnection conn = pulsarUrlProvider.openConnection();
+            inputStream = conn.getInputStream();
+            return new String(IOUtils.toByteArray(inputStream), StandardCharsets.UTF_8);

Review comment:
       Using the service url as the response body might not be very extensible.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java
##########
@@ -0,0 +1,234 @@
+/**
+ * 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 com.google.common.base.Strings;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.ServiceUrlProvider;
+
+@Slf4j
+public class AutoClusterFailover implements ServiceUrlProvider {
+    private PulsarClient pulsarClient;
+    private volatile String currentPulsarServiceUrl;
+    private final String primary;
+    private final String secondary;
+    private final long failoverDelayMs;
+    private final long switchBackDelayMs;
+    private final Timer timer;
+    private volatile long primaryFailedTimestamp;
+    private long primaryRecoverTimestamp;
+    private long secondaryFailedTimestamp;
+    private final int timeout = 30_000;
+
+    private AutoClusterFailover(String primary, String secondary, long failoverDelayMs, long switchBackDelayMs) {
+        this.primary = primary;
+        this.secondary = secondary;
+        this.failoverDelayMs = failoverDelayMs;
+        this.switchBackDelayMs = switchBackDelayMs;
+        this.currentPulsarServiceUrl = primary;
+        this.primaryFailedTimestamp = -1;
+        this.primaryRecoverTimestamp = -1;
+        this.secondaryFailedTimestamp = -1;
+        this.timer = new Timer("pulsar-service-provider");
+    }
+
+    @Override
+    public void initialize(PulsarClient client) {
+        this.pulsarClient = client;
+
+        // start to probe primary cluster active or not
+        this.timer.scheduleAtFixedRate(new TimerTask() {
+            @Override
+            public void run() {
+                // current pulsar serviceUrl is primary
+                if (currentPulsarServiceUrl.equals(primary)) {
+                    if (probeAvailable(primary, timeout)) {
+                        primaryFailedTimestamp = -1;
+                        return;
+                    }
+
+                    if (primaryFailedTimestamp == -1) {
+                        primaryFailedTimestamp = System.currentTimeMillis();
+                    } else if (System.currentTimeMillis() - primaryFailedTimestamp < failoverDelayMs) {
+                        return;
+                    } else if (probeAvailable(secondary, timeout)){
+                        log.info("Current Pulsar service is primary: {}, it has been down for {} ms, "
+                                        + "switch to the secondary service: {}. The first primary service down at: {}",
+                                currentPulsarServiceUrl, System.currentTimeMillis() - primaryFailedTimestamp,
+                                secondary, primaryFailedTimestamp);
+                        try {
+                            pulsarClient.updateServiceUrl(secondary);
+                            currentPulsarServiceUrl = secondary;
+                        } catch (PulsarClientException e) {
+                            log.error("Failed to switch to secondary service URL ", e);
+                        }
+                    } else {
+                        log.error("Current Pulsar service is primary: {}, it has been down for {} ms. "
+                                + "Failed to switch to secondary service URL, "
+                                + "because secondary service URL is not available",
+                                currentPulsarServiceUrl, System.currentTimeMillis() - primaryFailedTimestamp);
+                    }
+                } else { // current pulsar service URL is secondary, probe whether we need to switch back to primary.
+                    if (!probeAvailable(currentPulsarServiceUrl, timeout)) {
+                        if (secondaryFailedTimestamp == -1) {
+                            secondaryFailedTimestamp = System.currentTimeMillis();
+                        } else if (System.currentTimeMillis() - secondaryFailedTimestamp >= failoverDelayMs
+                                && probeAvailable(primary, timeout)) {
+                            log.info("Current Pulsar service is secondary: {}, it has been down for {} ms, "
+                                    + "switch back to primary service: {}", currentPulsarServiceUrl,
+                                    System.currentTimeMillis() - secondaryFailedTimestamp, primary);
+                            try {
+                                pulsarClient.updateServiceUrl(primary);
+                                currentPulsarServiceUrl = primary;
+                                return;
+                            } catch (PulsarClientException e) {
+                                log.error("Current Pulsar service is secondary: {}, it has been down for {} ms. "
+                                        + "Failed to switch to secondary service URL ",
+                                        currentPulsarServiceUrl,
+                                        System.currentTimeMillis() - secondaryFailedTimestamp, e);
+                            }
+                        }
+
+                        return;
+                    }
+
+                    secondaryFailedTimestamp = -1;
+
+                    if (!probeAvailable(primary, timeout)) {
+                        primaryRecoverTimestamp = -1;
+                        return;
+                    }
+                    if (primaryRecoverTimestamp == -1) {
+                        primaryRecoverTimestamp = System.currentTimeMillis();
+                    } else if (System.currentTimeMillis() - primaryRecoverTimestamp >= switchBackDelayMs) {
+                        log.info("Current Pulsar service is secondary: {}, "
+                                        + "the primary service: {} has been recover for {} ms, "
+                                        + "switch back to the primary service",
+                                currentPulsarServiceUrl, primary, System.currentTimeMillis() - primaryRecoverTimestamp);
+                        try {
+                            pulsarClient.updateServiceUrl(primary);
+                            currentPulsarServiceUrl = primary;
+                        } catch (PulsarClientException e) {
+                            log.error("Current Pulsar service is secondary: {}, "
+                                    + "failed to switch back to primary service URL ", currentPulsarServiceUrl, e);
+                        }
+                    }
+                }
+            }
+        }, 30_000, 30_000);
+
+    }
+
+    @Override
+    public String getServiceUrl() {
+        return this.currentPulsarServiceUrl;
+    }
+
+    @Override
+    public void close() {
+        this.timer.cancel();
+    }
+
+    private boolean probeAvailable(String url, int timeout) {
+        try {
+            String hostAndPort = parseHostAndPort(url);
+            if (Strings.isNullOrEmpty(hostAndPort)) {
+                return false;
+            }
+
+            Socket socket = new Socket();
+            socket.connect(new InetSocketAddress(parseHost(hostAndPort), parsePort(hostAndPort)), timeout);

Review comment:
       It might be better to make a request to the server to ensure it is actually in a good state.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java
##########
@@ -0,0 +1,234 @@
+/**
+ * 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 com.google.common.base.Strings;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.ServiceUrlProvider;
+
+@Slf4j
+public class AutoClusterFailover implements ServiceUrlProvider {
+    private PulsarClient pulsarClient;
+    private volatile String currentPulsarServiceUrl;
+    private final String primary;
+    private final String secondary;
+    private final long failoverDelayMs;
+    private final long switchBackDelayMs;
+    private final Timer timer;
+    private volatile long primaryFailedTimestamp;
+    private long primaryRecoverTimestamp;
+    private long secondaryFailedTimestamp;
+    private final int timeout = 30_000;
+
+    private AutoClusterFailover(String primary, String secondary, long failoverDelayMs, long switchBackDelayMs) {
+        this.primary = primary;
+        this.secondary = secondary;
+        this.failoverDelayMs = failoverDelayMs;
+        this.switchBackDelayMs = switchBackDelayMs;
+        this.currentPulsarServiceUrl = primary;
+        this.primaryFailedTimestamp = -1;
+        this.primaryRecoverTimestamp = -1;
+        this.secondaryFailedTimestamp = -1;
+        this.timer = new Timer("pulsar-service-provider");

Review comment:
       Would it be better to use `ScheduledThreadPoolExecutor` here?

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java
##########
@@ -0,0 +1,234 @@
+/**
+ * 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 com.google.common.base.Strings;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.ServiceUrlProvider;
+
+@Slf4j
+public class AutoClusterFailover implements ServiceUrlProvider {
+    private PulsarClient pulsarClient;
+    private volatile String currentPulsarServiceUrl;
+    private final String primary;
+    private final String secondary;
+    private final long failoverDelayMs;
+    private final long switchBackDelayMs;
+    private final Timer timer;
+    private volatile long primaryFailedTimestamp;
+    private long primaryRecoverTimestamp;
+    private long secondaryFailedTimestamp;
+    private final int timeout = 30_000;
+
+    private AutoClusterFailover(String primary, String secondary, long failoverDelayMs, long switchBackDelayMs) {
+        this.primary = primary;
+        this.secondary = secondary;
+        this.failoverDelayMs = failoverDelayMs;
+        this.switchBackDelayMs = switchBackDelayMs;
+        this.currentPulsarServiceUrl = primary;
+        this.primaryFailedTimestamp = -1;
+        this.primaryRecoverTimestamp = -1;
+        this.secondaryFailedTimestamp = -1;
+        this.timer = new Timer("pulsar-service-provider");
+    }
+
+    @Override
+    public void initialize(PulsarClient client) {
+        this.pulsarClient = client;
+
+        // start to probe primary cluster active or not
+        this.timer.scheduleAtFixedRate(new TimerTask() {
+            @Override
+            public void run() {
+                // current pulsar serviceUrl is primary
+                if (currentPulsarServiceUrl.equals(primary)) {
+                    if (probeAvailable(primary, timeout)) {
+                        primaryFailedTimestamp = -1;
+                        return;
+                    }
+
+                    if (primaryFailedTimestamp == -1) {
+                        primaryFailedTimestamp = System.currentTimeMillis();

Review comment:
       It might be better to use `System.nanoTime()` to calculate the time locally to avoid time drifts.

##########
File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java
##########
@@ -0,0 +1,234 @@
+/**
+ * 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 com.google.common.base.Strings;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.Timer;
+import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.pulsar.client.api.PulsarClient;
+import org.apache.pulsar.client.api.PulsarClientException;
+import org.apache.pulsar.client.api.ServiceUrlProvider;
+
+@Slf4j
+public class AutoClusterFailover implements ServiceUrlProvider {
+    private PulsarClient pulsarClient;
+    private volatile String currentPulsarServiceUrl;
+    private final String primary;
+    private final String secondary;
+    private final long failoverDelayMs;
+    private final long switchBackDelayMs;
+    private final Timer timer;
+    private volatile long primaryFailedTimestamp;
+    private long primaryRecoverTimestamp;
+    private long secondaryFailedTimestamp;
+    private final int timeout = 30_000;
+
+    private AutoClusterFailover(String primary, String secondary, long failoverDelayMs, long switchBackDelayMs) {
+        this.primary = primary;
+        this.secondary = secondary;
+        this.failoverDelayMs = failoverDelayMs;
+        this.switchBackDelayMs = switchBackDelayMs;
+        this.currentPulsarServiceUrl = primary;
+        this.primaryFailedTimestamp = -1;
+        this.primaryRecoverTimestamp = -1;
+        this.secondaryFailedTimestamp = -1;
+        this.timer = new Timer("pulsar-service-provider");
+    }
+
+    @Override
+    public void initialize(PulsarClient client) {
+        this.pulsarClient = client;
+
+        // start to probe primary cluster active or not
+        this.timer.scheduleAtFixedRate(new TimerTask() {
+            @Override
+            public void run() {
+                // current pulsar serviceUrl is primary
+                if (currentPulsarServiceUrl.equals(primary)) {
+                    if (probeAvailable(primary, timeout)) {

Review comment:
       How about failover after a succession of failures (3 for example, and should be configurable) to avoid unnecessary switches?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@pulsar.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org