merlimat commented on a change in pull request #13316: URL: https://github.com/apache/pulsar/pull/13316#discussion_r773224336
########## File path: pulsar-client-api/src/main/java/org/apache/pulsar/client/api/AutoClusterFailoverBuilder.java ########## @@ -0,0 +1,74 @@ +/** + * 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.api; + +import java.util.concurrent.TimeUnit; +import org.apache.pulsar.common.classification.InterfaceAudience; +import org.apache.pulsar.common.classification.InterfaceStability; + +/** + * {@link AutoClusterFailoverBuilder} is used to configure and create instance of {@link ServiceUrlProvider} + * + * @since 2.10.0 + */ [email protected] [email protected] +public interface AutoClusterFailoverBuilder { + /** + * Set the primary service url. + * + * @param primary + * @return + */ + AutoClusterFailoverBuilder primary(String primary); + + /** + * Set the secondary service url. + * + * @param secondary + * @return + */ + AutoClusterFailoverBuilder secondary(String secondary); Review comment: We could take a list/set of serviceUrls, just to make it more general ########## 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: 👍 We should use a JSON response 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(); + } 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); + socket.close(); + return true; + } catch (Exception e) { + return false; + } + } + + private static String parseHostAndPort(String url) { + if (Strings.isNullOrEmpty(url) || !url.startsWith("pulsar")) { + throw new IllegalArgumentException("'" + url + "' isn't an Pulsar service URL"); + } + + int uriSeparatorPos = url.indexOf("://"); + if (uriSeparatorPos == -1) { + throw new IllegalArgumentException("'" + url + "' isn't an URI."); + } + return url.substring(uriSeparatorPos + 3); + } + + private static String parseHost(String hostAndPort) { + int portSeparatorPos = hostAndPort.indexOf(":"); + if (portSeparatorPos == -1) { + throw new IllegalArgumentException("'" + hostAndPort + "' isn't an URI."); + } + return hostAndPort.substring(0, portSeparatorPos); + } + + private static Integer parsePort(String hostAndPort) { + int portSeparatorPos = hostAndPort.indexOf(":"); + if (portSeparatorPos == -1) { + throw new IllegalArgumentException("'" + hostAndPort + "' isn't an URI."); + } + return Integer.valueOf(hostAndPort.substring(portSeparatorPos+1)); + } Review comment: I think `URI` could be better here, because `URL` assumes a "known" protocol scheme. ########## File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/AutoClusterFailover.java ########## @@ -0,0 +1,232 @@ +/** + * 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 org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.base.Strings; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.pulsar.client.api.AutoClusterFailoverBuilder; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.PulsarClientException; +import org.apache.pulsar.client.api.ServiceUrlProvider; + +@Slf4j +@Data +public class AutoClusterFailover implements ServiceUrlProvider { + private PulsarClient pulsarClient; + private volatile String currentPulsarServiceUrl; + private final String primary; + private final String secondary; + private final long failoverDelayNs; + private final long switchBackDelayNs; + private final ScheduledExecutorService executor; + private long recoverTimestamp; + private long failedTimestamp; + private final int interval = 30_000; + private final int TIMEOUT = 30_000; + + private AutoClusterFailover(String primary, String secondary, long failoverDelayNs, long switchBackDelayNs) { + this.primary = primary; + this.secondary = secondary; + this.failoverDelayNs = failoverDelayNs; + this.switchBackDelayNs = switchBackDelayNs; + this.currentPulsarServiceUrl = primary; + this.recoverTimestamp = -1; + this.failedTimestamp = -1; + this.executor = Executors.newSingleThreadScheduledExecutor( + new DefaultThreadFactory("pulsar-service-provider")); + } + + @Override + public void initialize(PulsarClient client) { + this.pulsarClient = client; + + // start to probe primary cluster active or not + this.executor.scheduleAtFixedRate(catchingAndLoggingThrowables(() -> { + if (currentPulsarServiceUrl.equals(primary)) { + // current service url is primary, probe whether it is down + probeAndUpdateServiceUrl(secondary); + } else { + // current service url is secondary, probe whether it is down + probeAndUpdateServiceUrl(primary); + // secondary cluster is up, check whether need to switch back to primary + probeAndCheckSwitchBack(primary); + } + }), getInterval(), getInterval(), TimeUnit.MILLISECONDS); + + } + + @Override + public String getServiceUrl() { + return this.currentPulsarServiceUrl; + } + + @Override + public void close() { + this.executor.shutdown(); + } + + boolean probeAvailable(String url) { + 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: This will check for TCP connectivity, which is a good start, though there can be many cases in which it will give the false impression that cluster is healthy: * We're connecting to a Pulsar proxy, but there are no available brokers * Using Istio on server side, which always accept the connection even if the broker is in a bad state * We might have deadlocks in (all) brokers and while the connections get accepted, the brokers are not able to serve them. We should consider to have a more in depth test to: 1. Check that we can authenticate with brokers 2. (Maybe) Estimate how many brokers are up & healthy in the cluster ########## File path: pulsar-client/src/main/java/org/apache/pulsar/client/impl/ControlledClusterFailover.java ########## @@ -0,0 +1,123 @@ +/** + * 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 org.apache.pulsar.common.util.Runnables.catchingAndLoggingThrowables; +import com.google.common.base.Strings; +import io.netty.util.concurrent.DefaultThreadFactory; +import java.io.IOException; +import java.io.InputStream; +import java.net.URL; +import java.net.URLConnection; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import lombok.Data; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.io.IOUtils; +import org.apache.pulsar.client.api.ControlledClusterFailoverBuilder; +import org.apache.pulsar.client.api.PulsarClient; +import org.apache.pulsar.client.api.ServiceUrlProvider; + +@Slf4j +@Data +public class ControlledClusterFailover implements ServiceUrlProvider { + private PulsarClient pulsarClient; + private volatile String currentPulsarServiceUrl; + private final URL pulsarUrlProvider; + private final ScheduledExecutorService executor; + private final int interval = 30_000; Review comment: We should take the poll interval as a config in the builder -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
