frankgh commented on code in PR #139:
URL: https://github.com/apache/cassandra-sidecar/pull/139#discussion_r1829892416
##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/http/SidecarHttpHeaderNames.java:
##########
@@ -31,4 +31,9 @@ public final class SidecarHttpHeaderNames
* {@code "cassandra-content-xxhash32-seed"}
*/
public static final String CONTENT_XXHASH32_SEED =
"cassandra-content-xxhash32-seed";
+
+ /**
+ * {@code "cassandra-sidecar-job-uuid"}
+ */
+ public static final String ASYNC_JOB_UUID = "cassandra-sidecar-job-uuid";
Review Comment:
I don't think this should be a header name, instead we should return a the
UUID as part of the JSON payload
##########
server/src/main/java/org/apache/cassandra/sidecar/job/JobManager.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.util.List;
+import java.util.UUID;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.inject.Inject;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.Singleton;
+import org.apache.cassandra.sidecar.common.utils.JobResult.JobStatus;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+
+/**
+ * An abstraction of the management and tracking of long-running jobs running
on the sidecar.
+ */
+@Singleton
+public class JobManager
+{
+ private static final Logger LOGGER =
LoggerFactory.getLogger(JobManager.class);
+
+ private final JobTracker jobTracker;
+ private final ExecutorPools executorPools;
+
+ /**
+ * Creates a manager instance with a default sized job-tracker.
+ * @param executorPools
+ */
+ @Inject
+ public JobManager(ExecutorPools executorPools)
+ {
+ this.executorPools = executorPools;
+ jobTracker = new JobTracker(64);
Review Comment:
shouldn't job tracker be injected and we should use the singleton instance?
##########
server/src/main/java/org/apache/cassandra/sidecar/job/JobTracker.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Tracks and stores the results of long-running jobs running on the sidecar
+ */
+public class JobTracker extends LinkedHashMap<UUID, Job>
+{
+
+ private static final Logger LOGGER =
LoggerFactory.getLogger(JobTracker.class);
+ int capacity;
+ private final ReadWriteLock lock = new ReentrantReadWriteLock();
+
+ public JobTracker(final int initialCapacity)
Review Comment:
let's remove unnecessary `final` usage in this class
##########
server/src/main/java/org/apache/cassandra/sidecar/server/MainModule.java:
##########
@@ -588,6 +599,13 @@ public LocalTokenRangesProvider
localTokenRangesProvider(InstancesConfig instanc
return new CachedLocalTokenRanges(instancesConfig, dnsResolver);
}
+ @Provides
+ @Singleton
+ public JobTracker jobTracker()
+ {
+ return new JobTracker(64);
Review Comment:
should we make this configurable?
##########
server/src/main/java/org/apache/cassandra/sidecar/routes/JobStatusHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.util.UUID;
+import javax.inject.Inject;
+
+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.response.JobStatusResponse;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.job.Job;
+import org.apache.cassandra.sidecar.job.JobManager;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static
org.apache.cassandra.sidecar.common.http.SidecarHttpHeaderNames.ASYNC_JOB_UUID;
+import static
org.apache.cassandra.sidecar.utils.HttpExceptions.wrapHttpException;
+
+/**
+ * Handler for retrieving the status of async jobs running on the sidecar
+ */
+public class JobStatusHandler extends AbstractHandler<Void>
+{
+ private final JobManager jobManager;
+ @Inject
+ public JobStatusHandler(InstanceMetadataFetcher metadataFetcher,
ExecutorPools executorPools, CassandraInputValidator validator, JobManager
jobManager)
+ {
+ super(metadataFetcher, executorPools, validator);
+ this.jobManager = jobManager;
+ }
+
+ protected Void extractParamsOrThrow(RoutingContext context)
+ {
+ return null;
+ }
+
+ @Override
+ public void handleInternal(RoutingContext context, HttpServerRequest
httpRequest, String host, SocketAddress remoteAddress, Void request)
+ {
+ UUID jobUUID = verifyJobIdPresent(context);
+
+ executorPools.service().executeBlocking(() -> {
Review Comment:
this is getting data from memory, no need to execute it in the service
executor pool
##########
server/src/main/java/org/apache/cassandra/sidecar/routes/ListJobsHandler.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.util.List;
+import javax.inject.Inject;
+
+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.response.ListJobsResponse;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.job.Job;
+import org.apache.cassandra.sidecar.job.JobManager;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+/**
+ * Handler for retrieving the all the jobs running on the sidecar
+ */
+public class ListJobsHandler extends AbstractHandler<Void>
+{
+ private final JobManager jobManager;
+ @Inject
+ public ListJobsHandler(InstanceMetadataFetcher metadataFetcher,
ExecutorPools executorPools, CassandraInputValidator validator, JobManager
jobManager)
+ {
+ super(metadataFetcher, executorPools, validator);
+ this.jobManager = jobManager;
+ }
+
+ protected Void extractParamsOrThrow(RoutingContext context)
Review Comment:
let's add the `@Override` annotation.
##########
server/src/main/java/org/apache/cassandra/sidecar/routes/ListJobsHandler.java:
##########
@@ -0,0 +1,70 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.util.List;
+import javax.inject.Inject;
+
+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.response.ListJobsResponse;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.job.Job;
+import org.apache.cassandra.sidecar.job.JobManager;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+/**
+ * Handler for retrieving the all the jobs running on the sidecar
+ */
+public class ListJobsHandler extends AbstractHandler<Void>
+{
+ private final JobManager jobManager;
+ @Inject
+ public ListJobsHandler(InstanceMetadataFetcher metadataFetcher,
ExecutorPools executorPools, CassandraInputValidator validator, JobManager
jobManager)
+ {
+ super(metadataFetcher, executorPools, validator);
+ this.jobManager = jobManager;
+ }
+
+ protected Void extractParamsOrThrow(RoutingContext context)
+ {
+ return null;
+ }
+
+ @Override
+ protected void handleInternal(RoutingContext context, HttpServerRequest
httpRequest, String host, SocketAddress remoteAddress, Void request)
+ {
+ executorPools.service()
Review Comment:
There's no need to run this on the service executor pool. These jobs are in
memory.
##########
server/src/main/java/org/apache/cassandra/sidecar/routes/JobStatusHandler.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.sidecar.routes;
+
+import java.util.UUID;
+import javax.inject.Inject;
+
+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.response.JobStatusResponse;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+import org.apache.cassandra.sidecar.job.Job;
+import org.apache.cassandra.sidecar.job.JobManager;
+import org.apache.cassandra.sidecar.utils.CassandraInputValidator;
+import org.apache.cassandra.sidecar.utils.InstanceMetadataFetcher;
+
+import static
org.apache.cassandra.sidecar.common.http.SidecarHttpHeaderNames.ASYNC_JOB_UUID;
+import static
org.apache.cassandra.sidecar.utils.HttpExceptions.wrapHttpException;
+
+/**
+ * Handler for retrieving the status of async jobs running on the sidecar
+ */
+public class JobStatusHandler extends AbstractHandler<Void>
+{
+ private final JobManager jobManager;
+ @Inject
+ public JobStatusHandler(InstanceMetadataFetcher metadataFetcher,
ExecutorPools executorPools, CassandraInputValidator validator, JobManager
jobManager)
+ {
+ super(metadataFetcher, executorPools, validator);
+ this.jobManager = jobManager;
+ }
+
+ protected Void extractParamsOrThrow(RoutingContext context)
Review Comment:
this should return the jobUUID
##########
server/src/main/java/org/apache/cassandra/sidecar/job/JobManager.java:
##########
@@ -0,0 +1,102 @@
+/*
+ * 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.util.List;
+import java.util.UUID;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import javax.inject.Inject;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.inject.Singleton;
+import org.apache.cassandra.sidecar.common.utils.JobResult.JobStatus;
+import org.apache.cassandra.sidecar.concurrent.ExecutorPools;
+
+/**
+ * An abstraction of the management and tracking of long-running jobs running
on the sidecar.
+ */
+@Singleton
+public class JobManager
+{
+ private static final Logger LOGGER =
LoggerFactory.getLogger(JobManager.class);
+
+ private final JobTracker jobTracker;
+ private final ExecutorPools executorPools;
+
+ /**
+ * Creates a manager instance with a default sized job-tracker.
+ * @param executorPools
+ */
+ @Inject
+ public JobManager(ExecutorPools executorPools)
+ {
+ this.executorPools = executorPools;
+ jobTracker = new JobTracker(64);
+ }
+
+ /**
+ * Fetches the inflight jobs being tracked on the sidecar
+ * @return instances of the jobs that are in pending or running states
+ */
+ public List<Job> allInflightJobs()
+ {
+ return jobTracker.getJobsView().values()
+ .stream()
+ .filter(j -> j.status() == JobStatus.Pending ||
j.status() == JobStatus.Running)
+ .collect(Collectors.toList());
+ }
+
+ /**
+ * Fetch the job using its UUID
+ * @param jobId identifier of the job
+ * @return instance of the job or null
+ */
+ public Job getJobIfExists(UUID jobId)
+ {
+ return jobTracker.get(jobId);
+ }
+
+ /**
+ * Asynchronously submit (and lazily create, via the supplier) the job, if
it is not currently being
+ * tracked and is not running downstream. The job is triggered on a
separate internal thread-pool.
+ * The job execution failure behavior is tracked within the {@link Job}.
+ * @param jobId job identifier
+ * @param jobSupplier supplier used to create an instance of the job
+ * @return the instance of the job that is either being tracked or was
just submitted
+ */
+ public Job trySubmitJob(UUID jobId, Supplier<Job> jobSupplier)
+ {
+
+ return jobTracker.computeIfAbsent(jobId, id -> {
+ Job job = jobSupplier.get();
+ LOGGER.info("Created job with ID: {}, operation: {}", job.jobId(),
job.operation());
+ jobTracker.put(job.jobId(), job);
Review Comment:
this seems wrong, you shouldn't call put here, that's the job of
computeIfAbsent, no?
##########
server/src/main/java/org/apache/cassandra/sidecar/job/Job.java:
##########
@@ -0,0 +1,152 @@
+/*
+ * 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.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.function.Supplier;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.sidecar.common.utils.JobResult;
+import org.apache.cassandra.sidecar.common.utils.JobResult.JobStatus;
+
+/**
+ * An abstract class representing a Job managed by the sidecar.
+ *
+ */
+public abstract class Job
+{
+ private static final Logger LOGGER = LoggerFactory.getLogger(Job.class);
+
+ protected UUID jobId;
+ protected JobStatus status;
+ protected String failureReason;
+ final CountDownLatch latch = new CountDownLatch(1);
+
+ protected Job()
+ {
+
+ }
+ /**
+ * Constructs a job with a unique UUID, in Pending state
+ * @param jobId UUID representing the Job to be created
+ */
+ protected Job(UUID jobId)
+ {
+ this.jobId = jobId;
+ this.status = JobStatus.Pending;
+ this.failureReason = "";
+ }
+
+ @VisibleForTesting
+ protected Job(UUID jobId, JobStatus status)
+ {
+ this.jobId = jobId;
+ this.status = status;
+ this.failureReason = "";
+ }
+
+ public void setStatus(JobStatus status)
+ {
+ this.status = status;
+ }
+ public JobStatus status()
+ {
+ return status;
+ }
+ public String failureReason()
+ {
+ return failureReason;
+ }
+
+ public UUID jobId()
+ {
+ return jobId;
+ }
+
+ /**
+ * Supplier specifying the functionality of the job to be triggered when
the job is executed. Subclasses to
+ * provide operation-specific implementations
+ * @return a function with the operation implementation that returns a
{@code JobResult}
+ * @throws Exception
+ */
+ public abstract Supplier<JobResult> jobOperationSupplier() throws
Exception;
+
+ /**
+ * Provide a meaningful name of the operation executed by the concrete
subclass.
+ * @return the name of the operation. eg. nodetool command name
+ */
+ public abstract String operation();
+
+ /**
+ * Specifies the downstream operation to be performed to check if the job
is running on the Cassandra node/cluster.
+ * This functionality is provided by the Job when it is asynchronously
triggering the job via the {@code JobManager}
+ * For synchronous jobs this should always return false.
+ * @return true if the job is running downstream
+ */
+ public abstract boolean checkInflightJob();
+
+ /**
+ * Execute the job behavior as specified in the operation supplier {@link
#jobOperationSupplier()},
+ * while tracking the status of the job's lifecycle.
+ */
+ public void execute()
+ {
+ try
+ {
+ LOGGER.info("Executing job with ID: {}", jobId);
+ JobResult result = jobOperationSupplier().get();
Review Comment:
it feels we should use Future primitives here instead, execute should return
a future as well, instead of doing latch countdown
##########
client-common/src/main/java/org/apache/cassandra/sidecar/common/ApiEndpointsV1.java:
##########
@@ -109,6 +109,9 @@ public final class ApiEndpointsV1
public static final String ABORT_RESTORE_JOB_ROUTE = RESTORE_JOB_ROUTE +
ABORT;
public static final String RESTORE_JOB_PROGRESS_ROUTE = RESTORE_JOB_ROUTE
+ PROGRESS;
+ public static final String LIST_JOBS_ROUTE = API_V1 + CASSANDRA + "/jobs";
+ public static final String JOB_STATUS_ROUTE = API_V1 + CASSANDRA +
"/jobs/" + JOB_ID_PATH_PARAM + "/status";
Review Comment:
I think we should qualify these endpoints as admin. Also, I think we should
not have the `status` path part . I propose we use the following instead:
```
/api/v1/admin/cassandra/jobs
/api/v1/admin/cassandra/jobs/<job_id>
```
##########
server/src/main/java/org/apache/cassandra/sidecar/job/JobTracker.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.util.LinkedHashMap;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.locks.ReadWriteLock;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Tracks and stores the results of long-running jobs running on the sidecar
+ */
+public class JobTracker extends LinkedHashMap<UUID, Job>
Review Comment:
do we really need to expose all the methods from Map here? I only see usages
of `getJobsView` , `get`, `computeIfAbsent` (which is not even guarded by the
lock), and `put` which seems wrong to have . I think I'd prefer to have a
smaller API, that exposes only the things we need. Maybe we can simplify by
doing this:
```
Map<UUID, Job> internalTracker = Collections.synchronizedMap(new
LinkedHashMap<UUID, Job>()
{
@Override
protected boolean removeEldestEntry(Map.Entry<UUID, Job> eldest)
{
if (size() <= capacity)
{
return false;
}
LOGGER.warn("Job tracker reached max size, so expiring job
uuid={}", eldest.getKey());
return true;
}
});
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]