frankgh commented on code in PR #159:
URL: https://github.com/apache/cassandra-sidecar/pull/159#discussion_r1883338069
##########
client/src/main/java/org/apache/cassandra/sidecar/client/SidecarClient.java:
##########
@@ -215,6 +216,16 @@ public CompletableFuture<GossipInfoResponse> gossipInfo()
return
executor.executeRequestAsync(requestBuilder().gossipInfoRequest().build());
}
+ /**
+ * Executes the GET gossip status request using the default retry policy
and configured selection policy
+ *
+ * @return a completable future with gossip status response
+ */
+ public CompletableFuture<GossipStatusResponse> gossipStatus()
+ {
+ return
executor.executeRequestAsync(requestBuilder().gossipStatusRequest().build());
Review Comment:
this request should be instance-specific
##########
server/src/main/java/org/apache/cassandra/sidecar/routes/AbstractHandler.java:
##########
@@ -331,4 +332,16 @@ public static String extractHostAddressWithoutPort(String
address)
}
return address;
}
+
+ protected StorageOperations getStorageOperations(String host)
+ {
+ CassandraAdapterDelegate delegate =
this.metadataFetcher.delegate(host);
+ StorageOperations storageOperations = delegate == null ? null :
delegate.storageOperations();
+ if (storageOperations == null)
+ {
+ throw cassandraServiceUnavailable();
+ }
+
+ return storageOperations;
+ }
Review Comment:
we already have a method that does something similar in this class, we
should just generalize that method. I suggest renaming `ifMetricsOpsAvailable`
, and the code would look like this instead
```
protected <P> void ifAvailableFromDelegate(RoutingContext context,
String host,
Function<CassandraAdapterDelegate, P> mapper,
Consumer<P> ifAvailable)
{
CassandraAdapterDelegate delegate = metadataFetcher.delegate(host);
if (delegate == null)
{
context.fail(cassandraServiceUnavailable());
return;
}
P applied = mapper.apply(delegate);
if (applied == null)
{
context.fail(cassandraServiceUnavailable());
return;
}
ifAvailable.accept(applied);
}
```
The existing callsite would change to this:
```
ifAvailableFromDelegate(context, host,
CassandraAdapterDelegate::metricsOperations, operations -> {
```
##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/StorageOperations.java:
##########
@@ -102,4 +103,9 @@ default void outOfRangeDataCleanup(@NotNull String
keyspace, @NotNull String tab
{
outOfRangeDataCleanup(keyspace, table, 1);
}
+
+ /**
+ * @return returns gossip status
+ */
+ GossipStatusResponse isGossipRunning();
Review Comment:
Maybe better aligned with the action we are trying to perform
```suggestion
GossipStatusResponse gossipStatus();
```
##########
server/src/main/java/org/apache/cassandra/sidecar/routes/GossipStatusHandler.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.cassandra.sidecar.routes;
+
+import com.google.inject.Inject;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+
+/**
+ * Handler for to retrieve gossip status
+ */
+public class GossipStatusHandler extends AbstractHandler<Void>
+{
+ /**
+ * Constructs a handler with the provided {@code metadataFetcher}
+ * @param metadataFetcher the metadata fetcher
+ * @param executorPools executor pools for blocking executions
+ */
+ @Inject
+ protected GossipStatusHandler(InstanceMetadataFetcher metadataFetcher,
ExecutorPools executorPools)
+ {
+ super(metadataFetcher, executorPools, null);
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void handleInternal(RoutingContext context,
+ HttpServerRequest httpRequest,
+ String host,
+ SocketAddress remoteAddress,
+ Void request)
+ {
+ StorageOperations storageOperations = getStorageOperations(host);
+ logger.debug("Retrieving gossip status, remoteAddress={}, instance={}",
Review Comment:
this log entry is redundant, we already log
```
logger.debug("{} received request={}, remoteAddress={}, instance={}",
this.getClass().getSimpleName(), requestParams,
remoteAddress, host);
```
for all requests in handlers of the `AbstractHandler` type.
##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/response/GossipStatusResponse.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.common.response;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Response for GET gossip status API
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class GossipStatusResponse
+{
+ /**
+ * Constructs a response with gossip status
+ * @param gossipRunning running status of gossip
+ */
+ public GossipStatusResponse(@JsonProperty("gossipRunning") boolean
gossipRunning)
+ {
+ this.gossipRunning = gossipRunning;
+ }
+
+ @JsonProperty("gossipRunning")
+ public boolean gossipRunning()
+ {
+ return gossipRunning;
+ }
+
+ private final boolean gossipRunning;
Review Comment:
> Most of Sidecar request/response classes following this style
This is not true, I'm not sure which class you are looking at, but all
classes under
`client-common/src/main/java/org/apache/cassandra/sidecar/common/response`
define fields at the beginning of the class. And we should follow the Cassandra
code structure https://cassandra.apache.org/_/development/code_style.html under
Code Structure section > Class Layout.
##########
server/src/test/java/org/apache/cassandra/sidecar/routes/JmxCommonTest.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.cassandra.sidecar.routes;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.AbstractModule;
+import com.google.inject.Guice;
+import com.google.inject.Injector;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import com.google.inject.Singleton;
+import com.google.inject.util.Modules;
+import io.vertx.core.Vertx;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.sidecar.TestModule;
+import org.apache.cassandra.sidecar.cluster.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.cluster.InstancesConfig;
+import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.server.MainModule;
+import org.apache.cassandra.sidecar.server.Server;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Common test code to setup/teardown the server, populate test instances
metadata and mocks for JMX calls
+ */
+class JmxCommonTest
+{
+ static final Logger LOGGER = LoggerFactory.getLogger(JmxCommonTest.class);
Review Comment:
if we already have an in-jvm dtest this becomes less useful IMO. There is no
need to do mocking if we have the actual server running
##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/response/GossipStatusResponse.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.common.response;
+
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Response for GET gossip status API
+ */
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class GossipStatusResponse
+{
+ /**
+ * Constructs a response with gossip status
+ * @param gossipRunning running status of gossip
+ */
+ public GossipStatusResponse(@JsonProperty("gossipRunning") boolean
gossipRunning)
+ {
+ this.gossipRunning = gossipRunning;
+ }
+
+ @JsonProperty("gossipRunning")
+ public boolean gossipRunning()
+ {
+ return gossipRunning;
Review Comment:
returning a boolean becomes inflexible if you need to support a use case
where you have more than one value. I suggest we change this to a payload that
looks like
```json
{
"status": "RUNNING"
}
```
and
```json
{
"status": "NOT_RUNNING"
}
```
##########
server/src/test/integration/org/apache/cassandra/sidecar/routes/GossipStatusHandlerIntegrationTest.java:
##########
@@ -0,0 +1,76 @@
+/*
+ * 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.routes;
+
+import org.junit.jupiter.api.extension.ExtendWith;
+
+import io.vertx.core.buffer.Buffer;
+import io.vertx.core.json.JsonObject;
+import io.vertx.ext.web.client.HttpResponse;
+import io.vertx.ext.web.client.predicate.ResponsePredicate;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.sidecar.testing.IntegrationTestBase;
+import org.apache.cassandra.testing.CassandraIntegrationTest;
+import org.apache.cassandra.testing.CassandraTestContext;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Test GET gossip status endpoint with C* container
+ */
+@ExtendWith(VertxExtension.class)
+public class GossipStatusHandlerIntegrationTest extends IntegrationTestBase
+{
+ private static final String testRoute = "/api/v1/cassandra/gossip/status";
+
+ @CassandraIntegrationTest()
+ void testGossipDisabled(CassandraTestContext context, VertxTestContext
testContext)
Review Comment:
can we combine both test cases into a single test case? these are expensive
to spin up, so whenever possible we should reuse the in-jvm dtest apparatus
available .
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]