frankgh commented on code in PR #258:
URL: https://github.com/apache/cassandra-sidecar/pull/258#discussion_r2369339591


##########
server/src/main/java/org/apache/cassandra/sidecar/job/NodeDrainJob.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.job;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.data.OperationalJobStatus;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import 
org.apache.cassandra.sidecar.common.server.exceptions.OperationalJobException;
+
+/**
+ * Implementation of {@link OperationalJob} to perform node drain operation.
+ */
+public class NodeDrainJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(NodeDrainJob.class);
+    private static final String OPERATION = "drain";
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the various drain states of a Cassandra node.
+     */
+    public enum NodeDrainStateEnum
+    {
+        DRAINING(OperationalJobStatus.RUNNING),
+        DRAINED(OperationalJobStatus.SUCCEEDED);
+
+        final OperationalJobStatus jobStatus;
+
+        NodeDrainStateEnum(OperationalJobStatus jobStatus)
+        {
+            this.jobStatus = jobStatus;
+        }
+
+        /**
+         * Converts a string operation mode to a DrainState enum.
+         *
+         * @param operationMode the operation mode string
+         * @return the corresponding DrainState, or null if not a drain state
+         */
+        public static NodeDrainStateEnum fromOperationMode(String 
operationMode)
+        {
+            try
+            {
+                return valueOf(operationMode);
+            }
+            catch (IllegalArgumentException | NullPointerException e)
+            {
+                return null;
+            }
+        }
+    }
+
+    public NodeDrainJob(UUID jobId, StorageOperations storageOps)
+    {
+        super(jobId);
+        this.storageOperations = storageOps;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public boolean isRunningOnCassandra()
+    {
+        String operationMode = storageOperations.operationMode();
+        NodeDrainStateEnum nodeDrainStateEnum = 
NodeDrainStateEnum.fromOperationMode(operationMode);
+        return nodeDrainStateEnum != null && nodeDrainStateEnum.jobStatus == 
OperationalJobStatus.RUNNING;

Review Comment:
   I find this logic very confusing to follow, would it make more sense to just 
check whether the node is draining here instead?
   ```suggestion
           return nodeDrainStateEnum == NodeDrainStateEnum.DRAINING;
   ```



##########
server/src/main/java/org/apache/cassandra/sidecar/job/NodeDrainJob.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.job;
+
+import java.io.IOException;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.data.OperationalJobStatus;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import 
org.apache.cassandra.sidecar.common.server.exceptions.OperationalJobException;
+
+/**
+ * Implementation of {@link OperationalJob} to perform node drain operation.
+ */
+public class NodeDrainJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(NodeDrainJob.class);
+    private static final String OPERATION = "drain";
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the various drain states of a Cassandra node.
+     */
+    public enum NodeDrainStateEnum
+    {
+        DRAINING(OperationalJobStatus.RUNNING),
+        DRAINED(OperationalJobStatus.SUCCEEDED);
+
+        final OperationalJobStatus jobStatus;
+
+        NodeDrainStateEnum(OperationalJobStatus jobStatus)
+        {
+            this.jobStatus = jobStatus;
+        }
+
+        /**
+         * Converts a string operation mode to a DrainState enum.
+         *
+         * @param operationMode the operation mode string
+         * @return the corresponding DrainState, or null if not a drain state
+         */
+        public static NodeDrainStateEnum fromOperationMode(String 
operationMode)
+        {
+            try
+            {
+                return valueOf(operationMode);
+            }
+            catch (IllegalArgumentException | NullPointerException e)
+            {
+                return null;

Review Comment:
   should we log at debug when this happens at the very least?



##########
integration-tests/src/integrationTest/org/apache/cassandra/sidecar/routes/CassandraNodeOperationsIntegrationTest.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * 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.TimeUnit;
+
+import org.junit.jupiter.api.Test;
+
+import io.vertx.core.buffer.Buffer;
+import io.vertx.core.json.JsonObject;
+import io.vertx.ext.web.client.HttpResponse;
+import org.apache.cassandra.sidecar.common.ApiEndpointsV1;
+import org.apache.cassandra.sidecar.common.data.OperationalJobStatus;
+import 
org.apache.cassandra.sidecar.testing.SharedClusterSidecarIntegrationTestBase;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static org.apache.cassandra.testing.utils.AssertionUtils.getBlocking;
+import static org.assertj.core.api.Assertions.assertThat;
+
+/**
+ * Integration tests for Cassandra node drain operations
+ */
+public class CassandraNodeOperationsIntegrationTest extends 
SharedClusterSidecarIntegrationTestBase
+{
+    public static final String CASSANDRA_VERSION_4_0 = "4.0";
+
+    @Override
+    protected void initializeSchemaForTest()
+    {
+        // No schema init needed
+    }
+
+    @Override
+    protected void beforeTestStart()
+    {
+        // wait for the schema initialization
+        waitForSchemaReady(30, TimeUnit.SECONDS);
+    }
+
+    @Test
+    void testNodeDrainOperationSuccess()
+    {
+        // Initiate drain operation
+        HttpResponse<Buffer> drainResponse = getBlocking(
+        trustedClient().put(serverWrapper.serverPort, "localhost", 
ApiEndpointsV1.NODE_DRAIN_ROUTE)
+                       .send());
+
+        assertThat(drainResponse.statusCode()).isEqualTo(OK.code());
+
+        JsonObject responseBody = drainResponse.bodyAsJsonObject();
+        assertThat(responseBody).isNotNull();
+        assertThat(responseBody.getString("jobId")).isNotNull();
+        assertThat(responseBody.getString("jobStatus")).isIn(
+        OperationalJobStatus.CREATED.name(),
+        OperationalJobStatus.RUNNING.name(),
+        OperationalJobStatus.SUCCEEDED.name()
+        );
+
+        // Wait 30 seconds for drain operation to complete
+        try
+        {
+            Thread.sleep(30000);
+        }
+        catch (InterruptedException e)
+        {
+            throw new RuntimeException(e);
+        }
+
+        // Verify node status is DRAINED by checking the operationMode via 
stream stats endpoint
+        HttpResponse<Buffer> streamStatsResponse = getBlocking(
+        trustedClient().get(serverWrapper.serverPort, "localhost", 
ApiEndpointsV1.STREAM_STATS_ROUTE)
+                       .send());
+
+        assertThat(streamStatsResponse.statusCode()).isEqualTo(OK.code());
+
+        JsonObject streamStats = streamStatsResponse.bodyAsJsonObject();
+        assertThat(streamStats).isNotNull();
+        
assertThat(streamStats.getString("operationMode")).isEqualTo("DRAINED");

Review Comment:
   don't use thread.sleep, use loopassert instead
   ```suggestion
            // Wait up to 30 seconds for drain operation to complete
                   loopAssert(30, 500, () -> {
               // Verify node status is DRAINED by checking the operationMode 
via stream stats endpoint
               HttpResponse<Buffer> streamStatsResponse = getBlocking(
               trustedClient().get(serverWrapper.serverPort, "localhost", 
ApiEndpointsV1.STREAM_STATS_ROUTE)
                              .send());
   
               
assertThat(streamStatsResponse.statusCode()).isEqualTo(OK.code());
   
               JsonObject streamStats = streamStatsResponse.bodyAsJsonObject();
               assertThat(streamStats).isNotNull();
               
assertThat(streamStats.getString("operationMode")).isEqualTo("DRAINED");
           });
   ```



##########
server/src/main/java/org/apache/cassandra/sidecar/modules/CassandraOperationsModule.java:
##########
@@ -156,6 +157,22 @@ VertxRoute 
cassandraNodeDecommissionRoute(RouteBuilder.Factory factory,
         return factory.buildRouteWithHandler(nodeDecommissionHandler);
     }
 
+    @PUT
+    @Path(ApiEndpointsV1.NODE_DRAIN_ROUTE)
+    @Operation(summary = "Drain node",
+               description = "Drains the Cassandra node by flushing memtables 
and stopping writes")
+    @APIResponse(description = "Node drain operation initiated successfully",
+                 responseCode = "200",

Review Comment:
   I think we have two response codes 202 and 200, correct?



##########
server/src/test/java/org/apache/cassandra/sidecar/handlers/NodeDrainHandlerTest.java:
##########
@@ -0,0 +1,243 @@
+/*
+ * 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.handlers;
+
+import java.util.Collections;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+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.ext.web.client.WebClient;
+import io.vertx.ext.web.client.predicate.ResponsePredicate;
+import io.vertx.junit5.VertxExtension;
+import io.vertx.junit5.VertxTestContext;
+import org.apache.cassandra.sidecar.TestModule;
+import org.apache.cassandra.sidecar.cluster.CassandraAdapterDelegate;
+import org.apache.cassandra.sidecar.cluster.InstancesMetadata;
+import org.apache.cassandra.sidecar.cluster.instance.InstanceMetadata;
+import org.apache.cassandra.sidecar.common.response.OperationalJobResponse;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.modules.SidecarModules;
+import org.apache.cassandra.sidecar.server.Server;
+import org.mockito.AdditionalAnswers;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.ACCEPTED;
+import static io.netty.handler.codec.http.HttpResponseStatus.CONFLICT;
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static 
org.apache.cassandra.sidecar.common.data.OperationalJobStatus.RUNNING;
+import static 
org.apache.cassandra.sidecar.common.data.OperationalJobStatus.SUCCEEDED;
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link NodeDrainHandler}
+ */
+@ExtendWith(VertxExtension.class)
+public class NodeDrainHandlerTest
+{
+    static final Logger LOGGER = 
LoggerFactory.getLogger(NodeDrainHandlerTest.class);
+    static final String TEST_ROUTE = "/api/v1/cassandra/operations/drain";
+    static final String OPERATION_MODE_NORMAL = "NORMAL";
+    static final String OPERATION_MODE_DRAINING = "DRAINING";
+    static final String OPERATION_MODE_JOINING = "JOINING";
+    static final String EXPECTED_OPERATION_NAME = "drain";
+    static final String SIMULATED_DRAIN_FAILURE = "Simulated drain failure";
+
+    Vertx vertx;
+    Server server;
+    StorageOperations mockStorageOperations = mock(StorageOperations.class);
+
+    @BeforeEach
+    void before() throws InterruptedException
+    {
+        Injector injector;
+        Module testOverride = Modules.override(new TestModule())
+                                     .with(new 
NodeDrainHandlerTest.NodeDrainTestModule());
+        injector = Guice.createInjector(Modules.override(SidecarModules.all())
+                                               .with(testOverride));
+        vertx = injector.getInstance(Vertx.class);
+        server = injector.getInstance(Server.class);
+        VertxTestContext context = new VertxTestContext();
+        server.start()
+              .onSuccess(s -> context.completeNow())
+              .onFailure(context::failNow);
+        context.awaitCompletion(5, TimeUnit.SECONDS);
+    }
+
+    @AfterEach
+    void after() throws InterruptedException
+    {
+        CountDownLatch closeLatch = new CountDownLatch(1);
+        server.close().onSuccess(res -> closeLatch.countDown());
+        if (closeLatch.await(60, TimeUnit.SECONDS))
+            LOGGER.info("Close event received before timeout.");
+        else
+            LOGGER.error("Close event timed out.");
+    }
+
+    @Test
+    void testDrainLongRunning(VertxTestContext context) throws Exception
+    {
+        
when(mockStorageOperations.operationMode()).thenReturn(OPERATION_MODE_NORMAL);
+        doAnswer(AdditionalAnswers.answersWithDelay(6000, invocation -> null))
+        .when(mockStorageOperations).drain();
+
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", TEST_ROUTE)
+              .expect(ResponsePredicate.SC_ACCEPTED)
+              .send(context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(ACCEPTED.code());
+                  OperationalJobResponse drainResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(drainResponse).isNotNull();
+                  assertThat(drainResponse.status()).isEqualTo(RUNNING);
+                  assertThat(drainResponse.jobId()).isNotNull();
+                  
assertThat(drainResponse.operation()).isEqualTo(EXPECTED_OPERATION_NAME);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testDrainCompleted(VertxTestContext context)
+    {
+        
when(mockStorageOperations.operationMode()).thenReturn(OPERATION_MODE_NORMAL);
+
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", TEST_ROUTE)
+              .send(context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Drain Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse drainResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(drainResponse).isNotNull();
+                  assertThat(drainResponse.status()).isEqualTo(SUCCEEDED);
+                  assertThat(drainResponse.jobId()).isNotNull();
+                  
assertThat(drainResponse.operation()).isEqualTo(EXPECTED_OPERATION_NAME);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testDrainFailed(VertxTestContext context) throws Exception
+    {
+        
when(mockStorageOperations.operationMode()).thenReturn(OPERATION_MODE_NORMAL);
+        doThrow(new 
RuntimeException(SIMULATED_DRAIN_FAILURE)).when(mockStorageOperations).drain();
+
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", TEST_ROUTE)
+              .expect(ResponsePredicate.SC_OK)
+              .send(context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Drain Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse drainResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(drainResponse).isNotNull();
+                  assertThat(drainResponse.jobId()).isNotNull();
+                  
assertThat(drainResponse.operation()).isEqualTo(EXPECTED_OPERATION_NAME);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testDrainConflictWhenDraining(VertxTestContext context)
+    {
+        
when(mockStorageOperations.operationMode()).thenReturn(OPERATION_MODE_DRAINING);
+
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", TEST_ROUTE)
+              .expect(ResponsePredicate.SC_CONFLICT)
+              .send(context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(CONFLICT.code());
+                  LOGGER.info("Drain Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse drainResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(drainResponse).isNotNull();
+                  assertThat(drainResponse.jobId()).isNotNull();
+                  
assertThat(drainResponse.operation()).isEqualTo(EXPECTED_OPERATION_NAME);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testDrainAllowedWhenJoining(VertxTestContext context)
+    {
+        
when(mockStorageOperations.operationMode()).thenReturn(OPERATION_MODE_JOINING);
+
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", TEST_ROUTE)
+              .send(context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Drain Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse drainResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(drainResponse).isNotNull();
+                  assertThat(drainResponse.status()).isEqualTo(SUCCEEDED);
+                  assertThat(drainResponse.jobId()).isNotNull();
+                  
assertThat(drainResponse.operation()).isEqualTo(EXPECTED_OPERATION_NAME);
+                  context.completeNow();
+              }));
+    }
+
+    /**
+     * Test guice module for Node Drain handler tests
+     */
+    class NodeDrainTestModule extends AbstractModule
+    {
+        @Provides
+        @Singleton
+        public InstancesMetadata instanceMetadata()
+        {
+            final int instanceId = 100;
+            final String host = "127.0.0.1";
+            final InstanceMetadata instanceMetadata = 
mock(InstanceMetadata.class);

Review Comment:
   NIT: remove final keyword from local variables
   ```suggestion
               int instanceId = 100;
               String host = "127.0.0.1";
               InstanceMetadata instanceMetadata = mock(InstanceMetadata.class);
   ```



-- 
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]

Reply via email to