yifan-c commented on code in PR #231:
URL: https://github.com/apache/cassandra-sidecar/pull/231#discussion_r2252593843


##########
.gitignore:
##########
@@ -95,3 +95,4 @@ dtest-jars
 scripts/dependency-reduced-pom.xml
 
 default-stylesheet.xsl
+.aider*

Review Comment:
   Do you want to revert this line?



##########
adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RepairOptions.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.adapters.base;
+
+/**
+ * Enum representing the repair options supported
+ */
+public enum RepairOptions
+{
+    /**
+     * Whether to repair only the primary range of the node (true/false)
+     */
+    PRIMARY_RANGE("primaryRange"),
+    /**
+     * Whether to perform an incremental repair (true/false)
+     * If false, a full repair is performed
+     */
+    INCREMENTAL("incremental"),
+    /**
+     * Specific token ranges to repair
+     */
+    RANGES("ranges"),
+    /**
+     * List of column families (tables) to repair (comma-separated)
+     */
+    COLUMNFAMILIES("columnFamilies"),

Review Comment:
   nit: `COLUMN_FAMILIES`?  Just to be consistent with other multi-words enum 
names. 



##########
adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RepairOptions.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.adapters.base;
+
+/**
+ * Enum representing the repair options supported
+ */
+public enum RepairOptions
+{
+    /**
+     * Whether to repair only the primary range of the node (true/false)
+     */
+    PRIMARY_RANGE("primaryRange"),
+    /**
+     * Whether to perform an incremental repair (true/false)
+     * If false, a full repair is performed
+     */
+    INCREMENTAL("incremental"),
+    /**
+     * Specific token ranges to repair
+     */
+    RANGES("ranges"),
+    /**
+     * List of column families (tables) to repair (comma-separated)
+     */
+    COLUMNFAMILIES("columnFamilies"),
+    /**
+     * Restrict repair to specific data centers (comma-separated)
+     */
+    DATACENTERS("dataCenters"),
+    /**
+     * Restrict repair to specific hosts (comma-separated IPs or hostnames)
+     */
+    HOSTS("hosts"),
+    /**
+     * force the repair operation

Review Comment:
   nit
   
   ```suggestion
        * Force the repair operation
   ```



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.vertx.core.Future;
+import io.vertx.core.Promise;
+import org.apache.cassandra.sidecar.adapters.base.RepairOptions;
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool;
+import org.apache.cassandra.sidecar.config.RepairJobsConfiguration;
+import org.apache.cassandra.sidecar.handlers.data.RepairRequestParam;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Implementation of {@link OperationalJob} to perform repair operation.
+ */
+public class RepairJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(RepairJob.class);
+    private static final String OPERATION = "repair";
+    private static final String PREVIEW_KIND_REPAIRED = "REPAIRED";
+
+    private final RepairRequestParam repairParams;
+    private final RepairJobsConfiguration config;
+    private final TaskExecutorPool internalPool;
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the status of a parent repair session
+     */
+    public enum ParentRepairStatus
+    {
+        IN_PROGRESS, COMPLETED, FAILED
+    }
+
+    /**
+     * Constructor for creation of RepairJob
+     *
+     * @param taskExecutorPool    TaskExecutorPool instance (for testing)
+     * @param config              Repair job configuration
+     * @param jobId               UUID representing the Job to be created
+     * @param storageOps          Reference to the storage operations interface
+     * @param repairParams        Repair request parameters
+     */
+    public RepairJob(TaskExecutorPool taskExecutorPool,
+                     RepairJobsConfiguration config,
+                     UUID jobId,
+                     StorageOperations storageOps,
+                     RepairRequestParam repairParams)
+    {
+        super(jobId);
+        this.internalPool = taskExecutorPool;
+        this.config = config;
+        this.storageOperations = storageOps;
+        this.repairParams = repairParams;
+    }
+
+    /**
+     * {@inheritDoc}
+     * RepairJob allows parallel executions since multiple repairs can run 
concurrently
+     * on different tables or with different parameters.
+     */
+    @Override
+    public boolean hasConflict(@NotNull List<OperationalJob> sameOperationJobs)
+    {
+        // For now, we simply allow all repair jobs to run in parallel
+        // A more sophisticated implementation could check the repair 
parameters
+        // against currently running repairs to detect potential conflicts
+        return false;
+    }
+
+    @Override
+    protected Future<Void> executeInternal()
+    {
+        Map<String, String> options = 
generateRepairOptions(repairParams.requestpayload());
+        String keyspace = repairParams.keyspace().name();
+
+        LOGGER.info("Executing repair operation for keyspace {} jobId={} 
maxRuntime={}",
+                    keyspace, this.jobId(), config.maxRepairJobRuntime());
+
+        int cmd = storageOperations.repair(keyspace, options);
+        if (cmd <= 0)
+        {
+            // When there are no relevant token ranges for the keyspace or RF 
is 1, the repair is inapplicable
+            LOGGER.info("Repair is not applicable for the provided options and 
keyspace '{}' jobId '{}'", keyspace, this.jobId());
+            return Future.succeededFuture();
+        }
+        Promise<Void> repairPromise = Promise.promise();
+        queryForCompletedRepair(repairPromise, cmd);
+
+        if (!repairPromise.future().isComplete())
+        {
+            Promise<Boolean> maxWaitTimePromise = Promise.promise();
+            long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                                 d -> 
maxWaitTimePromise.tryComplete(true));
+
+            long periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                       id -> 
queryForCompletedRepair(repairPromise, cmd));
+
+            repairPromise.future().onComplete(ar -> {
+                internalPool.cancelTimer(timerId);
+                internalPool.cancelTimer(periodicId);
+                maxWaitTimePromise.tryComplete(false);
+            });
+
+            return Future.any(maxWaitTimePromise.future(), 
repairPromise.future())
+                         .compose(f -> {
+                             boolean isTimeout = 
maxWaitTimePromise.future().result();
+                             if (isTimeout)
+                             {
+                                 LOGGER.error("Timer ran out before the repair 
job completed. Repair took too long");
+                                 return Future.failedFuture("Repair job taking 
too long");
+                             }
+                             return repairPromise.future();
+                         });
+        }
+        return repairPromise.future();
+    }
+
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String name()
+    {
+        return OPERATION;
+    }
+
+    private Map<String, String> generateRepairOptions(RepairPayload 
repairPayload)
+    {
+        Map<String, String> options = new HashMap<>();
+
+        List<String> tables = repairPayload.tables();
+        if (tables != null && !tables.isEmpty())
+        {
+            options.put(RepairOptions.COLUMNFAMILIES.getValue(), 
String.join(",", tables));
+        }
+
+        Boolean isPrimaryRange = repairPayload.isPrimaryRange();
+        if (isPrimaryRange != null)
+        {
+            options.put(RepairOptions.PRIMARY_RANGE.getValue(), 
String.valueOf(isPrimaryRange));
+        }
+        // TODO: Verify use-cases involving multiple DCs
+
+        String dc = repairPayload.datacenter();
+        if (dc != null)
+        {
+            options.put(RepairOptions.DATACENTERS.getValue(), dc);
+        }
+
+        List<String> hosts = repairPayload.hosts();
+        if (hosts != null && !hosts.isEmpty())
+        {
+            options.put(RepairOptions.HOSTS.getValue(), String.join(",", 
hosts));
+        }
+
+        if (repairPayload.startToken() != null && repairPayload.endToken() != 
null)
+        {
+            options.put(RepairOptions.RANGES.getValue(), 
repairPayload.startToken() + ":" + repairPayload.endToken());
+        }
+
+        if (repairPayload.repairType() == RepairPayload.RepairType.INCREMENTAL)
+        {
+            options.put(RepairOptions.INCREMENTAL.getValue(), 
Boolean.TRUE.toString());
+        }
+
+        if (repairPayload.force() != null)
+        {
+            options.put(RepairOptions.FORCE_REPAIR.getValue(), 
String.valueOf(repairPayload.force()));
+        }
+
+        if (repairPayload.isValidate() != null)
+        {
+            options.put(RepairOptions.PREVIEW.getValue(), 
PREVIEW_KIND_REPAIRED);
+        }
+        return options;
+    }
+
+    private void queryForCompletedRepair(Promise<Void> promise, int cmd)
+    {
+        LOGGER.info("Polling repair operation for jobId={} status={}", 
this.jobId(), promise.future().isComplete());
+        List<String> status = storageOperations.getParentRepairStatus(cmd);
+        String queriedString = "queried for parent session status and";
+        if (status == null || status.isEmpty())
+        {
+            LOGGER.error("{} couldn't find repair status for cmd: {}", 
queriedString, cmd);
+            promise.fail("Couldn't find repair status for cmd: " + cmd);
+        }
+        else
+        {
+            ParentRepairStatus parentRepairStatus = 
ParentRepairStatus.valueOf(status.get(0));
+            List<String> messages = status.subList(1, status.size());
+            switch (parentRepairStatus)
+            {
+                case COMPLETED:
+                case FAILED:
+                    LOGGER.info("{} discovered repair {}", queriedString, 
parentRepairStatus.name().toLowerCase());
+                    if (parentRepairStatus == ParentRepairStatus.FAILED)

Review Comment:
   I think you want to check `!messages.isEmpty()` too. Otherwise, it would not 
fail the `promise`, but rather fail the task execution. 



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.vertx.core.Future;
+import io.vertx.core.Promise;
+import org.apache.cassandra.sidecar.adapters.base.RepairOptions;
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool;
+import org.apache.cassandra.sidecar.config.RepairJobsConfiguration;
+import org.apache.cassandra.sidecar.handlers.data.RepairRequestParam;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Implementation of {@link OperationalJob} to perform repair operation.
+ */
+public class RepairJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(RepairJob.class);
+    private static final String OPERATION = "repair";
+    private static final String PREVIEW_KIND_REPAIRED = "REPAIRED";
+
+    private final RepairRequestParam repairParams;
+    private final RepairJobsConfiguration config;
+    private final TaskExecutorPool internalPool;
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the status of a parent repair session
+     */
+    public enum ParentRepairStatus
+    {
+        IN_PROGRESS, COMPLETED, FAILED
+    }
+
+    /**
+     * Constructor for creation of RepairJob
+     *
+     * @param taskExecutorPool    TaskExecutorPool instance (for testing)
+     * @param config              Repair job configuration
+     * @param jobId               UUID representing the Job to be created
+     * @param storageOps          Reference to the storage operations interface
+     * @param repairParams        Repair request parameters
+     */
+    public RepairJob(TaskExecutorPool taskExecutorPool,
+                     RepairJobsConfiguration config,
+                     UUID jobId,
+                     StorageOperations storageOps,
+                     RepairRequestParam repairParams)
+    {
+        super(jobId);
+        this.internalPool = taskExecutorPool;
+        this.config = config;
+        this.storageOperations = storageOps;
+        this.repairParams = repairParams;
+    }
+
+    /**
+     * {@inheritDoc}
+     * RepairJob allows parallel executions since multiple repairs can run 
concurrently
+     * on different tables or with different parameters.
+     */
+    @Override
+    public boolean hasConflict(@NotNull List<OperationalJob> sameOperationJobs)
+    {
+        // For now, we simply allow all repair jobs to run in parallel
+        // A more sophisticated implementation could check the repair 
parameters
+        // against currently running repairs to detect potential conflicts
+        return false;
+    }
+
+    @Override
+    protected Future<Void> executeInternal()
+    {
+        Map<String, String> options = 
generateRepairOptions(repairParams.requestpayload());
+        String keyspace = repairParams.keyspace().name();
+
+        LOGGER.info("Executing repair operation for keyspace {} jobId={} 
maxRuntime={}",
+                    keyspace, this.jobId(), config.maxRepairJobRuntime());
+
+        int cmd = storageOperations.repair(keyspace, options);
+        if (cmd <= 0)
+        {
+            // When there are no relevant token ranges for the keyspace or RF 
is 1, the repair is inapplicable
+            LOGGER.info("Repair is not applicable for the provided options and 
keyspace '{}' jobId '{}'", keyspace, this.jobId());
+            return Future.succeededFuture();
+        }
+        Promise<Void> repairPromise = Promise.promise();
+        queryForCompletedRepair(repairPromise, cmd);
+
+        if (!repairPromise.future().isComplete())
+        {
+            Promise<Boolean> maxWaitTimePromise = Promise.promise();
+            long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                                 d -> 
maxWaitTimePromise.tryComplete(true));
+
+            long periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                       id -> 
queryForCompletedRepair(repairPromise, cmd));
+
+            repairPromise.future().onComplete(ar -> {
+                internalPool.cancelTimer(timerId);
+                internalPool.cancelTimer(periodicId);
+                maxWaitTimePromise.tryComplete(false);
+            });
+
+            return Future.any(maxWaitTimePromise.future(), 
repairPromise.future())
+                         .compose(f -> {
+                             boolean isTimeout = 
maxWaitTimePromise.future().result();
+                             if (isTimeout)
+                             {
+                                 LOGGER.error("Timer ran out before the repair 
job completed. Repair took too long");
+                                 return Future.failedFuture("Repair job taking 
too long");
+                             }
+                             return repairPromise.future();
+                         });
+        }
+        return repairPromise.future();
+    }
+
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String name()
+    {
+        return OPERATION;
+    }
+
+    private Map<String, String> generateRepairOptions(RepairPayload 
repairPayload)
+    {
+        Map<String, String> options = new HashMap<>();
+
+        List<String> tables = repairPayload.tables();
+        if (tables != null && !tables.isEmpty())
+        {
+            options.put(RepairOptions.COLUMNFAMILIES.getValue(), 
String.join(",", tables));
+        }
+
+        Boolean isPrimaryRange = repairPayload.isPrimaryRange();
+        if (isPrimaryRange != null)
+        {
+            options.put(RepairOptions.PRIMARY_RANGE.getValue(), 
String.valueOf(isPrimaryRange));
+        }
+        // TODO: Verify use-cases involving multiple DCs
+
+        String dc = repairPayload.datacenter();
+        if (dc != null)
+        {
+            options.put(RepairOptions.DATACENTERS.getValue(), dc);
+        }
+
+        List<String> hosts = repairPayload.hosts();
+        if (hosts != null && !hosts.isEmpty())
+        {
+            options.put(RepairOptions.HOSTS.getValue(), String.join(",", 
hosts));
+        }
+
+        if (repairPayload.startToken() != null && repairPayload.endToken() != 
null)
+        {
+            options.put(RepairOptions.RANGES.getValue(), 
repairPayload.startToken() + ":" + repairPayload.endToken());
+        }
+
+        if (repairPayload.repairType() == RepairPayload.RepairType.INCREMENTAL)
+        {
+            options.put(RepairOptions.INCREMENTAL.getValue(), 
Boolean.TRUE.toString());
+        }
+
+        if (repairPayload.force() != null)
+        {
+            options.put(RepairOptions.FORCE_REPAIR.getValue(), 
String.valueOf(repairPayload.force()));
+        }
+
+        if (repairPayload.isValidate() != null)
+        {
+            options.put(RepairOptions.PREVIEW.getValue(), 
PREVIEW_KIND_REPAIRED);
+        }
+        return options;
+    }
+
+    private void queryForCompletedRepair(Promise<Void> promise, int cmd)
+    {
+        LOGGER.info("Polling repair operation for jobId={} status={}", 
this.jobId(), promise.future().isComplete());
+        List<String> status = storageOperations.getParentRepairStatus(cmd);
+        String queriedString = "queried for parent session status and";
+        if (status == null || status.isEmpty())
+        {
+            LOGGER.error("{} couldn't find repair status for cmd: {}", 
queriedString, cmd);
+            promise.fail("Couldn't find repair status for cmd: " + cmd);
+        }
+        else
+        {
+            ParentRepairStatus parentRepairStatus = 
ParentRepairStatus.valueOf(status.get(0));
+            List<String> messages = status.subList(1, status.size());
+            switch (parentRepairStatus)
+            {
+                case COMPLETED:
+                case FAILED:
+                    LOGGER.info("{} discovered repair {}", queriedString, 
parentRepairStatus.name().toLowerCase());
+                    if (parentRepairStatus == ParentRepairStatus.FAILED)
+                    {
+                        promise.fail(new IOException(messages.get(0)));
+                    }
+                    LOGGER.info("Repair {} Messages: {}", 
parentRepairStatus.name().toLowerCase(), String.join("\n", messages));
+                    promise.tryComplete();
+                    break;
+                case IN_PROGRESS:
+                    LOGGER.info("Repair in progress. Messages:{}", 
String.join("\n", messages));
+                    break;
+                default:
+                    String message = String.format("Encountered unexpected 
repair status: %s Messages: %s", parentRepairStatus, String.join("\n", 
messages));
+                    LOGGER.error(message);
+                    promise.fail(message);
+                    break;

Review Comment:
   I do not think you can ever hit this case for switch on enum. Any 
unrecognized status would fail the statement `ParentRepairStatus.valueOf` 
first. 



##########
server/src/main/java/org/apache/cassandra/sidecar/handlers/RepairHandler.java:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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.Set;
+
+import com.datastax.driver.core.utils.UUIDs;
+import com.google.inject.Inject;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.Vertx;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.json.DecodeException;
+import io.vertx.core.json.Json;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.auth.authorization.Authorization;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.cassandra.sidecar.acl.authorization.BasicPermissions;
+import org.apache.cassandra.sidecar.common.data.OperationalJobStatus;
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.response.OperationalJobResponse;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.common.server.data.Name;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.config.ServiceConfiguration;
+import org.apache.cassandra.sidecar.exceptions.OperationalJobConflictException;
+import org.apache.cassandra.sidecar.handlers.data.RepairRequestParam;
+import org.apache.cassandra.sidecar.job.OperationalJobManager;
+import org.apache.cassandra.sidecar.job.RepairJob;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+import org.apache.cassandra.sidecar.utils.OperationalJobUtils;
+import org.jetbrains.annotations.NotNull;
+
+import static 
org.apache.cassandra.sidecar.utils.HttpExceptions.wrapHttpException;
+
+/**
+ * Handler for triggering repair
+ */
+public class RepairHandler extends AbstractHandler<RepairRequestParam> 
implements AccessProtected
+{
+    private final ServiceConfiguration config;
+    private final OperationalJobManager jobManager;
+    private final Vertx vertx;

Review Comment:
   not used.



##########
server/src/main/java/org/apache/cassandra/sidecar/handlers/data/RepairRequestParam.java:
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.data;
+
+
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.server.data.Name;
+
+/**
+ * Holder class for the {@link 
org.apache.cassandra.sidecar.handlers.RepairHandler}
+ * request parameters
+ */
+public class RepairRequestParam
+{
+    private final Name keyspace;
+    private final RepairPayload repairRequestPayload;
+
+    private RepairRequestParam(Name keyspace, RepairPayload requestpayload)
+    {
+
+        this.keyspace = keyspace;
+        this.repairRequestPayload = requestpayload;
+    }
+
+    public static RepairRequestParam from(Name keyspace, RepairPayload payload)
+    {
+        return new RepairRequestParam(keyspace, payload);
+    }
+
+    /**
+     * @return the keyspace in Cassandra
+     */
+    public Name keyspace()
+    {
+        return keyspace;
+    }
+
+    /**
+     * @return the Repair request payload
+     */
+    public RepairPayload requestpayload()

Review Comment:
   please use camel case method name `requestPayload` 



##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/request/data/RepairPayload.java:
##########
@@ -0,0 +1,269 @@
+/*
+ * 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.request.data;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Optional;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonInclude;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
+import com.fasterxml.jackson.databind.annotation.JsonDeserialize;
+import org.apache.cassandra.sidecar.common.DataObjectBuilder;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Request payload for a repair job
+ */
+@JsonDeserialize(builder = RepairPayload.Builder.class)
+@JsonInclude(JsonInclude.Include.NON_NULL)
+public class RepairPayload
+{
+    private static final String TABLES = "tables";
+    private static final String IS_PRIMARY_RANGE = "primaryRange";
+    private static final String DATACENTER = "datacenter";
+    private static final String HOSTS = "hosts";
+    private static final String START_TOKEN = "startToken";
+    private static final String END_TOKEN = "endToken";
+    private static final String REPAIR_TYPE = "repairType";
+    private static final String FORCE = "force";
+    private static final String VALIDATE = "validate";
+
+    private final List<String> tables;
+    private final Boolean isPrimaryRange;
+    private final String datacenter;
+    private final List<String> hosts;
+    private final String startToken;
+    private final String endToken;
+    private RepairType repairType;
+    private final Boolean force;
+    private final Boolean validate;
+
+    /**
+     * Constructs a new {@link RepairPayload}.
+     */
+    public RepairPayload()
+    {
+        this(builder());
+    }
+
+    public static RepairPayload.Builder builder()
+    {
+        return new RepairPayload.Builder();
+    }
+
+    /**
+     * Constructs a new {@link RepairPayload} from the configured {@link 
RepairPayload.Builder}.
+     *
+     * @param builder the builder used to create this object
+     */
+    protected RepairPayload(RepairPayload.Builder builder)
+    {
+        tables = builder.tables;
+        repairType = builder.repairType;
+        isPrimaryRange = builder.isPrimaryRange;
+        datacenter = builder.datacenter;
+        hosts = builder.hosts;
+        startToken = builder.startToken;
+        endToken = builder.endToken;
+        force = builder.force;
+        validate = builder.validate;
+    }
+
+    @Nullable @JsonProperty(TABLES)
+    public List<String> tables()
+    {
+        return tables;
+    }
+
+    @JsonProperty(IS_PRIMARY_RANGE)
+    public Boolean isPrimaryRange()
+    {
+        return isPrimaryRange;
+    }
+
+    @JsonProperty(DATACENTER)
+    public String datacenter()
+    {
+        return datacenter;
+    }
+
+    @JsonProperty(HOSTS)
+    public List<String> hosts()
+    {
+        return hosts;
+    }
+
+    @JsonProperty(START_TOKEN)
+    public String startToken()
+    {
+        return startToken;
+    }
+
+    @JsonProperty(END_TOKEN)
+    public String endToken()
+    {
+        return endToken;
+    }
+
+    @JsonProperty(REPAIR_TYPE)
+    public RepairType repairType()
+    {
+        return repairType;
+    }
+
+    @Nullable @JsonProperty(FORCE)
+    public Boolean force()
+    {
+        return force;
+    }
+
+    @JsonProperty(VALIDATE)
+    public Boolean isValidate()

Review Comment:
   Please either name the method `validate` or `isValidated`. 
   
   The current `isValidate` is not correct. 



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.vertx.core.Future;
+import io.vertx.core.Promise;
+import org.apache.cassandra.sidecar.adapters.base.RepairOptions;
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool;
+import org.apache.cassandra.sidecar.config.RepairJobsConfiguration;
+import org.apache.cassandra.sidecar.handlers.data.RepairRequestParam;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Implementation of {@link OperationalJob} to perform repair operation.
+ */
+public class RepairJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(RepairJob.class);
+    private static final String OPERATION = "repair";
+    private static final String PREVIEW_KIND_REPAIRED = "REPAIRED";
+
+    private final RepairRequestParam repairParams;
+    private final RepairJobsConfiguration config;
+    private final TaskExecutorPool internalPool;
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the status of a parent repair session
+     */
+    public enum ParentRepairStatus
+    {
+        IN_PROGRESS, COMPLETED, FAILED
+    }
+
+    /**
+     * Constructor for creation of RepairJob
+     *
+     * @param taskExecutorPool    TaskExecutorPool instance (for testing)
+     * @param config              Repair job configuration
+     * @param jobId               UUID representing the Job to be created
+     * @param storageOps          Reference to the storage operations interface
+     * @param repairParams        Repair request parameters
+     */
+    public RepairJob(TaskExecutorPool taskExecutorPool,
+                     RepairJobsConfiguration config,
+                     UUID jobId,
+                     StorageOperations storageOps,
+                     RepairRequestParam repairParams)
+    {
+        super(jobId);
+        this.internalPool = taskExecutorPool;
+        this.config = config;
+        this.storageOperations = storageOps;
+        this.repairParams = repairParams;
+    }
+
+    /**
+     * {@inheritDoc}
+     * RepairJob allows parallel executions since multiple repairs can run 
concurrently
+     * on different tables or with different parameters.
+     */
+    @Override
+    public boolean hasConflict(@NotNull List<OperationalJob> sameOperationJobs)
+    {
+        // For now, we simply allow all repair jobs to run in parallel
+        // A more sophisticated implementation could check the repair 
parameters
+        // against currently running repairs to detect potential conflicts
+        return false;
+    }
+
+    @Override
+    protected Future<Void> executeInternal()
+    {
+        Map<String, String> options = 
generateRepairOptions(repairParams.requestpayload());
+        String keyspace = repairParams.keyspace().name();
+
+        LOGGER.info("Executing repair operation for keyspace {} jobId={} 
maxRuntime={}",
+                    keyspace, this.jobId(), config.maxRepairJobRuntime());

Review Comment:
   I do not find the code that validate the existence of `keyspace`. It could 
create unnecessary load on Cassandra process.
   
   Please take a look at 
`org.apache.cassandra.sidecar.handlers.validations.ValidateTableExistenceHandler`.
 You can enhance it to only check for keyspace or add a new validator.



##########
adapters/adapters-base/src/main/java/org/apache/cassandra/sidecar/adapters/base/RepairOptions.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.adapters.base;
+
+/**
+ * Enum representing the repair options supported
+ */
+public enum RepairOptions
+{
+    /**
+     * Whether to repair only the primary range of the node (true/false)
+     */
+    PRIMARY_RANGE("primaryRange"),
+    /**
+     * Whether to perform an incremental repair (true/false)
+     * If false, a full repair is performed
+     */
+    INCREMENTAL("incremental"),
+    /**
+     * Specific token ranges to repair
+     */
+    RANGES("ranges"),
+    /**
+     * List of column families (tables) to repair (comma-separated)
+     */
+    COLUMNFAMILIES("columnFamilies"),
+    /**
+     * Restrict repair to specific data centers (comma-separated)
+     */
+    DATACENTERS("dataCenters"),

Review Comment:
   nit: `datacenters`? datacenter is often recognized as a single word in this 
domain. 



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.vertx.core.Future;
+import io.vertx.core.Promise;
+import org.apache.cassandra.sidecar.adapters.base.RepairOptions;
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool;
+import org.apache.cassandra.sidecar.config.RepairJobsConfiguration;
+import org.apache.cassandra.sidecar.handlers.data.RepairRequestParam;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Implementation of {@link OperationalJob} to perform repair operation.
+ */
+public class RepairJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(RepairJob.class);
+    private static final String OPERATION = "repair";
+    private static final String PREVIEW_KIND_REPAIRED = "REPAIRED";
+
+    private final RepairRequestParam repairParams;
+    private final RepairJobsConfiguration config;
+    private final TaskExecutorPool internalPool;
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the status of a parent repair session
+     */
+    public enum ParentRepairStatus
+    {
+        IN_PROGRESS, COMPLETED, FAILED
+    }
+
+    /**
+     * Constructor for creation of RepairJob
+     *
+     * @param taskExecutorPool    TaskExecutorPool instance (for testing)
+     * @param config              Repair job configuration
+     * @param jobId               UUID representing the Job to be created
+     * @param storageOps          Reference to the storage operations interface
+     * @param repairParams        Repair request parameters
+     */
+    public RepairJob(TaskExecutorPool taskExecutorPool,
+                     RepairJobsConfiguration config,
+                     UUID jobId,
+                     StorageOperations storageOps,
+                     RepairRequestParam repairParams)
+    {
+        super(jobId);
+        this.internalPool = taskExecutorPool;
+        this.config = config;
+        this.storageOperations = storageOps;
+        this.repairParams = repairParams;
+    }
+
+    /**
+     * {@inheritDoc}
+     * RepairJob allows parallel executions since multiple repairs can run 
concurrently
+     * on different tables or with different parameters.
+     */
+    @Override
+    public boolean hasConflict(@NotNull List<OperationalJob> sameOperationJobs)
+    {
+        // For now, we simply allow all repair jobs to run in parallel
+        // A more sophisticated implementation could check the repair 
parameters
+        // against currently running repairs to detect potential conflicts
+        return false;
+    }
+
+    @Override
+    protected Future<Void> executeInternal()
+    {
+        Map<String, String> options = 
generateRepairOptions(repairParams.requestpayload());
+        String keyspace = repairParams.keyspace().name();
+
+        LOGGER.info("Executing repair operation for keyspace {} jobId={} 
maxRuntime={}",
+                    keyspace, this.jobId(), config.maxRepairJobRuntime());
+
+        int cmd = storageOperations.repair(keyspace, options);
+        if (cmd <= 0)
+        {
+            // When there are no relevant token ranges for the keyspace or RF 
is 1, the repair is inapplicable
+            LOGGER.info("Repair is not applicable for the provided options and 
keyspace '{}' jobId '{}'", keyspace, this.jobId());
+            return Future.succeededFuture();
+        }
+        Promise<Void> repairPromise = Promise.promise();
+        queryForCompletedRepair(repairPromise, cmd);
+
+        if (!repairPromise.future().isComplete())
+        {
+            Promise<Boolean> maxWaitTimePromise = Promise.promise();
+            long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                                 d -> 
maxWaitTimePromise.tryComplete(true));
+
+            long periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                       id -> 
queryForCompletedRepair(repairPromise, cmd));
+
+            repairPromise.future().onComplete(ar -> {
+                internalPool.cancelTimer(timerId);
+                internalPool.cancelTimer(periodicId);
+                maxWaitTimePromise.tryComplete(false);
+            });
+
+            return Future.any(maxWaitTimePromise.future(), 
repairPromise.future())
+                         .compose(f -> {
+                             boolean isTimeout = 
maxWaitTimePromise.future().result();
+                             if (isTimeout)
+                             {
+                                 LOGGER.error("Timer ran out before the repair 
job completed. Repair took too long");
+                                 return Future.failedFuture("Repair job taking 
too long");
+                             }
+                             return repairPromise.future();
+                         });
+        }
+        return repairPromise.future();
+    }
+
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String name()
+    {
+        return OPERATION;
+    }
+
+    private Map<String, String> generateRepairOptions(RepairPayload 
repairPayload)
+    {
+        Map<String, String> options = new HashMap<>();
+
+        List<String> tables = repairPayload.tables();
+        if (tables != null && !tables.isEmpty())
+        {
+            options.put(RepairOptions.COLUMNFAMILIES.getValue(), 
String.join(",", tables));
+        }
+
+        Boolean isPrimaryRange = repairPayload.isPrimaryRange();
+        if (isPrimaryRange != null)
+        {
+            options.put(RepairOptions.PRIMARY_RANGE.getValue(), 
String.valueOf(isPrimaryRange));
+        }
+        // TODO: Verify use-cases involving multiple DCs
+
+        String dc = repairPayload.datacenter();
+        if (dc != null)
+        {
+            options.put(RepairOptions.DATACENTERS.getValue(), dc);
+        }
+
+        List<String> hosts = repairPayload.hosts();
+        if (hosts != null && !hosts.isEmpty())
+        {
+            options.put(RepairOptions.HOSTS.getValue(), String.join(",", 
hosts));
+        }
+
+        if (repairPayload.startToken() != null && repairPayload.endToken() != 
null)
+        {
+            options.put(RepairOptions.RANGES.getValue(), 
repairPayload.startToken() + ":" + repairPayload.endToken());
+        }
+
+        if (repairPayload.repairType() == RepairPayload.RepairType.INCREMENTAL)
+        {
+            options.put(RepairOptions.INCREMENTAL.getValue(), 
Boolean.TRUE.toString());
+        }
+
+        if (repairPayload.force() != null)
+        {
+            options.put(RepairOptions.FORCE_REPAIR.getValue(), 
String.valueOf(repairPayload.force()));
+        }
+
+        if (repairPayload.isValidate() != null)
+        {
+            options.put(RepairOptions.PREVIEW.getValue(), 
PREVIEW_KIND_REPAIRED);
+        }
+        return options;
+    }
+
+    private void queryForCompletedRepair(Promise<Void> promise, int cmd)

Review Comment:
   Should it be synchronized? in case that there are 2 overlapping tasks 
scheduled. 



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,244 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import io.vertx.core.Future;
+import io.vertx.core.Promise;
+import org.apache.cassandra.sidecar.adapters.base.RepairOptions;
+import org.apache.cassandra.sidecar.common.request.data.RepairPayload;
+import org.apache.cassandra.sidecar.common.server.StorageOperations;
+import org.apache.cassandra.sidecar.concurrent.TaskExecutorPool;
+import org.apache.cassandra.sidecar.config.RepairJobsConfiguration;
+import org.apache.cassandra.sidecar.handlers.data.RepairRequestParam;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Implementation of {@link OperationalJob} to perform repair operation.
+ */
+public class RepairJob extends OperationalJob
+{
+    private static final Logger LOGGER = 
LoggerFactory.getLogger(RepairJob.class);
+    private static final String OPERATION = "repair";
+    private static final String PREVIEW_KIND_REPAIRED = "REPAIRED";
+
+    private final RepairRequestParam repairParams;
+    private final RepairJobsConfiguration config;
+    private final TaskExecutorPool internalPool;
+    protected StorageOperations storageOperations;
+
+    /**
+     * Enum representing the status of a parent repair session
+     */
+    public enum ParentRepairStatus
+    {
+        IN_PROGRESS, COMPLETED, FAILED
+    }
+
+    /**
+     * Constructor for creation of RepairJob
+     *
+     * @param taskExecutorPool    TaskExecutorPool instance (for testing)
+     * @param config              Repair job configuration
+     * @param jobId               UUID representing the Job to be created
+     * @param storageOps          Reference to the storage operations interface
+     * @param repairParams        Repair request parameters
+     */
+    public RepairJob(TaskExecutorPool taskExecutorPool,
+                     RepairJobsConfiguration config,
+                     UUID jobId,
+                     StorageOperations storageOps,
+                     RepairRequestParam repairParams)
+    {
+        super(jobId);
+        this.internalPool = taskExecutorPool;
+        this.config = config;
+        this.storageOperations = storageOps;
+        this.repairParams = repairParams;
+    }
+
+    /**
+     * {@inheritDoc}
+     * RepairJob allows parallel executions since multiple repairs can run 
concurrently
+     * on different tables or with different parameters.
+     */
+    @Override
+    public boolean hasConflict(@NotNull List<OperationalJob> sameOperationJobs)
+    {
+        // For now, we simply allow all repair jobs to run in parallel
+        // A more sophisticated implementation could check the repair 
parameters
+        // against currently running repairs to detect potential conflicts
+        return false;
+    }
+
+    @Override
+    protected Future<Void> executeInternal()
+    {
+        Map<String, String> options = 
generateRepairOptions(repairParams.requestpayload());
+        String keyspace = repairParams.keyspace().name();
+
+        LOGGER.info("Executing repair operation for keyspace {} jobId={} 
maxRuntime={}",
+                    keyspace, this.jobId(), config.maxRepairJobRuntime());
+
+        int cmd = storageOperations.repair(keyspace, options);
+        if (cmd <= 0)
+        {
+            // When there are no relevant token ranges for the keyspace or RF 
is 1, the repair is inapplicable
+            LOGGER.info("Repair is not applicable for the provided options and 
keyspace '{}' jobId '{}'", keyspace, this.jobId());
+            return Future.succeededFuture();
+        }
+        Promise<Void> repairPromise = Promise.promise();
+        queryForCompletedRepair(repairPromise, cmd);
+
+        if (!repairPromise.future().isComplete())
+        {
+            Promise<Boolean> maxWaitTimePromise = Promise.promise();
+            long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                                 d -> 
maxWaitTimePromise.tryComplete(true));
+
+            long periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                       id -> 
queryForCompletedRepair(repairPromise, cmd));
+
+            repairPromise.future().onComplete(ar -> {
+                internalPool.cancelTimer(timerId);
+                internalPool.cancelTimer(periodicId);
+                maxWaitTimePromise.tryComplete(false);
+            });
+
+            return Future.any(maxWaitTimePromise.future(), 
repairPromise.future())
+                         .compose(f -> {
+                             boolean isTimeout = 
maxWaitTimePromise.future().result();
+                             if (isTimeout)
+                             {
+                                 LOGGER.error("Timer ran out before the repair 
job completed. Repair took too long");
+                                 return Future.failedFuture("Repair job taking 
too long");
+                             }
+                             return repairPromise.future();
+                         });
+        }
+        return repairPromise.future();
+    }
+
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public String name()
+    {
+        return OPERATION;
+    }
+
+    private Map<String, String> generateRepairOptions(RepairPayload 
repairPayload)
+    {
+        Map<String, String> options = new HashMap<>();
+
+        List<String> tables = repairPayload.tables();
+        if (tables != null && !tables.isEmpty())
+        {
+            options.put(RepairOptions.COLUMNFAMILIES.getValue(), 
String.join(",", tables));
+        }
+
+        Boolean isPrimaryRange = repairPayload.isPrimaryRange();
+        if (isPrimaryRange != null)
+        {
+            options.put(RepairOptions.PRIMARY_RANGE.getValue(), 
String.valueOf(isPrimaryRange));
+        }
+        // TODO: Verify use-cases involving multiple DCs
+
+        String dc = repairPayload.datacenter();
+        if (dc != null)
+        {
+            options.put(RepairOptions.DATACENTERS.getValue(), dc);
+        }
+
+        List<String> hosts = repairPayload.hosts();
+        if (hosts != null && !hosts.isEmpty())
+        {
+            options.put(RepairOptions.HOSTS.getValue(), String.join(",", 
hosts));
+        }
+
+        if (repairPayload.startToken() != null && repairPayload.endToken() != 
null)
+        {
+            options.put(RepairOptions.RANGES.getValue(), 
repairPayload.startToken() + ":" + repairPayload.endToken());
+        }

Review Comment:
   I feel it is easy enough to also check that start and end tokens form a 
valid range, i.e. start <= end, in the spirit of fail early. 



##########
server/src/test/java/org/apache/cassandra/sidecar/handlers/RepairHandlerTest.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.List;
+import java.util.Map;
+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.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.request.data.RepairPayload;
+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 org.mockito.ArgumentCaptor;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.ACCEPTED;
+import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static 
org.apache.cassandra.sidecar.common.data.OperationalJobStatus.FAILED;
+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.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link RepairHandler}
+ */
+@ExtendWith(VertxExtension.class)
+public class RepairHandlerTest
+{
+    static final Logger LOGGER = 
LoggerFactory.getLogger(RepairHandlerTest.class);
+    private static final String REPAIR_ROUTE = 
"/api/v1/cassandra/keyspaces/testkeyspace/repair";
+    Vertx vertx;
+    Server server;
+    StorageOperations mockStorageOperations = mock(StorageOperations.class);
+
+    @BeforeEach
+    void before() throws InterruptedException
+    {
+        Injector injector;
+        Module testOverride = Modules.override(new TestModule())
+                                     .with(new RepairTestModule());
+        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 testRepairHandler(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        RepairPayload payload = RepairPayload.builder()
+                                             .isPrimaryRange(true)
+                                             
.repairType(RepairPayload.RepairType.INCREMENTAL)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerIR(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        ArgumentCaptor<Map<String, String>> jobCapture = 
ArgumentCaptor.forClass(Map.class);
+        RepairPayload payload = RepairPayload.builder()
+                                             
.repairType(RepairPayload.RepairType.INCREMENTAL)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  verify(mockStorageOperations).repair(anyString(), 
jobCapture.capture());
+                  assertThat(jobCapture.getValue()).containsKey("incremental");
+                  
assertThat(jobCapture.getValue().get("incremental")).isEqualTo("true");
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerWithRanges(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        String expectedRanges = 0L + ":" + Integer.MAX_VALUE;
+        ArgumentCaptor<Map<String, String>> jobCapture = 
ArgumentCaptor.forClass(Map.class);
+        RepairPayload payload = RepairPayload.builder()
+                                             .startToken("0")
+                                             
.endToken(Integer.toString(Integer.MAX_VALUE))
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  verify(mockStorageOperations).repair(anyString(), 
jobCapture.capture());
+                  assertThat(jobCapture.getValue()).containsKey("ranges");
+                  
assertThat(jobCapture.getValue().get("ranges")).isEqualTo(expectedRanges);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerWithHosts(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        ArgumentCaptor<Map<String, String>> jobCapture = 
ArgumentCaptor.forClass(Map.class);
+        RepairPayload payload = RepairPayload.builder()
+                                             .hosts(List.of("127.0.0.1"))
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  verify(mockStorageOperations).repair(anyString(), 
jobCapture.capture());
+                  assertThat(jobCapture.getValue()).containsKey("hosts");
+                  
assertThat(jobCapture.getValue().get("hosts")).isEqualTo("127.0.0.1");
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerLongRunning(VertxTestContext context)
+    {
+        doAnswer(AdditionalAnswers.answersWithDelay(6000, invocation -> null))
+        .when(mockStorageOperations).repair(anyString(), any());
+
+        WebClient client = WebClient.create(vertx);
+        RepairPayload payload = RepairPayload.builder()
+                                             .isPrimaryRange(true)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(ACCEPTED.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(RUNNING);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerBadRequest(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .send(context.succeeding(response -> {
+                  
assertThat(response.statusCode()).isEqualTo(BAD_REQUEST.code());
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerFailed(VertxTestContext context)
+    {
+        doThrow(new RuntimeException("Simulated 
failure")).when(mockStorageOperations).repair(anyString(), any());
+        WebClient client = WebClient.create(vertx);
+        String testRoute = "/api/v1/cassandra/keyspaces/testkeyspace/repair";
+        RepairPayload payload = RepairPayload.builder()
+                                             .isPrimaryRange(true)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+
+        client.put(server.actualPort(), "127.0.0.1", testRoute)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.jobId()).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(FAILED);
+                  context.completeNow();
+              }));
+    }
+
+    /**
+     * Test guice module for Node Decommission handler tests

Review Comment:
   It is not for "Node Decommission handler"



##########
server/src/test/java/org/apache/cassandra/sidecar/handlers/RepairHandlerTest.java:
##########
@@ -0,0 +1,307 @@
+/*
+ * 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.List;
+import java.util.Map;
+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.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.request.data.RepairPayload;
+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 org.mockito.ArgumentCaptor;
+
+import static io.netty.handler.codec.http.HttpResponseStatus.ACCEPTED;
+import static io.netty.handler.codec.http.HttpResponseStatus.BAD_REQUEST;
+import static io.netty.handler.codec.http.HttpResponseStatus.OK;
+import static 
org.apache.cassandra.sidecar.common.data.OperationalJobStatus.FAILED;
+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.ArgumentMatchers.any;
+import static org.mockito.ArgumentMatchers.anyString;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doThrow;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+/**
+ * Tests for the {@link RepairHandler}
+ */
+@ExtendWith(VertxExtension.class)
+public class RepairHandlerTest
+{
+    static final Logger LOGGER = 
LoggerFactory.getLogger(RepairHandlerTest.class);
+    private static final String REPAIR_ROUTE = 
"/api/v1/cassandra/keyspaces/testkeyspace/repair";
+    Vertx vertx;
+    Server server;
+    StorageOperations mockStorageOperations = mock(StorageOperations.class);
+
+    @BeforeEach
+    void before() throws InterruptedException
+    {
+        Injector injector;
+        Module testOverride = Modules.override(new TestModule())
+                                     .with(new RepairTestModule());
+        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 testRepairHandler(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        RepairPayload payload = RepairPayload.builder()
+                                             .isPrimaryRange(true)
+                                             
.repairType(RepairPayload.RepairType.INCREMENTAL)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerIR(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        ArgumentCaptor<Map<String, String>> jobCapture = 
ArgumentCaptor.forClass(Map.class);
+        RepairPayload payload = RepairPayload.builder()
+                                             
.repairType(RepairPayload.RepairType.INCREMENTAL)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  verify(mockStorageOperations).repair(anyString(), 
jobCapture.capture());
+                  assertThat(jobCapture.getValue()).containsKey("incremental");
+                  
assertThat(jobCapture.getValue().get("incremental")).isEqualTo("true");
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerWithRanges(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        String expectedRanges = 0L + ":" + Integer.MAX_VALUE;
+        ArgumentCaptor<Map<String, String>> jobCapture = 
ArgumentCaptor.forClass(Map.class);
+        RepairPayload payload = RepairPayload.builder()
+                                             .startToken("0")
+                                             
.endToken(Integer.toString(Integer.MAX_VALUE))
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  verify(mockStorageOperations).repair(anyString(), 
jobCapture.capture());
+                  assertThat(jobCapture.getValue()).containsKey("ranges");
+                  
assertThat(jobCapture.getValue().get("ranges")).isEqualTo(expectedRanges);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerWithHosts(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        ArgumentCaptor<Map<String, String>> jobCapture = 
ArgumentCaptor.forClass(Map.class);
+        RepairPayload payload = RepairPayload.builder()
+                                             .hosts(List.of("127.0.0.1"))
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(SUCCEEDED);
+                  verify(mockStorageOperations).repair(anyString(), 
jobCapture.capture());
+                  assertThat(jobCapture.getValue()).containsKey("hosts");
+                  
assertThat(jobCapture.getValue().get("hosts")).isEqualTo("127.0.0.1");
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerLongRunning(VertxTestContext context)
+    {
+        doAnswer(AdditionalAnswers.answersWithDelay(6000, invocation -> null))
+        .when(mockStorageOperations).repair(anyString(), any());
+
+        WebClient client = WebClient.create(vertx);
+        RepairPayload payload = RepairPayload.builder()
+                                             .isPrimaryRange(true)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(ACCEPTED.code());
+                  LOGGER.info("Repair Response: {}", response.bodyAsString());
+
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(RUNNING);
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerBadRequest(VertxTestContext context)
+    {
+        WebClient client = WebClient.create(vertx);
+        client.put(server.actualPort(), "127.0.0.1", REPAIR_ROUTE)
+              .putHeader("Content-Type", "application/json")
+              .send(context.succeeding(response -> {
+                  
assertThat(response.statusCode()).isEqualTo(BAD_REQUEST.code());
+                  context.completeNow();
+              }));
+    }
+
+    @Test
+    void testRepairHandlerFailed(VertxTestContext context)
+    {
+        doThrow(new RuntimeException("Simulated 
failure")).when(mockStorageOperations).repair(anyString(), any());
+        WebClient client = WebClient.create(vertx);
+        String testRoute = "/api/v1/cassandra/keyspaces/testkeyspace/repair";
+        RepairPayload payload = RepairPayload.builder()
+                                             .isPrimaryRange(true)
+                                             .tables(List.of("test_table"))
+                                             .build();
+
+
+        client.put(server.actualPort(), "127.0.0.1", testRoute)
+              .putHeader("Content-Type", "application/json")
+              .sendJson(payload, context.succeeding(response -> {
+                  assertThat(response.statusCode()).isEqualTo(OK.code());
+                  OperationalJobResponse repairResponse = 
response.bodyAsJson(OperationalJobResponse.class);
+                  assertThat(repairResponse).isNotNull();
+                  assertThat(repairResponse.jobId()).isNotNull();
+                  assertThat(repairResponse.status()).isEqualTo(FAILED);
+                  context.completeNow();
+              }));
+    }
+
+    /**
+     * Test guice module for Node Decommission handler tests
+     */
+    class RepairTestModule extends AbstractModule
+    {
+        @Provides
+        @Singleton
+        public InstancesMetadata instanceMetadata()
+        {
+            final int instanceId = 100;
+            final String host = "127.0.0.1";
+            final InstanceMetadata instanceMetadata = 
mock(InstanceMetadata.class);
+            when(instanceMetadata.host()).thenReturn(host);
+            when(instanceMetadata.port()).thenReturn(9042);
+            when(instanceMetadata.id()).thenReturn(instanceId);
+            when(instanceMetadata.stagingDir()).thenReturn("");
+
+            CassandraAdapterDelegate delegate = 
mock(CassandraAdapterDelegate.class);
+
+            
when(delegate.storageOperations()).thenReturn(mockStorageOperations);
+            when(instanceMetadata.delegate()).thenReturn(delegate);
+
+            InstancesMetadata mockInstancesMetadata = 
mock(InstancesMetadata.class);
+            
when(mockInstancesMetadata.instances()).thenReturn(Collections.singletonList(instanceMetadata));
+            
when(mockInstancesMetadata.instanceFromId(instanceId)).thenReturn(instanceMetadata);
+            
when(mockInstancesMetadata.instanceFromHost(host)).thenReturn(instanceMetadata);
+
+            return mockInstancesMetadata;
+        }
+    }

Review Comment:
   The test module is exactly the same as the one in 
`NodeDecommissionHandlerTest`, as well as the `before()` and `after()` methods 
in the test. Can you remove the code duplication? 



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

Reply via email to