yifan-c commented on code in PR #189: URL: https://github.com/apache/cassandra-sidecar/pull/189#discussion_r1965972683
########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/health/SidecarPeerDownDetectorIntegrationTest.java: ########## @@ -0,0 +1,201 @@ +/* + * 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.cassandra.sidecar.health; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import com.google.inject.name.Named; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.SidecarPeerHealthConfiguration; +import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarPeerHealthConfigurationImpl; +import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthMonitorTask; +import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthProvider; +import org.apache.cassandra.sidecar.server.Server; +import org.apache.cassandra.sidecar.testing.InnerDcTokenAdjacentPeerTestProvider.TestSidecarHostInfo; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.sidecar.testing.SharedClusterSidecarIntegrationTestBase; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; + +class SidecarPeerDownDetectorIntegrationTest extends SharedClusterSidecarIntegrationTestBase +{ + + private static final Logger LOGGER = LoggerFactory.getLogger(SidecarPeerDownDetectorIntegrationTest.class); + + List<TestSidecarHostInfo> sidecarServerList = new ArrayList<>(); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return new ClusterBuilderConfiguration().nodesPerDc(3); + } + + @Override + protected void startSidecar(ICluster<? extends IInstance> cluster) throws InterruptedException + { + Supplier<List<TestSidecarHostInfo>> supplier = () -> sidecarServerList; + PeersModule peersModule = new PeersModule(supplier); + for (IInstance cassandraInstance : cluster) + { + // Provider de una lista de Sidecar servers + LOGGER.info("Starting Sidecar instance for Cassandra instance {}", + cassandraInstance.config().num()); + Server server = startSidecarWithInstances(List.of(cassandraInstance), peersModule); Review Comment: Although we run each integration test in its own JVM, can we still ensure the server is properly closed? ########## conf/sidecar.yaml: ########## @@ -236,6 +236,36 @@ healthcheck: initial_delay: 0ms execute_interval: 30s +# Sidecar Peer Health Monitor settings +# Enables a periodic task checking for the health of adjacent Sidecar peers in the token ring +sidecar_peer_health: + enabled: true Review Comment: can it be off by default? It is only needed for CDC publishing, which is an optional feature ########## conf/sidecar.yaml: ########## @@ -236,6 +236,36 @@ healthcheck: initial_delay: 0ms execute_interval: 30s +# Sidecar Peer Health Monitor settings +# Enables a periodic task checking for the health of adjacent Sidecar peers in the token ring +sidecar_peer_health: + enabled: true + execute_interval: 30s + health_check_retries: 5 + health_check_retry_delay: 10s + +# Sidecar client settings used to interact with other sidecars +sidecar_client: + use_ssl: true + request_timeout: 1s + request_idle_timeout: 1s + connection_pool_max_size: 10 + connection_pool_clearing_period: 10s + connection_pool_event_loop_size: 10 + connection_pool_max_wait_queue_size: 10 + max_retries: 3 + retry_delay: 1s + max_retry_delay: 2s Review Comment: I would be easier for operators to parse this content if you add descriptions to each configuration. ########## server/src/main/java/org/apache/cassandra/sidecar/codecs/SidecarInstanceCodecs.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.cassandra.sidecar.codecs; + +import com.google.inject.Singleton; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.client.SidecarInstanceImpl; + +/** + * Codecs for Sidecar instances + */ +@Singleton +public class SidecarInstanceCodecs implements MessageCodec<SidecarInstance, SidecarInstance> +{ + @Override + public void encodeToWire(Buffer buf, SidecarInstance instance) + { + buf.appendInt(instance.port()); + CommonCodecs.STRING.encodeToWire(buf, instance.hostname()); + } + + @Override + public SidecarInstance decodeFromWire(int pos, Buffer buf) + { + final int port = buf.getInt(pos); + pos += 4; // advance 4 bytes after reading int + return new SidecarInstanceImpl(CommonCodecs.STRING.decodeFromWire(pos, buf), port); + } Review Comment: Just declare the string codec in the scope of this class. As commented above, there is no need to have a common codecs group. ```suggestion private static final StringMessageCodec STRING_MESSAGE_CODEC = new StringMessageCodec(); @Override public void encodeToWire(Buffer buf, SidecarInstance instance) { buf.appendInt(instance.port()); STRING_MESSAGE_CODEC.encodeToWire(buf, instance.hostname()); } @Override public SidecarInstance decodeFromWire(int pos, Buffer buf) { int port = buf.getInt(pos); pos += 4; // advance 4 bytes after reading int return new SidecarInstanceImpl(STRING_MESSAGE_CODEC.decodeFromWire(pos, buf), port); } ``` ########## server/src/main/java/org/apache/cassandra/sidecar/coordination/InnerDcTokenAdjacentPeerProvider.java: ########## @@ -0,0 +1,270 @@ +/* + * 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.cassandra.sidecar.coordination; + +import java.math.BigInteger; +import java.net.UnknownHostException; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.Host; +import com.datastax.driver.core.KeyspaceMetadata; +import com.datastax.driver.core.Metadata; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.cassandra.sidecar.cluster.InstancesMetadata; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.client.SidecarInstanceImpl; +import org.apache.cassandra.sidecar.common.server.cluster.locator.Token; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.common.server.dns.DnsResolver; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; + + +import static org.apache.cassandra.sidecar.config.yaml.CassandraInputValidationConfigurationImpl.DEFAULT_FORBIDDEN_KEYSPACES; + + +/** + * Return Sidecar(s) adjacent to current Sidecar in the token ring within the same datacenter. + */ +@Singleton +public class InnerDcTokenAdjacentPeerProvider implements SidecarPeerProvider +{ + private static final Logger LOGGER = LoggerFactory.getLogger(InnerDcTokenAdjacentPeerProvider.class); + + protected final InstancesMetadata instancesMetadata; + private final CassandraClientTokenRingProvider cassandraClientTokenRingProvider; + private final ServiceConfiguration serviceConfiguration; + private final DnsResolver dnsResolver; + + @Inject + public InnerDcTokenAdjacentPeerProvider(InstancesMetadata instancesMetadata, + CassandraClientTokenRingProvider cassandraClientTokenRingProvider, + ServiceConfiguration serviceConfiguration, + DnsResolver dnsResolver) + { + this.instancesMetadata = instancesMetadata; + this.cassandraClientTokenRingProvider = cassandraClientTokenRingProvider; + this.serviceConfiguration = serviceConfiguration; + this.dnsResolver = dnsResolver; + } + + public Set<SidecarInstance> get() + { + Map<Integer, InstanceMetadata> localInstances = instancesMetadata + .instances() + .stream() + .collect(Collectors.toMap(InstanceMetadata::id, Function.identity())); + + if (localInstances.isEmpty()) + { + LOGGER.debug("No local instances found"); + return Set.of(); + } + + Metadata metadata = localInstances.values().stream().findFirst() + .map(in -> in.delegate().metadata()) + .orElse(null); + if (metadata == null) + { + LOGGER.debug("Not yet connect to Cassandra cluster"); + return Set.of(); + } + + final List<KeyspaceMetadata> keyspaces = metadata.getKeyspaces() Review Comment: remove `final` and such `final`s elsewhere in the patch. ########## server/src/main/java/org/apache/cassandra/sidecar/coordination/SidecarPeerHealthMonitorTask.java: ########## @@ -0,0 +1,191 @@ +/* + * 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.cassandra.sidecar.coordination; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.CompositeFuture; +import io.vertx.core.Future; +import io.vertx.core.Promise; +import io.vertx.core.Vertx; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.codecs.SidecarInstanceCodecs; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.server.utils.DurationSpec; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.SidecarPeerHealthConfiguration; +import org.apache.cassandra.sidecar.tasks.PeriodicTask; +import org.apache.cassandra.sidecar.tasks.ScheduleDecision; +import org.jetbrains.annotations.NotNull; + +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_PEER_DOWN; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_PEER_UP; + +/** + * Pings other 'peer' Sidecar(s) that are relevant to this Sidecar over HTTP and notifies + * listeners when other Sidecar(s) goes DOWN or OK. + */ +@Singleton +public class SidecarPeerHealthMonitorTask implements PeriodicTask +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SidecarPeerHealthMonitorTask.class); + + private final Vertx vertx; + private final SidecarPeerHealthConfiguration config; + private final SidecarPeerProvider sidecarPeerProvider; + private final SidecarPeerHealthProvider healthProvider; + + private final Map<SidecarInstance, SidecarPeerHealthProvider.Health> status = new ConcurrentHashMap<>(); + + @Inject + public SidecarPeerHealthMonitorTask(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + SidecarPeerProvider sidecarPeerProvider, + SidecarPeerHealthProvider healthProvider, + SidecarInstanceCodecs sidecarInstanceCodecs) + { + this.vertx = vertx; + this.config = sidecarConfiguration.sidecarPeerHealthConfiguration(); + this.sidecarPeerProvider = sidecarPeerProvider; + this.healthProvider = healthProvider; + vertx.eventBus().registerDefaultCodec(SidecarInstance.class, sidecarInstanceCodecs); + } + + @NotNull + public SidecarPeerHealthProvider.Health status(InstanceMetadata instance) Review Comment: The method is not used. ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/health/SidecarPeerDownDetectorIntegrationTest.java: ########## @@ -0,0 +1,201 @@ +/* + * 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.cassandra.sidecar.health; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import com.google.inject.name.Named; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.SidecarPeerHealthConfiguration; +import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarPeerHealthConfigurationImpl; +import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthMonitorTask; +import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthProvider; +import org.apache.cassandra.sidecar.server.Server; +import org.apache.cassandra.sidecar.testing.InnerDcTokenAdjacentPeerTestProvider.TestSidecarHostInfo; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.sidecar.testing.SharedClusterSidecarIntegrationTestBase; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; + +class SidecarPeerDownDetectorIntegrationTest extends SharedClusterSidecarIntegrationTestBase +{ + + private static final Logger LOGGER = LoggerFactory.getLogger(SidecarPeerDownDetectorIntegrationTest.class); + + List<TestSidecarHostInfo> sidecarServerList = new ArrayList<>(); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return new ClusterBuilderConfiguration().nodesPerDc(3); + } + + @Override + protected void startSidecar(ICluster<? extends IInstance> cluster) throws InterruptedException + { + Supplier<List<TestSidecarHostInfo>> supplier = () -> sidecarServerList; + PeersModule peersModule = new PeersModule(supplier); + for (IInstance cassandraInstance : cluster) + { + // Provider de una lista de Sidecar servers Review Comment: Please translate it to English. I kind of guessed what it means, but there is not precedent of using Spanish in this repo :p Can you also double check elsewhere in the patch, in case I missed. ########## server/src/main/java/org/apache/cassandra/sidecar/coordination/InnerDcTokenAdjacentPeerProvider.java: ########## @@ -0,0 +1,270 @@ +/* + * 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.cassandra.sidecar.coordination; + +import java.math.BigInteger; +import java.net.UnknownHostException; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.Host; +import com.datastax.driver.core.KeyspaceMetadata; +import com.datastax.driver.core.Metadata; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.cassandra.sidecar.cluster.InstancesMetadata; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.client.SidecarInstanceImpl; +import org.apache.cassandra.sidecar.common.server.cluster.locator.Token; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.common.server.dns.DnsResolver; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; + + +import static org.apache.cassandra.sidecar.config.yaml.CassandraInputValidationConfigurationImpl.DEFAULT_FORBIDDEN_KEYSPACES; + + +/** + * Return Sidecar(s) adjacent to current Sidecar in the token ring within the same datacenter. + */ +@Singleton +public class InnerDcTokenAdjacentPeerProvider implements SidecarPeerProvider +{ + private static final Logger LOGGER = LoggerFactory.getLogger(InnerDcTokenAdjacentPeerProvider.class); + + protected final InstancesMetadata instancesMetadata; + private final CassandraClientTokenRingProvider cassandraClientTokenRingProvider; + private final ServiceConfiguration serviceConfiguration; + private final DnsResolver dnsResolver; + + @Inject + public InnerDcTokenAdjacentPeerProvider(InstancesMetadata instancesMetadata, + CassandraClientTokenRingProvider cassandraClientTokenRingProvider, + ServiceConfiguration serviceConfiguration, + DnsResolver dnsResolver) + { + this.instancesMetadata = instancesMetadata; + this.cassandraClientTokenRingProvider = cassandraClientTokenRingProvider; + this.serviceConfiguration = serviceConfiguration; + this.dnsResolver = dnsResolver; + } + + public Set<SidecarInstance> get() + { + Map<Integer, InstanceMetadata> localInstances = instancesMetadata + .instances() + .stream() + .collect(Collectors.toMap(InstanceMetadata::id, Function.identity())); + + if (localInstances.isEmpty()) + { + LOGGER.debug("No local instances found"); + return Set.of(); + } + + Metadata metadata = localInstances.values().stream().findFirst() + .map(in -> in.delegate().metadata()) + .orElse(null); + if (metadata == null) + { + LOGGER.debug("Not yet connect to Cassandra cluster"); + return Set.of(); + } Review Comment: `in.delegate().metadata()` throws `CassandraUnavailableException` when there is no connection. In such scenario, the chain won't be evaluated to `null`, but throws. Here is the code snippet of my suggestion. ```java // Replaces `InstancesMetadata instancesMetadata` InstanceMetadataFetcher instanceFetcher; Metadata metadata; try { metadata = instanceFetcher.callOnFirstAvailableInstance(instance -> instance.delegate().metadata()); } catch (Throwable cause) { LOGGER.debug("Unable to retrieve metadata", cause); return Set.of(); } ``` ########## server/src/main/java/org/apache/cassandra/sidecar/config/yaml/SidecarPeerHealthConfigurationImpl.java: ########## @@ -0,0 +1,83 @@ +/* + * 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.cassandra.sidecar.config.yaml; + +import java.util.concurrent.TimeUnit; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.config.SidecarPeerHealthConfiguration; + + +/** + * Configuration for Peer Health checks + */ +public class SidecarPeerHealthConfigurationImpl extends PeriodicTaskConfigurationImpl implements SidecarPeerHealthConfiguration +{ + public static final boolean DEFAULT_ENABLED = true; + public static final MillisecondBoundConfiguration DEFAULT_FREQUENCY = new MillisecondBoundConfiguration(30, TimeUnit.SECONDS); + public static final int DEFAULT_HEALTH_CHECK_RETRIES = 5; + public static final MillisecondBoundConfiguration DEFAULT_HEALTH_CHECK_RETRY_DELAY = new MillisecondBoundConfiguration(10, TimeUnit.SECONDS);; + + @JsonProperty(value = "health_check_retries") + private final int healthCheckRetries; + @JsonProperty(value = "health_check_retry_delay") Review Comment: Update the json key value accordingly, once the configuration is renamed. ########## server/src/main/java/org/apache/cassandra/sidecar/config/SidecarPeerHealthConfiguration.java: ########## @@ -0,0 +1,37 @@ +/* + * 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.cassandra.sidecar.config; + +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; + +/** + * Configuration for Sidecar peers health checks + */ +public interface SidecarPeerHealthConfiguration extends PeriodicTaskConfiguration Review Comment: Is `Sidecar` needed in the name? It seems to be redundant. `PeerHealthConfiguration` is good enough ########## server/src/main/java/org/apache/cassandra/sidecar/codecs/SidecarInstanceCodecs.java: ########## @@ -0,0 +1,65 @@ +/* + * 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.cassandra.sidecar.codecs; + +import com.google.inject.Singleton; +import io.vertx.core.buffer.Buffer; +import io.vertx.core.eventbus.MessageCodec; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.client.SidecarInstanceImpl; + +/** + * Codecs for Sidecar instances + */ +@Singleton +public class SidecarInstanceCodecs implements MessageCodec<SidecarInstance, SidecarInstance> Review Comment: It is one codec, so not in the plural form. Rename it to `SidecarInstanceCodec` ########## integration-tests/src/integrationTest/org/apache/cassandra/sidecar/health/SidecarPeerDownDetectorIntegrationTest.java: ########## @@ -0,0 +1,201 @@ +/* + * 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.cassandra.sidecar.health; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Supplier; + +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.AbstractModule; +import com.google.inject.Provides; +import com.google.inject.Singleton; +import com.google.inject.name.Named; +import org.apache.cassandra.distributed.api.ICluster; +import org.apache.cassandra.distributed.api.IInstance; +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; +import org.apache.cassandra.sidecar.config.SidecarPeerHealthConfiguration; +import org.apache.cassandra.sidecar.config.yaml.ServiceConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarConfigurationImpl; +import org.apache.cassandra.sidecar.config.yaml.SidecarPeerHealthConfigurationImpl; +import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthMonitorTask; +import org.apache.cassandra.sidecar.coordination.SidecarPeerHealthProvider; +import org.apache.cassandra.sidecar.server.Server; +import org.apache.cassandra.sidecar.testing.InnerDcTokenAdjacentPeerTestProvider.TestSidecarHostInfo; +import org.apache.cassandra.sidecar.testing.QualifiedName; +import org.apache.cassandra.sidecar.testing.SharedClusterSidecarIntegrationTestBase; +import org.apache.cassandra.testing.ClusterBuilderConfiguration; + +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.assertj.core.api.Assertions.assertThat; + +class SidecarPeerDownDetectorIntegrationTest extends SharedClusterSidecarIntegrationTestBase +{ + + private static final Logger LOGGER = LoggerFactory.getLogger(SidecarPeerDownDetectorIntegrationTest.class); + + List<TestSidecarHostInfo> sidecarServerList = new ArrayList<>(); + + @Override + protected ClusterBuilderConfiguration testClusterConfiguration() + { + return new ClusterBuilderConfiguration().nodesPerDc(3); + } + + @Override + protected void startSidecar(ICluster<? extends IInstance> cluster) throws InterruptedException + { + Supplier<List<TestSidecarHostInfo>> supplier = () -> sidecarServerList; + PeersModule peersModule = new PeersModule(supplier); + for (IInstance cassandraInstance : cluster) + { + // Provider de una lista de Sidecar servers + LOGGER.info("Starting Sidecar instance for Cassandra instance {}", + cassandraInstance.config().num()); + Server server = startSidecarWithInstances(List.of(cassandraInstance), peersModule); + sidecarServerList.add(new TestSidecarHostInfo(cassandraInstance, server, server.actualPort())); + } + + assertThat(sidecarServerList.size()).as("Each Cassandra Instance will be managed by a single Sidecar instance") + .isEqualTo(cluster.size()); + + + // assign the server to the first instance + server = sidecarServerList.get(0).getServer(); + } + + class PeersModule extends AbstractModule + { + Supplier<List<TestSidecarHostInfo>> supplier; + public PeersModule(Supplier<List<TestSidecarHostInfo>> supplier) + { + this.supplier = supplier; + } + + @Provides + @Singleton + @Named("sidecarInstanceSupplier") + public Supplier<List<TestSidecarHostInfo>> supplier() + { + return supplier; + } + + @Provides + @Singleton + public SidecarPeerHealthConfiguration sidecarPeerHealthConfiguration() + { + return new SidecarPeerHealthConfigurationImpl(true, + new MillisecondBoundConfiguration(1, TimeUnit.SECONDS), + 1, + new MillisecondBoundConfiguration(500, TimeUnit.MILLISECONDS)); + } + } + + void stopSidecarInstanceForTest(int instanceId) throws Exception + { + assertThat(sidecarServerList).isNotEmpty(); + Server server = sidecarServerList.get(instanceId).getServer(); + server.stop(serverDeploymentIds.get(server)).toCompletionStage().toCompletableFuture().get(30, TimeUnit.SECONDS); Review Comment: nit: you can use the `getBlocking` utility, which block the future and provides richer context on failure. ```suggestion String deploymentId = serverDeploymentIds.get(server); getBlocking(server.stop(serverDeploymentIds.get(server)), 30, TimeUnit.SECONDS, "Stopping server " + deploymentId); ``` ########## server/src/main/java/org/apache/cassandra/sidecar/codecs/CommonCodecs.java: ########## @@ -0,0 +1,37 @@ +/* + * 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.cassandra.sidecar.codecs; + +import io.vertx.core.eventbus.impl.codecs.BooleanMessageCodec; +import io.vertx.core.eventbus.impl.codecs.ByteArrayMessageCodec; +import io.vertx.core.eventbus.impl.codecs.IntMessageCodec; +import io.vertx.core.eventbus.impl.codecs.ShortMessageCodec; +import io.vertx.core.eventbus.impl.codecs.StringMessageCodec; + +/** + * Codecs common to Sidecar + */ +public class CommonCodecs Review Comment: The comment and the class name read vague to me. I cannot understand where the codec is going to be used. It cannot be "common". There does not seem to be a good value to have such a "common" codecs. Can you delete it and declare constant in `SidecarInstanceCodecs`? ########## server/src/main/java/org/apache/cassandra/sidecar/config/SidecarPeerHealthConfiguration.java: ########## @@ -0,0 +1,37 @@ +/* + * 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.cassandra.sidecar.config; + +import org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration; + +/** + * Configuration for Sidecar peers health checks + */ +public interface SidecarPeerHealthConfiguration extends PeriodicTaskConfiguration +{ + /** + * @return the number of maximum retries to be performed during a Sidecar peer health check + */ + int healthCheckRetries(); + + /** + * @return the delay between Sidecar peer health checks retries + */ + MillisecondBoundConfiguration healthCheckRetryDelay(); Review Comment: Those properties are consumed by the sidecar client. Can you rename the configurations? The current names confused me and made me thought that they were used for periodic task scheduling. My suggestion is `sidecarClientHealthCheckRetries` and `sidecarClientHealthCheckRetryDelay`. (It might be worthy to introduce a configuration for the client retry policy, but it is out of the scope of this patch. Let's do _not_ address this). We can always refactor if it becomes useful in the future.) ########## server/src/main/java/org/apache/cassandra/sidecar/coordination/InnerDcTokenAdjacentPeerProvider.java: ########## @@ -0,0 +1,270 @@ +/* + * 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.cassandra.sidecar.coordination; + +import java.math.BigInteger; +import java.net.UnknownHostException; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.base.Preconditions; +import org.apache.commons.lang3.tuple.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.datastax.driver.core.DataType; +import com.datastax.driver.core.Host; +import com.datastax.driver.core.KeyspaceMetadata; +import com.datastax.driver.core.Metadata; +import com.google.inject.Inject; +import com.google.inject.Singleton; +import org.apache.cassandra.sidecar.cluster.InstancesMetadata; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.client.SidecarInstanceImpl; +import org.apache.cassandra.sidecar.common.server.cluster.locator.Token; +import org.apache.cassandra.sidecar.common.server.cluster.locator.TokenRange; +import org.apache.cassandra.sidecar.common.server.dns.DnsResolver; +import org.apache.cassandra.sidecar.config.ServiceConfiguration; + + +import static org.apache.cassandra.sidecar.config.yaml.CassandraInputValidationConfigurationImpl.DEFAULT_FORBIDDEN_KEYSPACES; + + +/** + * Return Sidecar(s) adjacent to current Sidecar in the token ring within the same datacenter. + */ +@Singleton +public class InnerDcTokenAdjacentPeerProvider implements SidecarPeerProvider +{ + private static final Logger LOGGER = LoggerFactory.getLogger(InnerDcTokenAdjacentPeerProvider.class); + + protected final InstancesMetadata instancesMetadata; + private final CassandraClientTokenRingProvider cassandraClientTokenRingProvider; + private final ServiceConfiguration serviceConfiguration; + private final DnsResolver dnsResolver; + + @Inject + public InnerDcTokenAdjacentPeerProvider(InstancesMetadata instancesMetadata, + CassandraClientTokenRingProvider cassandraClientTokenRingProvider, + ServiceConfiguration serviceConfiguration, + DnsResolver dnsResolver) + { + this.instancesMetadata = instancesMetadata; + this.cassandraClientTokenRingProvider = cassandraClientTokenRingProvider; + this.serviceConfiguration = serviceConfiguration; + this.dnsResolver = dnsResolver; + } + + public Set<SidecarInstance> get() + { + Map<Integer, InstanceMetadata> localInstances = instancesMetadata + .instances() + .stream() + .collect(Collectors.toMap(InstanceMetadata::id, Function.identity())); + + if (localInstances.isEmpty()) + { + LOGGER.debug("No local instances found"); + return Set.of(); + } Review Comment: With `InstanceMetadataFetcher`, this `if` statement can be removed too. It is covered by the suggestion for line#99 ########## server/src/main/java/org/apache/cassandra/sidecar/utils/SidecarClientProvider.java: ########## @@ -0,0 +1,150 @@ +/* + * 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.cassandra.sidecar.utils; + +import java.util.ArrayList; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Inject; +import com.google.inject.Provider; +import com.google.inject.Singleton; +import io.vertx.core.Vertx; +import io.vertx.core.http.HttpClient; +import io.vertx.core.net.JksOptions; +import io.vertx.core.net.OpenSSLEngineOptions; +import io.vertx.ext.web.client.WebClient; +import io.vertx.ext.web.client.WebClientOptions; +import org.apache.cassandra.sidecar.client.HttpClientConfig; +import org.apache.cassandra.sidecar.client.SidecarClient; +import org.apache.cassandra.sidecar.client.SidecarClientConfig; +import org.apache.cassandra.sidecar.client.SidecarClientConfigImpl; +import org.apache.cassandra.sidecar.client.SimpleSidecarInstancesProvider; +import org.apache.cassandra.sidecar.client.VertxHttpClient; +import org.apache.cassandra.sidecar.client.VertxRequestExecutor; +import org.apache.cassandra.sidecar.client.retry.ExponentialBackoffRetryPolicy; +import org.apache.cassandra.sidecar.client.retry.RetryPolicy; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.client.SidecarInstanceImpl; +import org.apache.cassandra.sidecar.common.server.utils.SidecarVersionProvider; +import org.apache.cassandra.sidecar.config.SidecarClientConfiguration; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; + +/** + * Provider class for retrieving the singleton {@link SidecarClient} instance + */ +@Singleton +public class SidecarClientProvider implements Provider<SidecarClient> +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SidecarClientProvider.class); + private final Vertx vertx; + private final SidecarClientConfiguration clientConfig; + private final SidecarVersionProvider sidecarVersionProvider; + private final SidecarClient client; + + @Inject + public SidecarClientProvider(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + SidecarVersionProvider sidecarVersionProvider) + { + this.vertx = vertx; + this.clientConfig = sidecarConfiguration.sidecarClientConfiguration(); + this.sidecarVersionProvider = sidecarVersionProvider; + this.client = initializeSidecarClient(); + } + + @Override + public SidecarClient get() + { + return client; Review Comment: The client is not closed. Can you add close to this class, and call it when server is closing? ########## server/src/main/java/org/apache/cassandra/sidecar/coordination/SidecarPeerHealthMonitorTask.java: ########## @@ -0,0 +1,191 @@ +/* + * 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.cassandra.sidecar.coordination; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.stream.Collectors; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import io.vertx.core.CompositeFuture; +import io.vertx.core.Future; +import io.vertx.core.Promise; +import io.vertx.core.Vertx; +import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata; +import org.apache.cassandra.sidecar.codecs.SidecarInstanceCodecs; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; +import org.apache.cassandra.sidecar.common.server.utils.DurationSpec; +import org.apache.cassandra.sidecar.config.SidecarConfiguration; +import org.apache.cassandra.sidecar.config.SidecarPeerHealthConfiguration; +import org.apache.cassandra.sidecar.tasks.PeriodicTask; +import org.apache.cassandra.sidecar.tasks.ScheduleDecision; +import org.jetbrains.annotations.NotNull; + +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_PEER_DOWN; +import static org.apache.cassandra.sidecar.server.SidecarServerEvents.ON_SIDECAR_PEER_UP; + +/** + * Pings other 'peer' Sidecar(s) that are relevant to this Sidecar over HTTP and notifies + * listeners when other Sidecar(s) goes DOWN or OK. + */ +@Singleton +public class SidecarPeerHealthMonitorTask implements PeriodicTask +{ + private static final Logger LOGGER = LoggerFactory.getLogger(SidecarPeerHealthMonitorTask.class); + + private final Vertx vertx; + private final SidecarPeerHealthConfiguration config; + private final SidecarPeerProvider sidecarPeerProvider; + private final SidecarPeerHealthProvider healthProvider; + + private final Map<SidecarInstance, SidecarPeerHealthProvider.Health> status = new ConcurrentHashMap<>(); + + @Inject + public SidecarPeerHealthMonitorTask(Vertx vertx, + SidecarConfiguration sidecarConfiguration, + SidecarPeerProvider sidecarPeerProvider, + SidecarPeerHealthProvider healthProvider, + SidecarInstanceCodecs sidecarInstanceCodecs) + { + this.vertx = vertx; + this.config = sidecarConfiguration.sidecarPeerHealthConfiguration(); + this.sidecarPeerProvider = sidecarPeerProvider; + this.healthProvider = healthProvider; + vertx.eventBus().registerDefaultCodec(SidecarInstance.class, sidecarInstanceCodecs); + } + + @NotNull + public SidecarPeerHealthProvider.Health status(InstanceMetadata instance) + { + return status.getOrDefault(instance, SidecarPeerHealthProvider.Health.UNKNOWN); Review Comment: It looks wrong to look up `InstanceMetadata` from the map `Map<SidecarInstance, SidecarPeerHealthProvider.Health> status`. The key type does not match. ########## server/src/main/java/org/apache/cassandra/sidecar/coordination/SidecarPeerHealthProvider.java: ########## @@ -0,0 +1,43 @@ +/* + * 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.cassandra.sidecar.coordination; + + +import io.vertx.core.Future; +import org.apache.cassandra.sidecar.common.client.SidecarInstance; + +/** + * Interface to provide health information for other Sidecar instances + */ +public interface SidecarPeerHealthProvider +{ + /** + * Possible Health states + */ + enum Health + { + OK, DOWN, UNKNOWN Review Comment: - nit: either `UP` and `DOWN`, or `OK` and `NOT_OK`. But, let's not mix them. - not a nit: Add javadoc to explain when a peer is considered in such health status -- 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: pr-unsubscr...@cassandra.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: pr-unsubscr...@cassandra.apache.org For additional commands, e-mail: pr-h...@cassandra.apache.org