nvharikrishna commented on code in PR #231:
URL: https://github.com/apache/cassandra-sidecar/pull/231#discussion_r2265990589


##########
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 shouldValidate()
+    {
+        return validate;
+    }
+
+    /**
+     * Enum representing types of repair supported
+     */
+    public enum RepairType
+    {
+        FULL("full"),
+        INCREMENTAL("incremental");

Review Comment:
   Is preview repair out of scope?



##########
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 shouldValidate()
+    {
+        return validate;
+    }
+
+    /**
+     * Enum representing types of repair supported
+     */
+    public enum RepairType
+    {
+        FULL("full"),
+        INCREMENTAL("incremental");
+
+        private final String value;
+
+        RepairType(String value)
+        {
+            this.value = value;
+        }
+
+        @JsonValue
+        public String getValue()
+        {
+            return value;
+        }
+
+        @JsonCreator
+        public static RepairType fromValue(String text)
+        {
+            String normalized = Optional.ofNullable(text)

Review Comment:
   Any reason to accept a null value? A null value will not match with any of 
the enum values. Shouldn't it be `@NotNull`?



##########
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 shouldValidate()
+    {
+        return validate;
+    }
+
+    /**
+     * Enum representing types of repair supported
+     */
+    public enum RepairType
+    {
+        FULL("full"),
+        INCREMENTAL("incremental");
+
+        private final String value;
+
+        RepairType(String value)
+        {
+            this.value = value;
+        }
+
+        @JsonValue
+        public String getValue()
+        {
+            return value;
+        }
+
+        @JsonCreator
+        public static RepairType fromValue(String text)
+        {
+            String normalized = Optional.ofNullable(text)
+                                        .map(String::trim)
+                                        .filter(s -> !s.isEmpty())
+                                        .map(String::toLowerCase)
+                                        .orElse(null);
+
+            return Arrays.stream(RepairType.values())

Review Comment:
   Since we need to compare with only two values, can it be replaced with 
if-else or switch instead of stream? Or Simplify the enum to not to have any 
value (just the enum name)?



##########
server/src/main/java/org/apache/cassandra/sidecar/config/yaml/RepairJobsConfigurationImpl.java:
##########
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.config.yaml;
+
+import java.util.concurrent.TimeUnit;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import 
org.apache.cassandra.sidecar.common.server.utils.MillisecondBoundConfiguration;
+import org.apache.cassandra.sidecar.config.RepairJobsConfiguration;
+
+/**
+ * Configuration for Repair jobs
+ */
+public class RepairJobsConfigurationImpl implements RepairJobsConfiguration
+{
+    // 1 day in milliseconds
+    public static final long DEFAULT_MAX_REPAIR_RUNTIME_MILLIS = 24 * 60 * 60 
* 1000L;
+    public static final long DEFAULT_REPAIR_POLLING_INTERVAL_MILLIS = 2_000L;
+
+    @JsonProperty(value = "max_repair_runtime", defaultValue = 
DEFAULT_MAX_REPAIR_RUNTIME_MILLIS + "")
+    protected final long maxRepairRuntimeMillis;
+
+    @JsonProperty(value = "repair_polling_interval", defaultValue = 
DEFAULT_REPAIR_POLLING_INTERVAL_MILLIS + "")
+    protected final long repairPollIntervalMillis;
+
+    /**
+     * Default constructor that sets default values
+     */
+    public RepairJobsConfigurationImpl()
+    {
+        this(DEFAULT_MAX_REPAIR_RUNTIME_MILLIS, 
DEFAULT_REPAIR_POLLING_INTERVAL_MILLIS);
+    }
+
+    /**
+     * Constructor with parameters for JSON deserialization
+     *
+     * @param maxRepairRuntimeMillis the maximum runtime for repair jobs in 
milliseconds
+     * @param repairPollIntervalMillis the polling interval for repair jobs in 
milliseconds
+     */
+    @JsonCreator
+    public RepairJobsConfigurationImpl(
+        @JsonProperty(value = "max_repair_runtime", defaultValue = 
DEFAULT_MAX_REPAIR_RUNTIME_MILLIS + "") long maxRepairRuntimeMillis,
+        @JsonProperty(value = "repair_polling_interval", defaultValue = 
DEFAULT_REPAIR_POLLING_INTERVAL_MILLIS + "") long repairPollIntervalMillis)
+    {
+        this.maxRepairRuntimeMillis = maxRepairRuntimeMillis;

Review Comment:
   What are the valid values/range for these configs? Shall we add some 
validation?



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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, NEW_STATUS
+    }
+
+    /**
+     * 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();
+        }
+        final Promise<Void> repairPromise = Promise.promise();
+        final Promise<Boolean> maxWaitTimePromise = Promise.promise();
+        final long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                             d -> 
maxWaitTimePromise.tryComplete(true));
+
+        queryForCompletedRepair(repairPromise, cmd);
+        final boolean repairCompleted = repairPromise.future().isComplete();
+        
+        final Long periodicId;
+        if (!repairCompleted)
+        {
+            periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                id -> 
queryForCompletedRepair(repairPromise, cmd));
+            LOGGER.debug("Setting up periodic polling for repair job {}", 
this.jobId());
+        }
+        else
+        {
+            periodicId = null;
+            LOGGER.info("Repair job {} completed from eager status check", 
this.jobId());
+        }
+
+        repairPromise.future().onComplete(ar -> {
+            internalPool.cancelTimer(timerId);
+            if (periodicId != null)
+            {
+                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();

Review Comment:
   Here repairPromise's future is returned. If the repair job has been running 
more than `config.maxRepairJobRuntime()`, then `maxWaitTimePromise` is 
resolved, but not `repairPromise`. Then caller will keep on waiting till it is 
resolved by `queryForCompletedRepair`, right?
   
   I think cancellation of timer should be called in this handler i.e. when any 
of the futures has completed. 
   



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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, NEW_STATUS
+    }
+
+    /**
+     * 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();
+        }
+        final Promise<Void> repairPromise = Promise.promise();
+        final Promise<Boolean> maxWaitTimePromise = Promise.promise();
+        final long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                             d -> 
maxWaitTimePromise.tryComplete(true));
+
+        queryForCompletedRepair(repairPromise, cmd);

Review Comment:
   I think code get simplified if `.setPeriodic` is called with initially dealy 
as `0`, intead of calling it explicitly and setting periodic task 
conditionally. 



##########
server/src/main/java/org/apache/cassandra/sidecar/handlers/validations/ValidateKeyspaceExistenceHandler.java:
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.validations;
+
+import com.datastax.driver.core.KeyspaceMetadata;
+import com.google.inject.Inject;
+import com.google.inject.Singleton;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.http.HttpServerRequest;
+import io.vertx.core.net.SocketAddress;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.cassandra.sidecar.common.server.data.Name;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.handlers.AbstractHandler;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+import org.jetbrains.annotations.NotNull;
+
+import static 
org.apache.cassandra.sidecar.utils.HttpExceptions.wrapHttpException;
+
+/**
+ * Validate the request keyspace should exist in Cassandra, when the endpoint
+ * contains keyspace name.
+ * On successful validation, it stores the fetched {@link KeyspaceMetadata}
+ * in the {@link RoutingContext}
+ */
+@Singleton
+public class ValidateKeyspaceExistenceHandler extends AbstractHandler<Name>
+{
+    @Inject
+    public ValidateKeyspaceExistenceHandler(InstanceMetadataFetcher 
metadataFetcher,
+                                          ExecutorPools executorPools,
+                                          CassandraInputValidator validator)
+    {
+        super(metadataFetcher, executorPools, validator);
+    }
+
+    @Override
+    protected Name extractParamsOrThrow(RoutingContext context)
+    {
+        return keyspace(context, false);
+    }
+
+    @Override
+    protected void handleInternal(RoutingContext context,
+                                  HttpServerRequest httpRequest,
+                                  @NotNull String host,
+                                  SocketAddress remoteAddress,
+                                  Name keyspace)
+    {
+        if (keyspace == null)
+        {
+            context.fail(wrapHttpException(HttpResponseStatus.BAD_REQUEST, 
"Keyspace parameter is required but not provided"));
+            return;
+        }
+
+        ValidationUtils.validateKeyspaceExists(context, metadataFetcher, 
executorPools, host, keyspace.name())
+        .onComplete(ar -> {
+            if (ar.succeeded() && !context.failed())
+            {
+                context.next();
+            }
+            // Context has already been failed by the utility method when 
validation fails

Review Comment:
   Is there a case where ar succeedds but context fails? Which scenarios 
satisfies the if block?



##########
server-common/src/main/java/org/apache/cassandra/sidecar/common/server/StorageOperations.java:
##########
@@ -130,6 +130,24 @@ default void outOfRangeDataCleanup(@NotNull String 
keyspace, @NotNull String tab
      */
     String clusterName();
 
+    /**
+     * Triggers a repair operation for the given keyspace and options
+     *
+     * @param keyspace keyspace for the repair ioeration

Review Comment:
   ```suggestion
        * @param keyspace keyspace for the repair operation
   ```



##########
server/src/main/java/org/apache/cassandra/sidecar/handlers/validations/ValidationUtils.java:
##########
@@ -0,0 +1,95 @@
+/*
+ * 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.validations;
+
+import com.datastax.driver.core.KeyspaceMetadata;
+import io.netty.handler.codec.http.HttpResponseStatus;
+import io.vertx.core.Future;
+import io.vertx.ext.web.RoutingContext;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.routes.RoutingContextUtils;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static 
org.apache.cassandra.sidecar.utils.HttpExceptions.wrapHttpException;
+
+/**
+ * Utility class for validation handlers that check the existence of Cassandra 
schema elements.
+ */
+public class ValidationUtils
+{
+    private ValidationUtils()
+    {
+        // Utility class, not meant to be instantiated
+    }
+
+    /**
+     * Fetches keyspace metadata from Cassandra for the given host and 
keyspace name.
+     *
+     * @param metadataFetcher the metadata fetcher
+     * @param executorPools   the executor pools
+     * @param host            the host to fetch metadata from
+     * @param keyspace        the keyspace name
+     * @return a Future containing the KeyspaceMetadata, or null if the 
keyspace doesn't exist
+     */
+    public static Future<KeyspaceMetadata> 
getKeyspaceMetadata(InstanceMetadataFetcher metadataFetcher,
+                                                              ExecutorPools 
executorPools,
+                                                              String host,
+                                                              String keyspace)
+    {
+        return executorPools.service().executeBlocking(() -> 
metadataFetcher.instance(host)
+                                                                          
.delegate()
+                                                                          
.metadata()
+                                                                          
.getKeyspace(keyspace));
+    }
+
+    /**
+     * Validates that a keyspace exists and stores its metadata in the routing 
context.
+     *
+     * @param context         the routing context
+     * @param metadataFetcher the metadata fetcher
+     * @param executorPools   the executor pools
+     * @param host            the host to validate against
+     * @param keyspace        the keyspace name to validate
+     * @return a Future that completes when validation is done
+     */
+    public static Future<KeyspaceMetadata> 
validateKeyspaceExists(RoutingContext context,
+                                                                 
InstanceMetadataFetcher metadataFetcher,
+                                                                 ExecutorPools 
executorPools,
+                                                                 String host,
+                                                                 String 
keyspace)
+    {
+        return getKeyspaceMetadata(metadataFetcher, executorPools, host, 
keyspace)
+               .compose(keyspaceMetadata -> {
+                   if (keyspaceMetadata == null)
+                   {
+                       
context.fail(wrapHttpException(HttpResponseStatus.NOT_FOUND,
+                                                     "Keyspace " + keyspace + 
" was not found"));
+                       return Future.failedFuture("Keyspace " + keyspace + " 
was not found");
+                   }
+                   else
+                   {
+                       RoutingContextUtils.put(context, 
RoutingContextUtils.SC_KEYSPACE_METADATA, keyspaceMetadata);
+                       return Future.succeededFuture(keyspaceMetadata);
+                   }
+               }, throwable -> {
+                   context.fail(throwable);

Review Comment:
   Since it is in utility class, I think it is better to leave it to the caller 
how to handle the failure. Caller can do additional things if needed (like 
logging or wrapping it up with HttpException etc...)



##########
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;
+
+    /**
+     * Constructs a handler with the provided {@code metadataFetcher}
+     *
+     * @param vertx                the vertx instance
+     * @param metadataFetcher      the metadata fetcher
+     * @param executorPools        executor pools for blocking executions
+     * @param serviceConfiguration configuration object holding config details 
of Sidecar
+     * @param validator            a validator instance to validate 
Cassandra-specific input
+     * @param jobManager           manager for long-running operational jobs
+     */
+    @Inject
+    protected RepairHandler(Vertx vertx,
+                            InstanceMetadataFetcher metadataFetcher,
+                            ExecutorPools executorPools,
+                            ServiceConfiguration serviceConfiguration,
+                            CassandraInputValidator validator,
+                            OperationalJobManager jobManager)
+    {
+        super(metadataFetcher, executorPools, validator);
+        this.vertx = vertx;
+        this.jobManager = jobManager;
+        this.config = serviceConfiguration;
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    protected RepairRequestParam extractParamsOrThrow(RoutingContext context)
+    {
+        Name keyspace = keyspace(context, true);
+        if (keyspace == null)
+        {
+            throw wrapHttpException(HttpResponseStatus.BAD_REQUEST, 
"'keyspace' is required but not supplied");
+        }
+
+        String bodyString = context.body().asString();
+        if (bodyString == null || bodyString.equalsIgnoreCase("null")) // json 
encoder writes null as "null"
+        {
+            logger.warn("Bad request to create repair job. Received null 
payload.");
+            throw wrapHttpException(HttpResponseStatus.BAD_REQUEST, 
"Unexpected null payload for request");
+        }
+
+        RepairPayload payload;
+        try
+        {
+            payload = Json.decodeValue(bodyString, RepairPayload.class);
+        }
+        catch (DecodeException decodeException)
+        {
+            logger.warn("Bad request to create repair job. Received invalid 
JSON payload.");
+            throw wrapHttpException(HttpResponseStatus.BAD_REQUEST,
+                                    "Invalid request payload",
+                                    decodeException);
+        }
+
+        return RepairRequestParam.from(keyspace, payload);
+    }
+
+    @Override
+    protected void handleInternal(RoutingContext context,
+                                  HttpServerRequest httpRequest,
+                                  @NotNull String host,
+                                  SocketAddress remoteAddress,
+                                  RepairRequestParam repairRequestParam)
+    {
+        StorageOperations operations = 
metadataFetcher.delegate(host).storageOperations();
+        RepairJob job = new RepairJob(vertx, config.repairConfiguration(), 
UUIDs.timeBased(), operations, repairRequestParam);
+        try
+        {
+            jobManager.trySubmitJob(job);
+        }
+        catch (OperationalJobConflictException oje)

Review Comment:
   got it!



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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, NEW_STATUS
+    }
+
+    /**
+     * 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();
+        }
+        final Promise<Void> repairPromise = Promise.promise();
+        final Promise<Boolean> maxWaitTimePromise = Promise.promise();
+        final long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                             d -> 
maxWaitTimePromise.tryComplete(true));
+
+        queryForCompletedRepair(repairPromise, cmd);
+        final boolean repairCompleted = repairPromise.future().isComplete();
+        
+        final Long periodicId;
+        if (!repairCompleted)
+        {
+            periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                id -> 
queryForCompletedRepair(repairPromise, cmd));
+            LOGGER.debug("Setting up periodic polling for repair job {}", 
this.jobId());
+        }
+        else
+        {
+            periodicId = null;
+            LOGGER.info("Repair job {} completed from eager status check", 
this.jobId());
+        }
+
+        repairPromise.future().onComplete(ar -> {
+            internalPool.cancelTimer(timerId);
+            if (periodicId != null)
+            {
+                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();
+                     });
+    }
+
+
+    /**
+     * {@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)
+        {
+            try
+            {
+                long startToken = Long.parseLong(repairPayload.startToken());
+                long endToken = Long.parseLong(repairPayload.endToken());
+                if (startToken >= endToken)
+                {
+                    throw new IllegalArgumentException("Start token must be 
less than end token. " +
+                                                      "Got start: " + 
startToken + ", end: " + endToken);
+                }
+                options.put(RepairOptions.RANGES.getValue(), 
repairPayload.startToken() + ":" + repairPayload.endToken());
+            }
+            catch (NumberFormatException e)
+            {
+                throw new IllegalArgumentException("Invalid token format. 
Tokens must be numeric values.", e);
+            }
+        }
+
+        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.shouldValidate() != null)
+        {
+            options.put(RepairOptions.PREVIEW.getValue(), 
PREVIEW_KIND_REPAIRED);
+        }
+        return options;
+    }
+
+    private int missingJobStatusAttempt = 0;

Review Comment:
   Use `AtomicInteger` instead?



##########
server/src/main/java/org/apache/cassandra/sidecar/job/RepairJob.java:
##########
@@ -0,0 +1,285 @@
+/*
+ * 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, NEW_STATUS
+    }
+
+    /**
+     * 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();
+        }
+        final Promise<Void> repairPromise = Promise.promise();
+        final Promise<Boolean> maxWaitTimePromise = Promise.promise();
+        final long timerId = 
internalPool.setTimer(config.maxRepairJobRuntime().toMillis(),
+                                             d -> 
maxWaitTimePromise.tryComplete(true));
+
+        queryForCompletedRepair(repairPromise, cmd);
+        final boolean repairCompleted = repairPromise.future().isComplete();
+        
+        final Long periodicId;
+        if (!repairCompleted)
+        {
+            periodicId = 
internalPool.setPeriodic(config.repairPollInterval().toMillis(),
+                                                id -> 
queryForCompletedRepair(repairPromise, cmd));
+            LOGGER.debug("Setting up periodic polling for repair job {}", 
this.jobId());
+        }
+        else
+        {
+            periodicId = null;
+            LOGGER.info("Repair job {} completed from eager status check", 
this.jobId());
+        }
+
+        repairPromise.future().onComplete(ar -> {
+            internalPool.cancelTimer(timerId);
+            if (periodicId != null)
+            {
+                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();
+                     });
+    }
+
+
+    /**
+     * {@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)
+        {
+            try
+            {
+                long startToken = Long.parseLong(repairPayload.startToken());
+                long endToken = Long.parseLong(repairPayload.endToken());
+                if (startToken >= endToken)
+                {
+                    throw new IllegalArgumentException("Start token must be 
less than end token. " +
+                                                      "Got start: " + 
startToken + ", end: " + endToken);
+                }
+                options.put(RepairOptions.RANGES.getValue(), 
repairPayload.startToken() + ":" + repairPayload.endToken());
+            }
+            catch (NumberFormatException e)
+            {
+                throw new IllegalArgumentException("Invalid token format. 
Tokens must be numeric values.", e);
+            }
+        }
+
+        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.shouldValidate() != null)
+        {
+            options.put(RepairOptions.PREVIEW.getValue(), 
PREVIEW_KIND_REPAIRED);
+        }
+        return options;
+    }
+
+    private int missingJobStatusAttempt = 0;
+    private static final int MAX_MISSING_JOB_RETRIES = 5;
+    
+    private synchronized void queryForCompletedRepair(Promise<Void> promise, 
int cmd)

Review Comment:
   If `queryForCompletedRepair` is not called explicitly (like suggested for 
line: 119), then we may not need synchornized.



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