diff --git a/docs/api-reference/automatic-compaction-api.md b/docs/api-reference/automatic-compaction-api.md index c09183c09680..ae0595b00b86 100644 --- a/docs/api-reference/automatic-compaction-api.md +++ b/docs/api-reference/automatic-compaction-api.md @@ -190,6 +190,10 @@ A successful request returns an HTTP `200 OK` message code and an empty response ### Update capacity for compaction tasks +:::info +This API is now deprecated. Use [Update cluster-level compaction config](#update-cluster-level-compaction-config) instead. +::: + Updates the capacity for compaction tasks. The minimum number of compaction tasks is 1 and the maximum is 2147483647. Note that while the max compaction tasks can theoretically be set to 2147483647, the practical limit is determined by the available cluster capacity and is capped at 10% of the cluster's total capacity. @@ -258,102 +262,6 @@ Host: http://ROUTER_IP:ROUTER_PORT A successful request returns an HTTP `200 OK` message code and an empty response body. -### Update cluster-level compaction config - -Updates cluster-level configuration for compaction tasks which applies to all datasources, unless explicitly overridden in the datasource compaction config. -This includes the following fields: - -|Config|Description|Default value| -|------|-----------|-------------| -|`compactionTaskSlotRatio`|Ratio of number of slots taken up by compaction tasks to the number of total task slots across all workers.|0.1| -|`maxCompactionTaskSlots`|Maximum number of task slots that can be taken up by compaction tasks and sub-tasks. Minimum number of task slots available for compaction is 1. When using MSQ engine or Native engine with range partitioning, a single compaction job occupies more than one task slot. In this case, the minimum is 2 so that at least one compaction job can always run in the cluster.|2147483647 (i.e. total task slots)| -|`compactionPolicy`|Policy to choose intervals for compaction. Currently, the only supported policy is [Newest segment first](#compaction-policy-newestsegmentfirst).|Newest segment first| -|`useSupervisors`|Whether compaction should be run on Overlord using supervisors instead of Coordinator duties.|false| -|`engine`|Engine used for running compaction tasks, unless overridden in the datasource-level compaction config. Possible values are `native` and `msq`. `msq` engine can be used for compaction only if `useSupervisors` is `true`.|`native`| - -#### Compaction policy `newestSegmentFirst` - -|Field|Description|Default value| -|-----|-----------|-------------| -|`type`|This must always be `newestSegmentFirst`|| -|`priorityDatasource`|Datasource to prioritize for compaction. The intervals of this datasource are chosen for compaction before the intervals of any other datasource. Within this datasource, the intervals are prioritized based on the chosen compaction policy.|None| - - -#### URL - -`POST` `/druid/coordinator/v1/config/compaction/cluster` - -#### Responses - - - - - - -*Successfully updated compaction configuration* - - - - - -*Invalid `max` value* - - - - ---- - -#### Sample request - - - - - - -```shell -curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction/cluster" \ ---header 'Content-Type: application/json' \ ---data '{ - "compactionTaskSlotRatio": 0.5, - "maxCompactionTaskSlots": 1500, - "compactionPolicy": { - "type": "newestSegmentFirst", - "priorityDatasource": "wikipedia" - }, - "useSupervisors": true, - "engine": "msq" -}' - -``` - - - - - -```HTTP -POST /druid/coordinator/v1/config/compaction/cluster HTTP/1.1 -Host: http://ROUTER_IP:ROUTER_PORT -Content-Type: application/json - -{ - "compactionTaskSlotRatio": 0.5, - "maxCompactionTaskSlots": 1500, - "compactionPolicy": { - "type": "newestSegmentFirst", - "priorityDatasource": "wikipedia" - }, - "useSupervisors": true, - "engine": "msq" -} -``` - - - - -#### Sample response - -A successful request returns an HTTP `200 OK` message code and an empty response body. - ## View automatic compaction configuration ### Get all automatic compaction configurations @@ -863,6 +771,8 @@ Host: http://ROUTER_IP:ROUTER_PORT Retrieves an array of `latestStatus` objects representing the status and statistics from the latest automatic compaction run for all datasources with automatic compaction enabled. +#### Compaction status response + The `latestStatus` object has the following properties: * `dataSource`: Name of the datasource for this status information. * `scheduleStatus`: Automatic compaction scheduling status. Possible values are `NOT_ENABLED` and `RUNNING`. Returns `RUNNING ` if the datasource has an active automatic compaction configuration submitted. Otherwise, returns `NOT_ENABLED`. @@ -959,3 +869,724 @@ Host: http://ROUTER_IP:ROUTER_PORT } ``` + +## [Experimental] Unified Compaction APIs + +This section describes the new unified compaction APIs which can be used regardless of whether compaction supervisors are enabled (i.e. `useSupervisors` is `true`) or not in the compaction dynamic config. + +- If compaction supervisors are disabled, the APIs read or write the compaction dynamic config, same as the Coordinator-based compaction APIs above. +- If compaction supervisors are enabled, the APIs read or write the corresponding compaction supervisors. In conjunction with the APIs described below, the supervisor APIs may also be used to read or write the compaction supervisors as they offer greater flexibility and also serve information related to supervisor and task statuses. + +### Update cluster-level compaction config + +Updates cluster-level configuration for compaction tasks which applies to all datasources, unless explicitly overridden in the datasource compaction config. +This includes the following fields: + +|Config|Description|Default value| +|------|-----------|-------------| +|`compactionTaskSlotRatio`|Ratio of number of slots taken up by compaction tasks to the number of total task slots across all workers.|0.1| +|`maxCompactionTaskSlots`|Maximum number of task slots that can be taken up by compaction tasks and sub-tasks. Minimum number of task slots available for compaction is 1. When using MSQ engine or Native engine with range partitioning, a single compaction job occupies more than one task slot. In this case, the minimum is 2 so that at least one compaction job can always run in the cluster.|2147483647 (i.e. total task slots)| +|`compactionPolicy`|Policy to choose intervals for compaction. Currently, the only supported policy is [Newest segment first](#compaction-policy-newestsegmentfirst).|Newest segment first| +|`useSupervisors`|Whether compaction should be run on Overlord using supervisors instead of Coordinator duties.|false| +|`engine`|Engine used for running compaction tasks, unless overridden in the datasource-level compaction config. Possible values are `native` and `msq`. `msq` engine can be used for compaction only if `useSupervisors` is `true`.|`native`| + +#### Compaction policy `newestSegmentFirst` + +|Field|Description|Default value| +|-----|-----------|-------------| +|`type`|This must always be `newestSegmentFirst`|| +|`priorityDatasource`|Datasource to prioritize for compaction. The intervals of this datasource are chosen for compaction before the intervals of any other datasource. Within this datasource, the intervals are prioritized based on the chosen compaction policy.|None| + + +#### URL + +`POST` `/druid/indexer/v1/compaction/config/cluster` + +#### Responses + + + + + + +*Successfully updated compaction configuration* + + + + + +*Invalid `max` value* + + + + +--- + +#### Sample request + + + + + + +```shell +curl --request POST "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config/compaction/cluster" \ +--header 'Content-Type: application/json' \ +--data '{ + "compactionTaskSlotRatio": 0.5, + "maxCompactionTaskSlots": 1500, + "compactionPolicy": { + "type": "newestSegmentFirst", + "priorityDatasource": "wikipedia" + }, + "useSupervisors": true, + "engine": "msq" +}' + +``` + + + + + +```HTTP +POST /druid/indexer/v1/compaction/config/cluster HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +Content-Type: application/json + +{ + "compactionTaskSlotRatio": 0.5, + "maxCompactionTaskSlots": 1500, + "compactionPolicy": { + "type": "newestSegmentFirst", + "priorityDatasource": "wikipedia" + }, + "useSupervisors": true, + "engine": "msq" +} +``` + + + + +#### Sample response + +A successful request returns an HTTP `200 OK` message code and an empty response body. + +### Get cluster-level compaction config + +Retrieves cluster-level configuration for compaction tasks which applies to all datasources, unless explicitly overridden in the datasource compaction config. +This includes all the fields listed in [Update cluster-level compaction config](#update-cluster-level-compaction-config). + +#### URL + +`GET` `/druid/indexer/v1/compaction/config/cluster` + +#### Responses + + + + + +*Successfully retrieved cluster compaction configuration* + + + + +--- + +#### Sample request + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/config/cluster" +``` + + + + +```HTTP +GET /druid/indexer/v1/compaction/config/cluster HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ View the response + +```json +{ + "compactionTaskSlotRatio": 0.5, + "maxCompactionTaskSlots": 1500, + "compactionPolicy": { + "type": "newestSegmentFirst", + "priorityDatasource": "wikipedia" + }, + "useSupervisors": true, + "engine": "msq" +} +``` + +
+ +### Get automatic compaction configurations for all datasources + +Retrieves all datasource compaction configurations. + +#### URL + +`GET` `/druid/indexer/v1/compaction/config/datasources` + +#### Responses + + + + + + +*Successfully retrieved automatic compaction configurations* + + + + +--- + +#### Sample request + + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/config/datasources" +``` + + + + + +```HTTP +GET /druid/indexer/v1/compaction/config/datasources HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ View the response + +```json +{ + "compactionConfigs": [ + { + "dataSource": "wikipedia_hour", + "taskPriority": 25, + "inputSegmentSizeBytes": 100000000000000, + "maxRowsPerSegment": null, + "skipOffsetFromLatest": "PT0S", + "tuningConfig": { + "maxRowsInMemory": null, + "appendableIndexSpec": null, + "maxBytesInMemory": null, + "maxTotalRows": null, + "splitHintSpec": null, + "partitionsSpec": { + "type": "dynamic", + "maxRowsPerSegment": 5000000, + "maxTotalRows": null + }, + "indexSpec": null, + "indexSpecForIntermediatePersists": null, + "maxPendingPersists": null, + "pushTimeout": null, + "segmentWriteOutMediumFactory": null, + "maxNumConcurrentSubTasks": null, + "maxRetry": null, + "taskStatusCheckPeriodMs": null, + "chatHandlerTimeout": null, + "chatHandlerNumRetries": null, + "maxNumSegmentsToMerge": null, + "totalNumMergeTasks": null, + "maxColumnsToMerge": null, + "type": "index_parallel", + "forceGuaranteedRollup": false + }, + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": null, + "rollup": null + }, + "dimensionsSpec": null, + "metricsSpec": null, + "transformSpec": null, + "ioConfig": null, + "taskContext": null + }, + { + "dataSource": "wikipedia", + "taskPriority": 25, + "inputSegmentSizeBytes": 100000000000000, + "maxRowsPerSegment": null, + "skipOffsetFromLatest": "PT0S", + "tuningConfig": { + "maxRowsInMemory": null, + "appendableIndexSpec": null, + "maxBytesInMemory": null, + "maxTotalRows": null, + "splitHintSpec": null, + "partitionsSpec": { + "type": "dynamic", + "maxRowsPerSegment": 5000000, + "maxTotalRows": null + }, + "indexSpec": null, + "indexSpecForIntermediatePersists": null, + "maxPendingPersists": null, + "pushTimeout": null, + "segmentWriteOutMediumFactory": null, + "maxNumConcurrentSubTasks": null, + "maxRetry": null, + "taskStatusCheckPeriodMs": null, + "chatHandlerTimeout": null, + "chatHandlerNumRetries": null, + "maxNumSegmentsToMerge": null, + "totalNumMergeTasks": null, + "maxColumnsToMerge": null, + "type": "index_parallel", + "forceGuaranteedRollup": false + }, + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": null, + "rollup": null + }, + "dimensionsSpec": null, + "metricsSpec": null, + "transformSpec": null, + "ioConfig": null, + "taskContext": null + } + ] +} +``` +
+ +### Get automatic compaction configuration for a datasource + +Retrieves the automatic compaction configuration for a datasource. + +#### URL + +`GET` `/druid/indexer/v1/compaction/config/datasources/{dataSource}` + +#### Responses + + + + + + +*Successfully retrieved configuration for datasource* + + + + + +*Invalid datasource or datasource does not have automatic compaction enabled* + + + + +--- + +#### Sample request + +The following example retrieves the automatic compaction configuration for datasource `wikipedia_hour`. + + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/config/datasources/wikipedia_hour" +``` + + + + + +```HTTP +GET /druid/indexer/v1/compaction/config/datasources/wikipedia_hour HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ View the response + +```json +{ + "dataSource": "wikipedia_hour", + "taskPriority": 25, + "inputSegmentSizeBytes": 100000000000000, + "maxRowsPerSegment": null, + "skipOffsetFromLatest": "PT0S", + "tuningConfig": { + "maxRowsInMemory": null, + "appendableIndexSpec": null, + "maxBytesInMemory": null, + "maxTotalRows": null, + "splitHintSpec": null, + "partitionsSpec": { + "type": "dynamic", + "maxRowsPerSegment": 5000000, + "maxTotalRows": null + }, + "indexSpec": null, + "indexSpecForIntermediatePersists": null, + "maxPendingPersists": null, + "pushTimeout": null, + "segmentWriteOutMediumFactory": null, + "maxNumConcurrentSubTasks": null, + "maxRetry": null, + "taskStatusCheckPeriodMs": null, + "chatHandlerTimeout": null, + "chatHandlerNumRetries": null, + "maxNumSegmentsToMerge": null, + "totalNumMergeTasks": null, + "maxColumnsToMerge": null, + "type": "index_parallel", + "forceGuaranteedRollup": false + }, + "granularitySpec": { + "segmentGranularity": "DAY", + "queryGranularity": null, + "rollup": null + }, + "dimensionsSpec": null, + "metricsSpec": null, + "transformSpec": null, + "ioConfig": null, + "taskContext": null +} +``` +
+ +### Create or update automatic compaction configuration for a datasource + +Creates or updates the automatic compaction configuration for a datasource. Pass the automatic compaction as a JSON object in the request body. + +The automatic compaction configuration requires only the `dataSource` property. Druid fills all other properties with default values if not specified. See [Automatic compaction dynamic configuration](../configuration/index.md#automatic-compaction-dynamic-configuration) for configuration details. + +Note that this endpoint returns an HTTP `200 OK` message code even if the datasource name does not exist. + +#### URL + +`POST` `/druid/indexer/v1/compaction/config/datasources/wikipedia_hour` + +#### Responses + + + + + + +*Successfully submitted auto compaction configuration* + + + + +--- +#### Sample request + +The following example creates an automatic compaction configuration for the datasource `wikipedia_hour`, which was ingested with `HOUR` segment granularity. This automatic compaction configuration performs compaction on `wikipedia_hour`, resulting in compacted segments that represent a day interval of data. + +In this example: + +* `wikipedia_hour` is a datasource with `HOUR` segment granularity. +* `skipOffsetFromLatest` is set to `PT0S`, meaning that no data is skipped. +* `partitionsSpec` is set to the default `dynamic`, allowing Druid to dynamically determine the optimal partitioning strategy. +* `type` is set to `index_parallel`, meaning that parallel indexing is used. +* `segmentGranularity` is set to `DAY`, meaning that each compacted segment is a day of data. + + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/config/datasources/wikipedia_hour"\ +--header 'Content-Type: application/json' \ +--data '{ + "dataSource": "wikipedia_hour", + "skipOffsetFromLatest": "PT0S", + "tuningConfig": { + "partitionsSpec": { + "type": "dynamic" + }, + "type": "index_parallel" + }, + "granularitySpec": { + "segmentGranularity": "DAY" + } +}' +``` + + + + + +```HTTP +POST /druid/indexer/v1/compaction/config/datasources/wikipedia_hour HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +Content-Type: application/json +Content-Length: 281 + +{ + "dataSource": "wikipedia_hour", + "skipOffsetFromLatest": "PT0S", + "tuningConfig": { + "partitionsSpec": { + "type": "dynamic" + }, + "type": "index_parallel" + }, + "granularitySpec": { + "segmentGranularity": "DAY" + } +} +``` + + + + +#### Sample response + +A successful request returns an HTTP `200 OK` message code and an empty response body. + + +### Delete automatic compaction configuration for a datasource + +Removes the automatic compaction configuration for a datasource. This updates the compaction status of the datasource to "Not enabled." + +#### URL + +`DELETE` `/druid/indexer/v1/compaction/config/datasources/{dataSource}` + +#### Responses + + + + + + +*Successfully deleted automatic compaction configuration* + + + + + +*Datasource does not have automatic compaction or invalid datasource name* + + + + +--- + + +#### Sample request + + + + + + +```shell +curl --request DELETE "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/config/datasources/wikipedia_hour" +``` + + + + + +```HTTP +DELETE /druid/indexer/v1/compaction/config/wikipedia_hour HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +A successful request returns an HTTP `200 OK` message code and an empty response body. + +### Get compaction status for all datasources + +Retrieves an array of `latestStatus` objects representing the status and statistics from the latest automatic compaction run for all the datasources to which the user has read access. +The response payload is in the same format as [Compaction status response](#compaction-status-response). + +#### URL + +`GET` `/druid/indexer/v1/compaction/status/datasources` + +#### Responses + + + + + + +*Successfully retrieved `latestStatus` object* + + + + +--- +#### Sample request + + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/status/datasources" +``` + + + + + +```HTTP +GET /druid/indexer/v1/compaction/status/datasources HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ View the response + +```json +{ + "latestStatus": [ + { + "dataSource": "wikipedia_api", + "scheduleStatus": "RUNNING", + "bytesAwaitingCompaction": 0, + "bytesCompacted": 0, + "bytesSkipped": 64133616, + "segmentCountAwaitingCompaction": 0, + "segmentCountCompacted": 0, + "segmentCountSkipped": 8, + "intervalCountAwaitingCompaction": 0, + "intervalCountCompacted": 0, + "intervalCountSkipped": 1 + }, + { + "dataSource": "wikipedia_hour", + "scheduleStatus": "RUNNING", + "bytesAwaitingCompaction": 0, + "bytesCompacted": 5998634, + "bytesSkipped": 0, + "segmentCountAwaitingCompaction": 0, + "segmentCountCompacted": 1, + "segmentCountSkipped": 0, + "intervalCountAwaitingCompaction": 0, + "intervalCountCompacted": 1, + "intervalCountSkipped": 0 + } + ] +} +``` +
+ +### Get compaction status for a single datasource + +Retrieves the latest status from the latest automatic compaction run for a datasource. The response payload is in the same format as [Compaction status response](#compaction-status-response) with zero or one entry. + +#### URL + +`GET` `/druid/indexer/v1/compaction/status/datasources/{dataSource}` + +#### Responses + + + + + + +*Successfully retrieved `latestStatus` object* + + + + +--- +#### Sample request + + + + + + +```shell +curl "http://ROUTER_IP:ROUTER_PORT/druid/indexer/v1/compaction/status/datasources/wikipedia_hour" +``` + + + + + +```HTTP +GET /druid/indexer/v1/compaction/status/datasources/wikipedia_hour HTTP/1.1 +Host: http://ROUTER_IP:ROUTER_PORT +``` + + + + +#### Sample response + +
+ View the response + +```json +{ + "latestStatus": [ + { + "dataSource": "wikipedia_hour", + "scheduleStatus": "RUNNING", + "bytesAwaitingCompaction": 0, + "bytesCompacted": 5998634, + "bytesSkipped": 0, + "segmentCountAwaitingCompaction": 0, + "segmentCountCompacted": 1, + "segmentCountSkipped": 0, + "intervalCountAwaitingCompaction": 0, + "intervalCountCompacted": 1, + "intervalCountSkipped": 0 + } + ] +} +``` +
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java index 0df084f362c3..5f0aa6e3ea2b 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionScheduler.java @@ -44,24 +44,46 @@ */ public interface CompactionScheduler { - void start(); + void becomeLeader(); - void stop(); + void stopBeingLeader(); + /** + * @return true if the scheduler is enabled i.e. when + * {@link DruidCompactionConfig#isUseSupervisors()} is true. + */ boolean isEnabled(); + /** + * @return true if the scheduler is currently running and submitting compaction + * tasks. + */ boolean isRunning(); CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig); + /** + * Starts compaction for a datasource if not already running. + */ void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig); + /** + * Stops compaction for a datasource if currently running. + */ void stopCompaction(String dataSourceName); Map getAllCompactionSnapshots(); + /** + * @return Non-null snapshot of the current status of compaction for the datasource. + */ AutoCompactionSnapshot getCompactionSnapshot(String dataSource); + /** + * Simulates a compaction run with the given cluster config. + * + * @return Result of the simulation + */ CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java index 8104ccde470a..851b3920b1a1 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisor.java @@ -83,7 +83,7 @@ public SupervisorReport getStatus() final AutoCompactionSnapshot snapshot; if (supervisorSpec.isSuspended()) { snapshot = AutoCompactionSnapshot.builder(dataSource) - .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.NOT_ENABLED) + .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) .build(); } else if (!supervisorSpec.getValidationResult().isValid()) { snapshot = AutoCompactionSnapshot.builder(dataSource) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java index 66e54e971cdb..71864c333222 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/CompactionSupervisorSpec.java @@ -30,6 +30,7 @@ import javax.annotation.Nullable; import java.util.Collections; import java.util.List; +import java.util.Objects; public class CompactionSupervisorSpec implements SupervisorSpec { @@ -41,6 +42,11 @@ public class CompactionSupervisorSpec implements SupervisorSpec private final CompactionScheduler scheduler; private final CompactionConfigValidationResult validationResult; + public static String getSupervisorIdForDatasource(String dataSource) + { + return ID_PREFIX + dataSource; + } + @JsonCreator public CompactionSupervisorSpec( @JsonProperty("spec") DataSourceCompactionConfig spec, @@ -70,7 +76,7 @@ public boolean isSuspended() @Override public String getId() { - return ID_PREFIX + spec.getDataSource(); + return getSupervisorIdForDatasource(spec.getDataSource()); } public CompactionConfigValidationResult getValidationResult() @@ -113,4 +119,23 @@ public String getSource() { return ""; } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompactionSupervisorSpec that = (CompactionSupervisorSpec) o; + return suspended == that.suspended && Objects.equals(spec, that.spec); + } + + @Override + public int hashCode() + { + return Objects.hash(suspended, spec); + } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java index 83a2c5f9f579..9db8a73d3810 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/compact/OverlordCompactionScheduler.java @@ -34,6 +34,8 @@ import org.apache.druid.java.util.common.Stopwatch; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import org.apache.druid.java.util.common.lifecycle.LifecycleStart; +import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; @@ -103,6 +105,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler */ private final TaskRunnerListener taskRunnerListener; + private final AtomicBoolean isLeader = new AtomicBoolean(false); private final AtomicBoolean started = new AtomicBoolean(false); private final CompactSegments duty; @@ -165,29 +168,36 @@ public void statusChanged(String taskId, TaskStatus status) }; } + @LifecycleStart + public synchronized void start() + { + // Do nothing + } + + @LifecycleStop + public synchronized void stop() + { + executor.shutdownNow(); + } + @Override - public void start() + public void becomeLeader() { - if (isEnabled() && started.compareAndSet(false, true)) { - log.info("Starting compaction scheduler."); - initState(); - scheduleOnExecutor(this::scheduledRun); + if (isLeader.compareAndSet(false, true)) { + scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); } } @Override - public void stop() + public void stopBeingLeader() { - if (isEnabled() && started.compareAndSet(true, false)) { - log.info("Stopping compaction scheduler."); - cleanupState(); - } + isLeader.set(false); } @Override public boolean isRunning() { - return isEnabled() && started.get(); + return started.get(); } @Override @@ -220,8 +230,16 @@ public void stopCompaction(String dataSourceName) statusTracker.removeDatasource(dataSourceName); } + /** + * Initializes scheduler state if required. + */ private synchronized void initState() { + if (!started.compareAndSet(false, true)) { + return; + } + + log.info("Starting compaction scheduler."); final Optional taskRunnerOptional = taskMaster.getTaskRunner(); if (taskRunnerOptional.isPresent()) { taskRunnerOptional.get().registerListener(taskRunnerListener, Execs.directExecutor()); @@ -231,8 +249,16 @@ private synchronized void initState() } } + /** + * Cleans up scheduler state if required. + */ private synchronized void cleanupState() { + if (!started.compareAndSet(true, false)) { + return; + } + + log.info("Stopping compaction scheduler."); final Optional taskRunnerOptional = taskMaster.getTaskRunner(); if (taskRunnerOptional.isPresent()) { taskRunnerOptional.get().unregisterListener(taskRunnerListener.getListenerId()); @@ -251,21 +277,36 @@ public boolean isEnabled() return compactionConfigSupplier.get().isUseSupervisors(); } + /** + * Periodic task which runs the compaction duty if we are leader and + * useSupervisors is true. Otherwise, the scheduler state is cleaned up. + */ private synchronized void scheduledRun() { - if (isRunning()) { + if (!isLeader.get()) { + cleanupState(); + return; + } + + if (isEnabled()) { + initState(); try { runCompactionDuty(); } catch (Exception e) { log.error(e, "Error processing compaction queue. Continuing schedule."); } - scheduleOnExecutor(this::scheduledRun); + scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS); } else { cleanupState(); + scheduleOnExecutor(this::scheduledRun, SCHEDULE_PERIOD_SECONDS * 4); } } + /** + * Runs the compaction duty and emits stats if {@link #METRIC_EMISSION_PERIOD} + * has elapsed. + */ private synchronized void runCompactionDuty() { final CoordinatorRunStats stats = new CoordinatorRunStats(); @@ -291,7 +332,22 @@ private synchronized void runCompactionDuty() @Override public AutoCompactionSnapshot getCompactionSnapshot(String dataSource) { - return duty.getAutoCompactionSnapshot(dataSource); + if (!activeDatasourceConfigs.containsKey(dataSource)) { + return AutoCompactionSnapshot.builder(dataSource) + .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) + .build(); + } + + final AutoCompactionSnapshot snapshot = duty.getAutoCompactionSnapshot(dataSource); + if (snapshot == null) { + final AutoCompactionSnapshot.ScheduleStatus status = + isEnabled() + ? AutoCompactionSnapshot.ScheduleStatus.AWAITING_FIRST_RUN + : AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED; + return AutoCompactionSnapshot.builder(dataSource).withStatus(status).build(); + } else { + return snapshot; + } } @Override @@ -336,7 +392,7 @@ private DataSourcesSnapshot getDatasourceSnapshot() return segmentManager.getSnapshotOfDataSourcesWithAllUsedSegments(); } - private void scheduleOnExecutor(Runnable runnable) + private void scheduleOnExecutor(Runnable runnable, long delaySeconds) { executor.schedule( () -> { @@ -347,7 +403,7 @@ private void scheduleOnExecutor(Runnable runnable) log.error(t, "Error while executing runnable"); } }, - SCHEDULE_PERIOD_SECONDS, + delaySeconds, TimeUnit.SECONDS ); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java index 9aacf6dea844..aa7a38a1f9ed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/DruidOverlord.java @@ -168,7 +168,7 @@ public void stop() public void start() { taskMaster.becomeFullLeader(); - compactionScheduler.start(); + compactionScheduler.becomeLeader(); scheduledBatchTaskManager.start(); // Announce the node only after all the services have been initialized @@ -181,7 +181,7 @@ public void stop() { serviceAnnouncer.unannounce(node); scheduledBatchTaskManager.stop(); - compactionScheduler.stop(); + compactionScheduler.stopBeingLeader(); taskMaster.downgradeToHalfLeader(); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/CompactionConfigHistoryResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/CompactionConfigHistoryResponse.java new file mode 100644 index 000000000000..dd7b76f9ac41 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/CompactionConfigHistoryResponse.java @@ -0,0 +1,63 @@ +/* + * 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.druid.indexing.overlord.http; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; + +import java.util.List; +import java.util.Objects; + +public class CompactionConfigHistoryResponse +{ + private final List entries; + + public CompactionConfigHistoryResponse( + @JsonProperty("entries") List entries + ) + { + this.entries = entries; + } + + @JsonProperty + public List getEntries() + { + return entries; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompactionConfigHistoryResponse that = (CompactionConfigHistoryResponse) o; + return Objects.equals(entries, that.entries); + } + + @Override + public int hashCode() + { + return Objects.hashCode(entries); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/CompactionConfigsResponse.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/CompactionConfigsResponse.java new file mode 100644 index 000000000000..5899083885b3 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/CompactionConfigsResponse.java @@ -0,0 +1,63 @@ +/* + * 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.druid.indexing.overlord.http; + +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; + +import java.util.List; +import java.util.Objects; + +public class CompactionConfigsResponse +{ + private final List compactionConfigs; + + public CompactionConfigsResponse( + @JsonProperty("compactionConfigs") List compactionConfigs + ) + { + this.compactionConfigs = compactionConfigs; + } + + @JsonProperty + public List getCompactionConfigs() + { + return compactionConfigs; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CompactionConfigsResponse that = (CompactionConfigsResponse) o; + return Objects.equals(compactionConfigs, that.compactionConfigs); + } + + @Override + public int hashCode() + { + return Objects.hashCode(compactionConfigs); + } +} diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java index b29cefef83c7..057de1aef233 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResource.java @@ -19,110 +19,332 @@ package org.apache.druid.indexing.overlord.http; +import com.google.common.collect.Iterators; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.error.DruidException; +import org.apache.druid.audit.AuditInfo; +import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.common.guava.FutureUtils; +import org.apache.druid.error.InternalServerError; import org.apache.druid.error.InvalidInput; import org.apache.druid.error.NotFound; import org.apache.druid.indexing.compact.CompactionScheduler; -import org.apache.druid.server.compaction.CompactionProgressResponse; +import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.overlord.supervisor.CompactionSupervisorManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorResource; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.rpc.HttpResponseException; import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; import org.apache.druid.server.http.ServletResourceUtils; -import org.apache.druid.server.http.security.StateResourceFilter; +import org.apache.druid.server.http.security.ConfigResourceFilter; +import org.apache.druid.server.http.security.DatasourceResourceFilter; +import org.apache.druid.server.security.AuthorizationUtils; +import org.apache.druid.server.security.AuthorizerMapper; +import org.joda.time.Interval; +import javax.annotation.Nullable; +import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; +import javax.ws.rs.DELETE; import javax.ws.rs.GET; import javax.ws.rs.POST; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.QueryParam; +import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.Collection; -import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; /** - * Contains the same logic as {@code CoordinatorCompactionResource} but the APIs - * are served by {@link CompactionScheduler} instead of {@code DruidCoordinator}. + * New compaction APIs exposed by the Overlord. + *

+ * If {@link #isCompactionSupervisorEnabled()} is true, then the APIs are served + * by the Overlord locally, either using the {@link CompactionScheduler} or the + * {@link SupervisorResource}. Otherwise, the APIs are redirected to the + * coordinator. */ @Path("/druid/indexer/v1/compaction") public class OverlordCompactionResource { private final CompactionScheduler scheduler; + private final AuthorizerMapper authorizerMapper; + private final CoordinatorClient coordinatorClient; + private final CoordinatorConfigManager configManager; + private final CompactionSupervisorManager supervisorManager; @Inject public OverlordCompactionResource( - CompactionScheduler scheduler + CompactionScheduler scheduler, + AuthorizerMapper authorizerMapper, + CoordinatorClient coordinatorClient, + CoordinatorConfigManager configManager, + CompactionSupervisorManager supervisorManager ) { this.scheduler = scheduler; + this.configManager = configManager; + this.authorizerMapper = authorizerMapper; + this.coordinatorClient = coordinatorClient; + this.supervisorManager = supervisorManager; } @GET @Path("/isSupervisorEnabled") @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(StateResourceFilter.class) + @ResourceFilters(ConfigResourceFilter.class) public Response isCompactionSupervisorEnabled() { - return Response.ok(scheduler.isEnabled()).build(); + return ServletResourceUtils.buildReadResponse( + scheduler::isEnabled + ); + } + + @POST + @Path("/config/cluster") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(ConfigResourceFilter.class) + public Response updateClusterCompactionConfig( + ClusterCompactionConfig updatePayload, + @Context HttpServletRequest req + ) + { + final AuditInfo auditInfo = AuthorizationUtils.buildAuditInfo(req); + return ServletResourceUtils.buildUpdateResponse( + () -> configManager.updateClusterCompactionConfig(updatePayload, auditInfo) + ); + } + + @GET + @Path("/config/cluster") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(ConfigResourceFilter.class) + public Response getClusterCompactionConfig() + { + return ServletResourceUtils.buildReadResponse( + configManager::getClusterCompactionConfig + ); } @GET - @Path("/progress") + @Path("/status/datasources") @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(StateResourceFilter.class) - public Response getCompactionProgress( - @QueryParam("dataSource") String dataSource + public Response getAllCompactionSnapshots( + @Context HttpServletRequest request ) { - if (!scheduler.isEnabled()) { - return buildErrorResponseIfSchedulerDisabled(); + if (scheduler.isEnabled()) { + return ServletResourceUtils.buildReadResponse(() -> { + final List allSnapshots = + List.copyOf(scheduler.getAllCompactionSnapshots().values()); + return new CompactionStatusResponse( + AuthorizationUtils.filterByAuthorizedDatasources( + request, + allSnapshots, + AutoCompactionSnapshot::getDataSource, + authorizerMapper + ) + ); + }); + } else { + return buildResponse(coordinatorClient.getCompactionSnapshots(null)); } + } - if (dataSource == null || dataSource.isEmpty()) { - return ServletResourceUtils.buildErrorResponseFrom(InvalidInput.exception("No DataSource specified")); + @GET + @Path("/status/datasources/{dataSource}") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getDatasourceCompactionSnapshot( + @PathParam("dataSource") String dataSource + ) + { + if (isEmpty(dataSource)) { + return invalidInputResponse("No DataSource specified"); } - final AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); - if (snapshot == null) { - return ServletResourceUtils.buildErrorResponseFrom(NotFound.exception("Unknown DataSource")); + if (scheduler.isEnabled()) { + AutoCompactionSnapshot snapshot = scheduler.getCompactionSnapshot(dataSource); + if (snapshot == null) { + return ServletResourceUtils.buildErrorResponseFrom(NotFound.exception("Unknown DataSource")); + } else { + return Response.ok(snapshot).build(); + } } else { - return Response.ok(new CompactionProgressResponse(snapshot.getBytesAwaitingCompaction())) - .build(); + return buildResponse( + Futures.transform( + coordinatorClient.getCompactionSnapshots(dataSource), + statusResponse -> Iterators.getOnlyElement(statusResponse.getLatestStatus().iterator()), + MoreExecutors.directExecutor() + ) + ); } } @GET - @Path("/status") + @Path("/config/datasources") @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(StateResourceFilter.class) - public Response getCompactionSnapshots( - @QueryParam("dataSource") String dataSource + public Response getAllCompactionConfigs( + @Context HttpServletRequest request ) { - if (!scheduler.isEnabled()) { - return buildErrorResponseIfSchedulerDisabled(); + if (scheduler.isEnabled()) { + return ServletResourceUtils.buildReadResponse(() -> { + final List configs = AuthorizationUtils.filterByAuthorizedDatasources( + request, + supervisorManager.getAllCompactionSupervisors(), + supervisor -> supervisor.getSpec().getDataSource(), + authorizerMapper + ); + return new CompactionConfigsResponse( + configs.stream() + .map(CompactionSupervisorSpec::getSpec) + .collect(Collectors.toList()) + ); + }); + } else { + return ServletResourceUtils.buildReadResponse(() -> { + final List configs = AuthorizationUtils.filterByAuthorizedDatasources( + request, + configManager.getCurrentCompactionConfig().getCompactionConfigs(), + DataSourceCompactionConfig::getDataSource, + authorizerMapper + ); + return new CompactionConfigsResponse(configs); + }); } + } - final Collection snapshots; - if (dataSource == null || dataSource.isEmpty()) { - snapshots = scheduler.getAllCompactionSnapshots().values(); + @POST + @Path("/config/datasources/{dataSource}") + @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response updateDatasourceCompactionConfig( + @PathParam("dataSource") String dataSource, + DataSourceCompactionConfig newConfig, + @Context HttpServletRequest request + ) + { + if (isEmpty(dataSource)) { + return invalidInputResponse("No DataSource specified"); + } else if (!dataSource.equals(newConfig.getDataSource())) { + return invalidInputResponse( + "DataSource in spec[%s] does not match DataSource in path[%s]", + newConfig.getDataSource(), dataSource + ); + } + + if (scheduler.isEnabled()) { + final CompactionSupervisorSpec spec = new CompactionSupervisorSpec(newConfig, false, scheduler); + return ServletResourceUtils.buildUpdateResponse( + () -> supervisorManager.updateCompactionSupervisor(spec, request) + ); } else { - AutoCompactionSnapshot autoCompactionSnapshot = scheduler.getCompactionSnapshot(dataSource); - if (autoCompactionSnapshot == null) { - return ServletResourceUtils.buildErrorResponseFrom(NotFound.exception("Unknown DataSource")); - } - snapshots = Collections.singleton(autoCompactionSnapshot); + final AuditInfo auditInfo = AuthorizationUtils.buildAuditInfo(request); + return ServletResourceUtils.buildUpdateResponse( + () -> configManager.updateDatasourceCompactionConfig(newConfig, auditInfo) + ); + } + } + + @GET + @Path("/config/datasources/{dataSource}") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getDatasourceCompactionConfig( + @PathParam("dataSource") String dataSource + ) + { + if (isEmpty(dataSource)) { + return invalidInputResponse("No DataSource specified"); + } + + if (scheduler.isEnabled()) { + return ServletResourceUtils.buildReadResponse( + () -> supervisorManager.getCompactionSupervisor(dataSource).getSpec() + ); + } else { + return ServletResourceUtils.buildReadResponse( + () -> configManager.getDatasourceCompactionConfig(dataSource) + ); + } + } + + @DELETE + @Path("/config/datasources/{dataSource}") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response deleteDatasourceCompactionConfig( + @PathParam("dataSource") String dataSource, + @Context HttpServletRequest req + ) + { + if (isEmpty(dataSource)) { + return invalidInputResponse("No DataSource specified"); + } + + if (scheduler.isEnabled()) { + return ServletResourceUtils.buildUpdateResponse( + () -> supervisorManager.deleteCompactionSupervisor(dataSource) + ); + } else { + final AuditInfo auditInfo = AuthorizationUtils.buildAuditInfo(req); + return ServletResourceUtils.buildUpdateResponse( + () -> configManager.deleteDatasourceCompactionConfig(dataSource, auditInfo) + ); + } + } + + @GET + @Path("/config/datasources/{dataSource}/history") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getDatasourceCompactionConfigHistory( + @PathParam("dataSource") String dataSource, + @QueryParam("interval") String interval, + @QueryParam("count") Integer count + ) + { + if (isEmpty(dataSource)) { + return invalidInputResponse("No DataSource specified"); + } + + if (scheduler.isEnabled()) { + return ServletResourceUtils.buildReadResponse( + () -> new CompactionConfigHistoryResponse( + filterByCountAndInterval( + supervisorManager.getCompactionSupervisorHistory(dataSource), + interval, + count + ) + ) + ); + } else { + return ServletResourceUtils.buildReadResponse( + () -> new CompactionConfigHistoryResponse( + configManager.getCompactionConfigHistory(dataSource, interval, count) + ) + ); } - return Response.ok(new CompactionStatusResponse(snapshots)).build(); } @POST @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) - @ResourceFilters(StateResourceFilter.class) + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(ConfigResourceFilter.class) public Response simulateRunWithConfigUpdate( ClusterCompactionConfig updatePayload ) @@ -132,14 +354,54 @@ public Response simulateRunWithConfigUpdate( ).build(); } - private Response buildErrorResponseIfSchedulerDisabled() + private static boolean isEmpty(String dataSource) { - final String msg = "Compaction Supervisors are disabled on the Overlord." - + " Use Coordinator APIs to fetch compaction status."; - return ServletResourceUtils.buildErrorResponseFrom( - DruidException.forPersona(DruidException.Persona.USER) - .ofCategory(DruidException.Category.UNSUPPORTED) - .build(msg) - ); + return dataSource == null || dataSource.isEmpty(); + } + + private static Response invalidInputResponse(String message, Object... args) + { + return ServletResourceUtils.buildErrorResponseFrom(InvalidInput.exception(message, args)); + } + + private static Response buildResponse(ListenableFuture future) + { + try { + return Response.ok(FutureUtils.getUnchecked(future, true)).build(); + } + catch (Exception e) { + if (e.getCause() instanceof HttpResponseException) { + final HttpResponseException cause = (HttpResponseException) e.getCause(); + return Response.status(cause.getResponse().getStatus().getCode()) + .entity(cause.getResponse().getContent()) + .build(); + } else { + return ServletResourceUtils.buildErrorResponseFrom( + InternalServerError.exception(e.getMessage()) + ); + } + } + } + + /** + * Filters the given list of audit entries by both interval and count, if + * specified. + */ + private static List filterByCountAndInterval( + List entries, + @Nullable String serializedInterval, + @Nullable Integer count + ) + { + final Interval interval = serializedInterval == null || serializedInterval.isEmpty() + ? null : Intervals.of(serializedInterval); + return entries.stream() + .filter( + entry -> interval == null + || entry.getAuditTime() == null + || interval.contains(entry.getAuditTime()) + ) + .limit(count == null ? Integer.MAX_VALUE : count) + .collect(Collectors.toList()); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/CompactionSupervisorManager.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/CompactionSupervisorManager.java new file mode 100644 index 000000000000..123eee62b449 --- /dev/null +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/supervisor/CompactionSupervisorManager.java @@ -0,0 +1,213 @@ +/* + * 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.druid.indexing.overlord.supervisor; + +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.inject.Inject; +import org.apache.druid.audit.AuditEntry; +import org.apache.druid.audit.AuditManager; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.NotFound; +import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; +import org.apache.druid.server.security.AuthorizationUtils; +import org.joda.time.DateTime; + +import javax.annotation.Nullable; +import javax.servlet.http.HttpServletRequest; +import java.util.ArrayList; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Wrapper over {@link SupervisorManager} used by + * {@link org.apache.druid.indexing.overlord.http.OverlordCompactionResource} + * to read and write compaction supervisor specs. + */ +public class CompactionSupervisorManager +{ + private final TaskMaster taskMaster; + private final AuditManager auditManager; + + @Inject + public CompactionSupervisorManager( + TaskMaster taskMaster, + AuditManager auditManager + ) + { + this.taskMaster = taskMaster; + this.auditManager = auditManager; + } + + /** + * Creates or updates a compaction supervisor spec. + * + * @return true if the supervisor was updated successfully or if the supervisor + * is already in the desired state. + */ + public boolean updateCompactionSupervisor( + CompactionSupervisorSpec spec, + HttpServletRequest request + ) + { + return performIfLeader(manager -> { + // Check if the spec needs to be updated + if (manager.shouldUpdateSupervisor(spec) && manager.createOrUpdateAndStartSupervisor(spec)) { + final String auditPayload + = StringUtils.format("Update supervisor[%s] for datasource[%s]", spec.getId(), spec.getDataSources()); + auditManager.doAudit( + AuditEntry.builder() + .key(spec.getId()) + .type("supervisor") + .auditInfo(AuthorizationUtils.buildAuditInfo(request)) + .request(AuthorizationUtils.buildRequestInfo("overlord", request)) + .payload(auditPayload) + .build() + ); + } + + return true; + }); + } + + /** + * Gets the compaction supervisor for the given datasource, if one exists. + * + * @throws DruidException if a compaction supervisor does not exist for this + * datasource or if the supervisor is of an unexpected type. + */ + public CompactionSupervisorSpec getCompactionSupervisor(String dataSource) + { + final String supervisorId = CompactionSupervisorSpec.getSupervisorIdForDatasource(dataSource); + + return performIfLeader(manager -> { + final Optional specOptional = manager.getSupervisorSpec(supervisorId); + + if (specOptional.isPresent()) { + SupervisorSpec spec = specOptional.get(); + if (spec instanceof CompactionSupervisorSpec) { + return (CompactionSupervisorSpec) spec; + } else { + throw DruidException.defensive( + "Supervisor for ID[%s] is of unexpected type[%s]", + supervisorId, spec.getClass().getSimpleName() + ); + } + } else { + throw NotFound.exception("Compaction supervisor for datasource[%s] does not exist", dataSource); + } + }); + } + + /** + * Deletes the compaction supervisor for the given datasource, if one exists. + * + * @return true if the supervisor was successfully deleted. + * @throws DruidException if a compaction supervisor does not exist for this + * datasource. + */ + public boolean deleteCompactionSupervisor(String dataSource) + { + final String supervisorId = CompactionSupervisorSpec.getSupervisorIdForDatasource(dataSource); + + return performIfLeader(manager -> { + if (manager.stopAndRemoveSupervisor(supervisorId)) { + return true; + } else { + throw NotFound.exception("Compaction supervisor for datasource[%s] does not exist", dataSource); + } + }); + } + + /** + * Returns all compaction supervisors. + */ + public List getAllCompactionSupervisors() + { + return performIfLeader(manager -> { + final List compactionSpecs = new ArrayList<>(); + for (String supervisorId : Set.copyOf(manager.getSupervisorIds())) { + Optional supervisorSpecOptional = manager.getSupervisorSpec(supervisorId); + if (!supervisorSpecOptional.isPresent()) { + continue; + } + + final SupervisorSpec supervisorSpec = supervisorSpecOptional.get(); + if (supervisorSpec instanceof CompactionSupervisorSpec) { + compactionSpecs.add(((CompactionSupervisorSpec) supervisorSpec)); + } + } + + return compactionSpecs; + }); + } + + /** + * Gets the change history for the compaction supervisor of the given datasource. + * + * @return Change history for the compaction supervisor of the given datasource + * in descending order by update time or an empty list if no history exists + * for the compaction supervisor of this datasource. + */ + public List getCompactionSupervisorHistory(String dataSource) + { + final String supervisorId = CompactionSupervisorSpec.getSupervisorIdForDatasource(dataSource); + + return performIfLeader( + manager -> manager + .getSupervisorHistoryForId(supervisorId) + .stream() + .filter(versionedSpec -> versionedSpec.getSpec() instanceof CompactionSupervisorSpec) + .map( + versionedSupervisorSpec -> new DataSourceCompactionConfigAuditEntry( + null, + ((CompactionSupervisorSpec) versionedSupervisorSpec.getSpec()).getSpec(), + null, + nullSafeDate(versionedSupervisorSpec.getVersion()) + ) + ) + .collect(Collectors.toList()) + ); + } + + private T performIfLeader(Function managerFunction) + { + Optional supervisorManager = taskMaster.getSupervisorManager(); + if (supervisorManager.isPresent()) { + return managerFunction.apply(supervisorManager.get()); + } else { + // Encourage client to try again soon, when we'll likely have a redirect set up + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.SERVICE_UNAVAILABLE) + .build("Overlord is not leader"); + } + } + + @Nullable + private static DateTime nullSafeDate(String date) + { + return date == null || date.isEmpty() ? null : DateTimes.of(date); + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java index 0e128e4cce5a..b204a69ed384 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/compact/OverlordCompactionSchedulerTest.java @@ -72,6 +72,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicReference; public class OverlordCompactionSchedulerTest { @@ -90,7 +91,7 @@ public class OverlordCompactionSchedulerTest ); } - private DruidCompactionConfig compactionConfig; + private AtomicReference compactionConfig; private CoordinatorOverlordServiceConfig coordinatorOverlordServiceConfig; private TaskMaster taskMaster; @@ -127,13 +128,13 @@ public void setUp() serviceEmitter = new StubServiceEmitter(); segmentsMetadataManager = new TestSegmentsMetadataManager(); - compactionConfig = DruidCompactionConfig.empty(); + compactionConfig = new AtomicReference<>(new ClusterCompactionConfig(null, null, null, true, null)); coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); - initScheduler(true); + initScheduler(); } - private void initScheduler(boolean isSupervisorEnabled) + private void initScheduler() { TaskLockbox taskLockbox = new TaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator()); WorkerBehaviorConfig defaultWorkerConfig @@ -142,9 +143,7 @@ private void initScheduler(boolean isSupervisorEnabled) taskMaster, new TaskQueryTool(taskStorage, taskLockbox, taskMaster, null, () -> defaultWorkerConfig), segmentsMetadataManager, - () -> compactionConfig.withClusterConfig( - new ClusterCompactionConfig(null, null, null, isSupervisorEnabled, null) - ), + () -> DruidCompactionConfig.empty().withClusterConfig(compactionConfig.get()), new CompactionStatusTracker(OBJECT_MAPPER), coordinatorOverlordServiceConfig, (nameFormat, numThreads) -> new WrappingScheduledExecutorService("test", executor, false), @@ -154,69 +153,120 @@ private void initScheduler(boolean isSupervisorEnabled) } @Test - public void testStartStopWhenSchedulerIsEnabled() + public void testBecomeLeader_triggersStart_ifEnabled() { + Assert.assertTrue(scheduler.isEnabled()); + Assert.assertFalse(scheduler.isRunning()); + Assert.assertFalse(executor.hasPendingTasks()); + + scheduler.becomeLeader(); + runScheduledJob(); - scheduler.start(); Assert.assertTrue(scheduler.isRunning()); - Assert.assertTrue(executor.hasPendingTasks()); - scheduler.stop(); + } + + @Test + public void testBecomeLeader_doesNotTriggerStart_ifDisabled() + { + disableScheduler(); + Assert.assertFalse(scheduler.isEnabled()); + + Assert.assertFalse(scheduler.isRunning()); + + scheduler.becomeLeader(); + runScheduledJob(); + Assert.assertFalse(scheduler.isRunning()); - Assert.assertTrue(executor.hasPendingTasks()); + } + + @Test + public void testStopBeingLeader_triggersStop() + { + Assert.assertFalse(scheduler.isRunning()); + + scheduler.becomeLeader(); + runScheduledJob(); + Assert.assertTrue(scheduler.isRunning()); - scheduler.start(); + scheduler.stopBeingLeader(); Assert.assertTrue(scheduler.isRunning()); - scheduler.stop(); + + runScheduledJob(); Assert.assertFalse(scheduler.isRunning()); } @Test - public void testStartStopWhenScheduledIsDisabled() + public void testDisablingScheduler_triggersStop() { - initScheduler(false); + // Start scheduler + scheduler.becomeLeader(); + runScheduledJob(); + Assert.assertTrue(scheduler.isRunning()); + // Disable scheduler to trigger stop + disableScheduler(); + Assert.assertFalse(scheduler.isEnabled()); + Assert.assertTrue(scheduler.isRunning()); + + // Scheduler finally stops in the next schedule cycle + runScheduledJob(); Assert.assertFalse(scheduler.isRunning()); - scheduler.start(); + } + + @Test + public void testEnablingScheduler_triggersStart() + { + disableScheduler(); + + // Becoming leader does not trigger start since scheduler is disabled + scheduler.becomeLeader(); + runScheduledJob(); Assert.assertFalse(scheduler.isRunning()); - Assert.assertFalse(executor.hasPendingTasks()); - scheduler.stop(); + + // Enable the schduler to trigger start + enableScheduler(); Assert.assertFalse(scheduler.isRunning()); - Assert.assertFalse(executor.hasPendingTasks()); + + // Scheduler finally starts in the next schedule cycle + runScheduledJob(); + Assert.assertTrue(scheduler.isRunning()); } @Test - public void testSegmentsAreNotPolledWhenSchedulerIsDisabled() + public void testSegmentsAreNotPolled_ifSupervisorsAreDisabled() { - initScheduler(false); + disableScheduler(); verifySegmentPolling(false); } @Test - public void testSegmentsArePolledWhenRunningInStandaloneMode() + public void testSegmentsArePolled_whenRunningInStandaloneMode() { coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null); - initScheduler(true); + initScheduler(); verifySegmentPolling(true); } @Test - public void testSegmentsAreNotPolledWhenRunningInCoordinatorMode() + public void testSegmentsAreNotPolled_whenRunningInCoordinatorMode() { coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(true, "overlord"); - initScheduler(true); + initScheduler(); verifySegmentPolling(false); } private void verifySegmentPolling(boolean enabled) { - scheduler.start(); + scheduler.becomeLeader(); + runScheduledJob(); Assert.assertEquals(enabled, segmentsMetadataManager.isPollingDatabasePeriodically()); - scheduler.stop(); + scheduler.stopBeingLeader(); + runScheduledJob(); Assert.assertFalse(segmentsMetadataManager.isPollingDatabasePeriodically()); } @@ -247,12 +297,12 @@ public void testMsqCompactionConfigWithOneMaxTasksIsInvalid() } @Test - public void testStartCompactionForDatasource() + public void testStartCompaction() { final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); wikiSegments.forEach(segmentsMetadataManager::addSegment); - scheduler.start(); + scheduler.becomeLeader(); scheduler.startCompaction( TestDataSource.WIKI, InlineSchemaDataSourceCompactionConfig.builder() @@ -288,16 +338,16 @@ public void testStartCompactionForDatasource() serviceEmitter.verifyValue(Stats.Compaction.SUBMITTED_TASKS.getMetricName(), 1L); serviceEmitter.verifyValue(Stats.Compaction.COMPACTED_BYTES.getMetricName(), 100_000_000L); - scheduler.stop(); + scheduler.stopBeingLeader(); } @Test - public void testStopCompactionForDatasource() + public void testStopCompaction() { final List wikiSegments = CreateDataSegments.ofDatasource(TestDataSource.WIKI).eachOfSizeInMb(100); wikiSegments.forEach(segmentsMetadataManager::addSegment); - scheduler.start(); + scheduler.becomeLeader(); scheduler.startCompaction( TestDataSource.WIKI, InlineSchemaDataSourceCompactionConfig.builder() @@ -311,17 +361,22 @@ public void testStopCompactionForDatasource() Mockito.verify(taskQueue, Mockito.never()).add(ArgumentMatchers.any()); - Assert.assertNull(scheduler.getCompactionSnapshot(TestDataSource.WIKI)); + Assert.assertEquals( + AutoCompactionSnapshot.builder(TestDataSource.WIKI) + .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) + .build(), + scheduler.getCompactionSnapshot(TestDataSource.WIKI) + ); Assert.assertTrue(scheduler.getAllCompactionSnapshots().isEmpty()); serviceEmitter.verifyNotEmitted(Stats.Compaction.SUBMITTED_TASKS.getMetricName()); serviceEmitter.verifyNotEmitted(Stats.Compaction.COMPACTED_BYTES.getMetricName()); - scheduler.stop(); + scheduler.stopBeingLeader(); } @Test - public void testRunSimulation() + public void testSimulateRun() { final List wikiSegments = CreateDataSegments .ofDatasource(TestDataSource.WIKI) @@ -331,7 +386,9 @@ public void testRunSimulation() .eachOfSizeInMb(100); wikiSegments.forEach(segmentsMetadataManager::addSegment); - scheduler.start(); + scheduler.becomeLeader(); + runScheduledJob(); + scheduler.startCompaction( TestDataSource.WIKI, InlineSchemaDataSourceCompactionConfig.builder() @@ -370,7 +427,22 @@ public void testRunSimulation() ); Assert.assertTrue(simulateResultWhenDisabled.getCompactionStates().isEmpty()); - scheduler.stop(); + scheduler.stopBeingLeader(); + } + + private void disableScheduler() + { + compactionConfig.set(new ClusterCompactionConfig(null, null, null, false, null)); + } + + private void enableScheduler() + { + compactionConfig.set(new ClusterCompactionConfig(null, null, null, true, null)); + } + + private void runScheduledJob() + { + executor.finishNextPendingTask(); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java index de9cd7ec6c06..50bae6c8d2a6 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordCompactionResourceTest.java @@ -19,215 +19,448 @@ package org.apache.druid.indexing.overlord.http; -import com.google.common.collect.ImmutableMap; -import org.apache.druid.error.DruidException; +import com.google.common.base.Optional; +import com.google.common.util.concurrent.Futures; +import org.apache.druid.audit.AuditManager; +import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.error.ErrorResponse; +import org.apache.druid.indexer.CompactionEngine; import org.apache.druid.indexing.compact.CompactionScheduler; +import org.apache.druid.indexing.compact.CompactionSupervisorSpec; +import org.apache.druid.indexing.overlord.TaskMaster; +import org.apache.druid.indexing.overlord.supervisor.CompactionSupervisorManager; +import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; +import org.apache.druid.indexing.overlord.supervisor.VersionedSupervisorSpec; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.segment.TestDataSource; -import org.apache.druid.server.compaction.CompactionProgressResponse; -import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; +import org.apache.druid.server.coordinator.CompactionConfigValidationResult; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; +import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; +import org.apache.druid.server.security.AllowAllAuthorizer; +import org.apache.druid.server.security.AuthConfig; +import org.apache.druid.server.security.AuthenticationResult; +import org.apache.druid.server.security.AuthorizerMapper; import org.easymock.EasyMock; import org.hamcrest.MatcherAssert; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import javax.servlet.http.HttpServletRequest; import javax.ws.rs.core.Response; -import java.util.Collections; +import java.util.List; import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; public class OverlordCompactionResourceTest { + private final Random random = new Random(1000); + private CompactionScheduler scheduler; + private OverlordCompactionResource compactionResource; + + private CoordinatorClient coordinatorClient; + private CoordinatorConfigManager configManager; + + private HttpServletRequest httpRequest; + private AuthorizerMapper authorizerMapper; + + private TaskMaster taskMaster; + private SupervisorManager supervisorManager; + private final AtomicBoolean useSupervisors = new AtomicBoolean(false); + + private DataSourceCompactionConfig wikiConfig; + + /** + * Mock instance of CompactionScheduler used only for validating compaction configs. + */ + private CompactionScheduler validator; @Before public void setUp() { + useSupervisors.set(true); scheduler = EasyMock.createStrictMock(CompactionScheduler.class); - EasyMock.expect(scheduler.isEnabled()).andReturn(true).anyTimes(); + EasyMock.expect(scheduler.isEnabled()).andAnswer(useSupervisors::get).anyTimes(); + + coordinatorClient = EasyMock.createStrictMock(CoordinatorClient.class); + supervisorManager = EasyMock.createStrictMock(SupervisorManager.class); + configManager = EasyMock.createStrictMock(CoordinatorConfigManager.class); + + httpRequest = EasyMock.createStrictMock(HttpServletRequest.class); + authorizerMapper = EasyMock.createStrictMock(AuthorizerMapper.class); + EasyMock.expect(authorizerMapper.getAuthorizer("druid")).andReturn(new AllowAllAuthorizer()).anyTimes(); + + taskMaster = EasyMock.createStrictMock(TaskMaster.class); + EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.of(supervisorManager)).anyTimes(); + + final AuditManager auditManager = EasyMock.createStrictMock(AuditManager.class); + + compactionResource = new OverlordCompactionResource( + scheduler, + authorizerMapper, + coordinatorClient, + configManager, + new CompactionSupervisorManager(taskMaster, auditManager) + ); + + wikiConfig = InlineSchemaDataSourceCompactionConfig + .builder() + .forDataSource(TestDataSource.WIKI) + .withTaskPriority(random.nextInt(100)) + .withSkipOffsetFromLatest(Period.days(random.nextInt(5))) + .build(); + + validator = EasyMock.createStrictMock(CompactionScheduler.class); + EasyMock.expect(validator.validateCompactionConfig(EasyMock.anyObject())) + .andReturn(CompactionConfigValidationResult.success()) + .anyTimes(); + EasyMock.replay(validator, taskMaster, authorizerMapper); } @After public void tearDown() { - EasyMock.verify(scheduler); + EasyMock.verify( + validator, + scheduler, + taskMaster, + httpRequest, + configManager, + authorizerMapper, + coordinatorClient, + supervisorManager + ); + } + + private void replayAll() + { + EasyMock.replay(scheduler, httpRequest, configManager, coordinatorClient, supervisorManager); } @Test - public void testGetCompactionSnapshotWithEmptyDatasource() + public void test_updateClusterConfig() { - final Map allSnapshots = ImmutableMap.of( - TestDataSource.WIKI, - AutoCompactionSnapshot.builder(TestDataSource.WIKI).build() - ); + EasyMock.expect(configManager.updateClusterCompactionConfig(EasyMock.anyObject(), EasyMock.anyObject())) + .andReturn(true) + .once(); - EasyMock.expect(scheduler.getAllCompactionSnapshots()) - .andReturn(allSnapshots).once(); - EasyMock.replay(scheduler); + setupMockRequestForAudit(); + replayAll(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionSnapshots(""); - Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals( - new CompactionStatusResponse(allSnapshots.values()), - response.getEntity() + Response response = compactionResource.updateClusterCompactionConfig( + new ClusterCompactionConfig(0.5, 10, null, true, CompactionEngine.MSQ), + httpRequest ); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(Map.of("success", true), response.getEntity()); } @Test - public void testGetCompactionSnapshotWithNullDatasource() + public void test_getClusterConfig() { - final Map allSnapshots = ImmutableMap.of( - TestDataSource.WIKI, - AutoCompactionSnapshot.builder(TestDataSource.WIKI).build() - ); + final ClusterCompactionConfig clusterConfig = + new ClusterCompactionConfig(0.4, 100, null, true, CompactionEngine.MSQ); + EasyMock.expect(configManager.getClusterCompactionConfig()) + .andReturn(clusterConfig) + .once(); + replayAll(); + + final Response response = compactionResource.getClusterCompactionConfig(); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(clusterConfig, response.getEntity()); + } - EasyMock.expect(scheduler.getAllCompactionSnapshots()) - .andReturn(allSnapshots).once(); - EasyMock.replay(scheduler); + @Test + public void test_getDatasourceCompactionSnapshot_returnsInvalidInput_ifDatasourceIsNullOrEmpty() + { + replayAll(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionSnapshots(null); - Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals( - new CompactionStatusResponse(allSnapshots.values()), - response.getEntity() + verifyInvalidInputResponse( + compactionResource.getDatasourceCompactionSnapshot(""), + "No DataSource specified" + ); + verifyInvalidInputResponse( + compactionResource.getDatasourceCompactionSnapshot(null), + "No DataSource specified" ); } @Test - public void testGetCompactionSnapshotWithValidDatasource() + public void test_getDatasourceCompactionSnapshot() { final AutoCompactionSnapshot snapshot = AutoCompactionSnapshot.builder(TestDataSource.WIKI).build(); EasyMock.expect(scheduler.getCompactionSnapshot(TestDataSource.WIKI)) .andReturn(snapshot).once(); - EasyMock.replay(scheduler); + replayAll(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionSnapshots(TestDataSource.WIKI); + final Response response = compactionResource.getDatasourceCompactionSnapshot(TestDataSource.WIKI); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals( - new CompactionStatusResponse(Collections.singleton(snapshot)), - response.getEntity() - ); + Assert.assertEquals(snapshot, response.getEntity()); } @Test - public void testGetCompactionSnapshotWithInvalidDatasource() + public void test_getDatasourceCompactionSnapshot_returnsNotFound_withInvalidDatasource() { EasyMock.expect(scheduler.getCompactionSnapshot(TestDataSource.KOALA)) .andReturn(null).once(); - EasyMock.replay(scheduler); + replayAll(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionSnapshots(TestDataSource.KOALA); + final Response response = compactionResource.getDatasourceCompactionSnapshot(TestDataSource.KOALA); Assert.assertEquals(404, response.getStatus()); } @Test - public void testGetProgressForValidDatasource() + public void test_getDatasourceCompactionSnapshot_redirectsToCoordinator_ifSchedulerIsDisabled() { - final AutoCompactionSnapshot.Builder snapshotBuilder - = AutoCompactionSnapshot.builder(TestDataSource.WIKI); - snapshotBuilder.incrementWaitingStats(CompactionStatistics.create(100L, 10L, 1L)); - final AutoCompactionSnapshot snapshot = snapshotBuilder.build(); + useSupervisors.set(false); - EasyMock.expect(scheduler.getCompactionSnapshot(TestDataSource.WIKI)) - .andReturn(snapshot).once(); - EasyMock.replay(scheduler); + final AutoCompactionSnapshot snapshot = + AutoCompactionSnapshot.builder(TestDataSource.WIKI).build(); + EasyMock.expect(coordinatorClient.getCompactionSnapshots(TestDataSource.WIKI)) + .andReturn(Futures.immediateFuture(new CompactionStatusResponse(List.of(snapshot)))); + replayAll(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionProgress(TestDataSource.WIKI); + final Response response = compactionResource.getDatasourceCompactionSnapshot(TestDataSource.WIKI); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(new CompactionProgressResponse(100L), response.getEntity()); + Assert.assertEquals(snapshot, response.getEntity()); } @Test - public void testGetProgressForNullDatasourceReturnsBadRequest() + public void test_getAllCompactionSnapshots() { - EasyMock.replay(scheduler); + final AutoCompactionSnapshot snapshot = + AutoCompactionSnapshot.builder(TestDataSource.WIKI).build(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionProgress(null); - Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); + setupMockRequestForUser("druid"); + EasyMock.expect(httpRequest.getMethod()).andReturn("POST").once(); - final Object responseEntity = response.getEntity(); - Assert.assertTrue(responseEntity instanceof ErrorResponse); + EasyMock.expect(scheduler.getAllCompactionSnapshots()) + .andReturn(Map.of(TestDataSource.WIKI, snapshot)).anyTimes(); + replayAll(); - MatcherAssert.assertThat( - ((ErrorResponse) responseEntity).getUnderlyingException(), - DruidExceptionMatcher.invalidInput().expectMessageIs("No DataSource specified") - ); + final Response response = compactionResource.getAllCompactionSnapshots(httpRequest); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(new CompactionStatusResponse(List.of(snapshot)), response.getEntity()); } @Test - public void testGetProgressForInvalidDatasourceReturnsNotFound() + public void test_getDatasourceCompactionConfig() { - EasyMock.expect(scheduler.getCompactionSnapshot(TestDataSource.KOALA)) - .andReturn(null).once(); - EasyMock.replay(scheduler); + final String supervisorId = CompactionSupervisorSpec.getSupervisorIdForDatasource(TestDataSource.WIKI); + EasyMock.expect(supervisorManager.getSupervisorSpec(supervisorId)) + .andReturn(Optional.of(new CompactionSupervisorSpec(wikiConfig, false, validator))) + .anyTimes(); + replayAll(); - final Response response = new OverlordCompactionResource(scheduler) - .getCompactionProgress(TestDataSource.KOALA); - Assert.assertEquals(Response.Status.NOT_FOUND.getStatusCode(), response.getStatus()); + final Response response = compactionResource.getDatasourceCompactionConfig(TestDataSource.WIKI); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(wikiConfig, response.getEntity()); + } - final Object responseEntity = response.getEntity(); - Assert.assertTrue(responseEntity instanceof ErrorResponse); + @Test + public void test_getDatasourceCompactionConfig_returnsInvalidInput_ifDatasourceIsNullOrEmpty() + { + replayAll(); - MatcherAssert.assertThat( - ((ErrorResponse) responseEntity).getUnderlyingException(), - DruidExceptionMatcher.notFound().expectMessageIs("Unknown DataSource") + verifyInvalidInputResponse( + compactionResource.getDatasourceCompactionConfig(""), + "No DataSource specified" + ); + verifyInvalidInputResponse( + compactionResource.getDatasourceCompactionConfig(null), + "No DataSource specified" ); } @Test - public void testGetProgressReturnsUnsupportedWhenSupervisorDisabled() + public void test_updateDatasourceCompactionConfig() { - scheduler = EasyMock.createStrictMock(CompactionScheduler.class); - EasyMock.expect(scheduler.isEnabled()).andReturn(false).once(); - EasyMock.replay(scheduler); + setupMockRequestForAudit(); + EasyMock.expect(httpRequest.getMethod()).andReturn("POST").once(); + EasyMock.expect(httpRequest.getRequestURI()).andReturn("supes").once(); + EasyMock.expect(httpRequest.getQueryString()).andReturn("a=b").once(); + + final CompactionSupervisorSpec supervisorSpec = + new CompactionSupervisorSpec(wikiConfig, false, validator); + + EasyMock.expect(supervisorManager.shouldUpdateSupervisor(supervisorSpec)) + .andReturn(true).once(); + EasyMock.expect(supervisorManager.createOrUpdateAndStartSupervisor(supervisorSpec)) + .andReturn(true).once(); + EasyMock.expect(scheduler.validateCompactionConfig(wikiConfig)) + .andReturn(CompactionConfigValidationResult.success()).once(); + replayAll(); + + final Response response = compactionResource + .updateDatasourceCompactionConfig(TestDataSource.WIKI, wikiConfig, httpRequest); + Assert.assertEquals(200, response.getStatus()); + } - verifyResponseWhenSupervisorDisabled( - new OverlordCompactionResource(scheduler) - .getCompactionProgress(TestDataSource.WIKI) + @Test + public void test_updateDatasourceCompactionConfig_returnsInvalidInput_ifDatasourceIsNullOrEmpty() + { + replayAll(); + + verifyInvalidInputResponse( + compactionResource.updateDatasourceCompactionConfig("", wikiConfig, httpRequest), + "No DataSource specified" + ); + verifyInvalidInputResponse( + compactionResource.updateDatasourceCompactionConfig(null, wikiConfig, httpRequest), + "No DataSource specified" ); } @Test - public void testGetSnapshotReturnsUnsupportedWhenSupervisorDisabled() + public void test_deleteDatasourceCompactionConfig() { - scheduler = EasyMock.createStrictMock(CompactionScheduler.class); - EasyMock.expect(scheduler.isEnabled()).andReturn(false).once(); - EasyMock.replay(scheduler); + final String supervisorId = CompactionSupervisorSpec.getSupervisorIdForDatasource(TestDataSource.WIKI); + EasyMock.expect(supervisorManager.stopAndRemoveSupervisor(supervisorId)) + .andReturn(true) + .once(); + replayAll(); + + final Response response = compactionResource + .deleteDatasourceCompactionConfig(TestDataSource.WIKI, httpRequest); + Assert.assertEquals(200, response.getStatus()); + } + + @Test + public void test_deleteDatasourceCompactionConfig_returnsInvalidInput_ifDatasourceIsNullOrEmpty() + { + replayAll(); - verifyResponseWhenSupervisorDisabled( - new OverlordCompactionResource(scheduler) - .getCompactionSnapshots(TestDataSource.WIKI) + verifyInvalidInputResponse( + compactionResource.deleteDatasourceCompactionConfig("", httpRequest), + "No DataSource specified" + ); + verifyInvalidInputResponse( + compactionResource.deleteDatasourceCompactionConfig(null, httpRequest), + "No DataSource specified" ); } - private void verifyResponseWhenSupervisorDisabled(Response response) + @Test + public void test_getAllCompactionConfigs() { - Assert.assertEquals(501, response.getStatus()); + final String supervisorId = CompactionSupervisorSpec + .getSupervisorIdForDatasource(TestDataSource.WIKI); + + setupMockRequestForUser("druid"); + EasyMock.expect(httpRequest.getMethod()).andReturn("POST").once(); + + EasyMock.expect(supervisorManager.getSupervisorIds()) + .andReturn(Set.of(supervisorId)) + .once(); + EasyMock.expect(supervisorManager.getSupervisorSpec(supervisorId)) + .andReturn(Optional.of(new CompactionSupervisorSpec(wikiConfig, false, validator))) + .once(); + replayAll(); + + final Response response = compactionResource.getAllCompactionConfigs(httpRequest); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(new CompactionConfigsResponse(List.of(wikiConfig)), response.getEntity()); + } + + @Test + public void test_getDatasourceCompactionConfigHistory_withFilters() + { + final String supervisorId = CompactionSupervisorSpec.getSupervisorIdForDatasource(TestDataSource.WIKI); + + final CompactionSupervisorSpec spec = new CompactionSupervisorSpec(wikiConfig, false, validator); + final List specVersions = List.of( + new VersionedSupervisorSpec(spec, "2025-03"), + new VersionedSupervisorSpec(spec, "2025-02"), + new VersionedSupervisorSpec(spec, "2025-01") + ); + + EasyMock.expect(supervisorManager.getSupervisorHistoryForId(supervisorId)) + .andReturn(specVersions) + .anyTimes(); + replayAll(); - final Object responseEntity = response.getEntity(); - Assert.assertTrue(responseEntity instanceof ErrorResponse); + final List history = List.of( + new DataSourceCompactionConfigAuditEntry(null, wikiConfig, null, DateTimes.of("2025-03")), + new DataSourceCompactionConfigAuditEntry(null, wikiConfig, null, DateTimes.of("2025-02")), + new DataSourceCompactionConfigAuditEntry(null, wikiConfig, null, DateTimes.of("2025-01")) + ); + + Response response = compactionResource + .getDatasourceCompactionConfigHistory(TestDataSource.WIKI, null, null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(toHistoryResponse(history), response.getEntity()); + + // Filter by count + response = compactionResource + .getDatasourceCompactionConfigHistory(TestDataSource.WIKI, null, 2); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(toHistoryResponse(history.subList(0, 2)), response.getEntity()); + + // Filter by interval + response = compactionResource + .getDatasourceCompactionConfigHistory(TestDataSource.WIKI, "2025-01/P40D", null); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(toHistoryResponse(history.subList(1, 3)), response.getEntity()); + + // Filter by interval and count + response = compactionResource + .getDatasourceCompactionConfigHistory(TestDataSource.WIKI, "2025-01/P40D", 1); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(toHistoryResponse(history.subList(1, 2)), response.getEntity()); + } + private static CompactionConfigHistoryResponse toHistoryResponse( + List entries + ) + { + return new CompactionConfigHistoryResponse(entries); + } + + private void verifyInvalidInputResponse(Response response, String message) + { + Assert.assertEquals(400, response.getStatus()); + Assert.assertTrue(response.getEntity() instanceof ErrorResponse); MatcherAssert.assertThat( - ((ErrorResponse) responseEntity).getUnderlyingException(), - new DruidExceptionMatcher( - DruidException.Persona.USER, - DruidException.Category.UNSUPPORTED, - "general" - ).expectMessageIs( - "Compaction Supervisors are disabled on the Overlord." - + " Use Coordinator APIs to fetch compaction status." - ) + ((ErrorResponse) response.getEntity()).getUnderlyingException(), + DruidExceptionMatcher.invalidInput().expectMessageIs(message) ); } + + private void setupMockRequestForUser(String user) + { + EasyMock.expect(httpRequest.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce(); + EasyMock.expect(httpRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)).andReturn(null).atLeastOnce(); + EasyMock.expect(httpRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(new AuthenticationResult(user, "druid", null, null)) + .atLeastOnce(); + httpRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + } + + private void setupMockRequestForAudit() + { + EasyMock.expect(httpRequest.getHeader(AuditManager.X_DRUID_AUTHOR)).andReturn("author").once(); + EasyMock.expect(httpRequest.getHeader(AuditManager.X_DRUID_COMMENT)).andReturn("comment").once(); + + EasyMock.expect(httpRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(new AuthenticationResult("druid", "druid", null, null)) + .atLeastOnce(); + + EasyMock.expect(httpRequest.getRemoteAddr()).andReturn("127.0.0.1").atLeastOnce(); + // EasyMock.expect(httpRequest.getMethod()).andReturn("POST").once(); + + // EasyMock.expect(httpRequest.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).atLeastOnce(); + } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java index da0b2f3516d0..9649a2cce707 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CompactionResourceTestClient.java @@ -22,12 +22,16 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; +import org.apache.druid.indexing.overlord.http.CompactionConfigsResponse; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.server.compaction.CompactionSimulateResult; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.testing.IntegrationTestingConfig; @@ -41,9 +45,12 @@ public class CompactionResourceTestClient { + private static final Logger log = new Logger(CompactionResourceTestClient.class); + private final ObjectMapper jsonMapper; private final HttpClient httpClient; private final String coordinator; + private final String overlord; private final StatusResponseHandler responseHandler; @Inject @@ -56,6 +63,7 @@ public class CompactionResourceTestClient this.jsonMapper = jsonMapper; this.httpClient = httpClient; this.coordinator = config.getCoordinatorUrl(); + this.overlord = config.getOverlordUrl(); this.responseHandler = StatusResponseHandler.getInstance(); } @@ -67,14 +75,24 @@ private String getCoordinatorURL() ); } + private String getOverlordURL() + { + return StringUtils.format("%s/druid/indexer/v1", overlord); + } + public void submitCompactionConfig(final DataSourceCompactionConfig dataSourceCompactionConfig) throws Exception { - String url = StringUtils.format("%sconfig/compaction", getCoordinatorURL()); + final String dataSource = dataSourceCompactionConfig.getDataSource(); + String url = StringUtils.format( + "%s/compaction/config/datasources/%s", + getOverlordURL(), StringUtils.urlEncode(dataSource) + ); StatusResponseHolder response = httpClient.go( new Request(HttpMethod.POST, new URL(url)).setContent( "application/json", jsonMapper.writeValueAsBytes(dataSourceCompactionConfig) - ), responseHandler + ), + responseHandler ).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -84,11 +102,18 @@ public void submitCompactionConfig(final DataSourceCompactionConfig dataSourceCo response.getContent() ); } + log.info( + "Submitted compaction config for datasource[%s] with response[%s]", + dataSource, response.getContent() + ); } public void deleteDataSourceCompactionConfig(final String dataSource) throws Exception { - String url = StringUtils.format("%sconfig/compaction/%s", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); + String url = StringUtils.format( + "%s/compaction/config/datasources/%s", + getOverlordURL(), StringUtils.urlEncode(dataSource) + ); StatusResponseHolder response = httpClient.go(new Request(HttpMethod.DELETE, new URL(url)), responseHandler).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -100,7 +125,12 @@ public void deleteDataSourceCompactionConfig(final String dataSource) throws Exc } } - public DruidCompactionConfig getCompactionConfig() throws Exception + /** + * For all purposes, use the new APIs {@link #getClusterConfig()} or + * {@link #getAllCompactionConfigs()}. + */ + @Deprecated + public DruidCompactionConfig getCoordinatorCompactionConfig() throws Exception { String url = StringUtils.format("%sconfig/compaction", getCoordinatorURL()); StatusResponseHolder response = httpClient.go( @@ -116,9 +146,32 @@ public DruidCompactionConfig getCompactionConfig() throws Exception return jsonMapper.readValue(response.getContent(), new TypeReference<>() {}); } + public List getAllCompactionConfigs() throws Exception + { + String url = StringUtils.format("%s/compaction/config/datasources", getOverlordURL()); + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.GET, new URL(url)), responseHandler + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while getting compaction config status[%s] content[%s]", + response.getStatus(), + response.getContent() + ); + } + final CompactionConfigsResponse payload = jsonMapper.readValue( + response.getContent(), + new TypeReference<>() {} + ); + return payload.getCompactionConfigs(); + } + public DataSourceCompactionConfig getDataSourceCompactionConfig(String dataSource) throws Exception { - String url = StringUtils.format("%sconfig/compaction/%s", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); + String url = StringUtils.format( + "%s/compaction/config/datasources/%s", + getOverlordURL(), StringUtils.urlEncode(dataSource) + ); StatusResponseHolder response = httpClient.go( new Request(HttpMethod.GET, new URL(url)), responseHandler ).get(); @@ -134,6 +187,22 @@ public DataSourceCompactionConfig getDataSourceCompactionConfig(String dataSourc public void forceTriggerAutoCompaction() throws Exception { + // Perform a dummy update of task slots to force the coordinator to refresh its compaction config + final ClusterCompactionConfig clusterConfig = getClusterConfig(); + updateCompactionTaskSlot( + clusterConfig.getCompactionTaskSlotRatio(), + clusterConfig.getMaxCompactionTaskSlots() + 10 + ); + updateCompactionTaskSlot( + clusterConfig.getCompactionTaskSlotRatio(), + clusterConfig.getMaxCompactionTaskSlots() + ); + final CompactionSimulateResult simulateResult = simulateRunOnCoordinator(); + log.info( + "Triggering compaction duty on Coordinator. Expected jobs: %s", + simulateResult.getCompactionStates() + ); + String url = StringUtils.format("%scompaction/compact", getCoordinatorURL()); StatusResponseHolder response = httpClient.go(new Request(HttpMethod.POST, new URL(url)), responseHandler).get(); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -145,9 +214,52 @@ public void forceTriggerAutoCompaction() throws Exception } } - public void updateCompactionTaskSlot(Double compactionTaskSlotRatio, Integer maxCompactionTaskSlots) throws Exception + public void updateClusterConfig(ClusterCompactionConfig config) throws Exception { - String url = StringUtils.format( + final String url = StringUtils.format( + "%s/compaction/config/cluster", + getOverlordURL() + ); + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.POST, new URL(url)).setContent( + "application/json", + jsonMapper.writeValueAsBytes(config) + ), + responseHandler + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while updating cluster compaction config, status[%s], content[%s]", + response.getStatus(), + response.getContent() + ); + } + } + + public ClusterCompactionConfig getClusterConfig() throws Exception + { + String url = StringUtils.format("%s/compaction/config/cluster", getOverlordURL()); + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.GET, new URL(url)), responseHandler + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while getting compaction config status[%s] content[%s]", + response.getStatus(), + response.getContent() + ); + } + return jsonMapper.readValue(response.getContent(), new TypeReference<>() {}); + } + + /** + * This API is currently only to force the coordinator to refresh its config. + * For all other purposes, use {@link #updateClusterConfig}. + */ + @Deprecated + private void updateCompactionTaskSlot(Double compactionTaskSlotRatio, Integer maxCompactionTaskSlots) throws Exception + { + final String url = StringUtils.format( "%sconfig/compaction/taskslots?ratio=%s&max=%s", getCoordinatorURL(), StringUtils.urlEncode(compactionTaskSlotRatio.toString()), @@ -197,4 +309,26 @@ public Map getCompactionStatus(String dataSource) throws Excepti Map>> latestSnapshots = jsonMapper.readValue(response.getContent(), new TypeReference<>() {}); return latestSnapshots.get("latestStatus").get(0); } + + public CompactionSimulateResult simulateRunOnCoordinator() throws Exception + { + final ClusterCompactionConfig clusterConfig = getClusterConfig(); + + final String url = StringUtils.format("%scompaction/simulate", getCoordinatorURL()); + StatusResponseHolder response = httpClient.go( + new Request(HttpMethod.POST, new URL(url)).setContent( + "application/json", + jsonMapper.writeValueAsBytes(clusterConfig) + ), + responseHandler + ).get(); + if (!response.getStatus().equals(HttpResponseStatus.OK)) { + throw new ISE( + "Error while running simulation on Coordinator: status[%s], content[%s]", + response.getStatus(), response.getContent() + ); + } + + return jsonMapper.readValue(response.getContent(), new TypeReference<>() {}); + } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java index 3101786a4387..4fbbbdbb92a4 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionLockContentionTest.java @@ -25,8 +25,8 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; import org.apache.druid.testing.clients.CompactionResourceTestClient; import org.apache.druid.testing.clients.TaskResponseObject; @@ -290,19 +290,12 @@ private void submitAndVerifyCompactionConfig() throws Exception .withSkipOffsetFromLatest(Period.ZERO) .withMaxRowsPerSegment(Specs.MAX_ROWS_PER_SEGMENT) .build(); - compactionResource.updateCompactionTaskSlot(0.5, 10); + compactionResource.updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, null, null)); compactionResource.submitCompactionConfig(dataSourceCompactionConfig); - // Wait for compaction config to persist - Thread.sleep(2000); - // Verify that the compaction config is updated correctly. - DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); DataSourceCompactionConfig observedCompactionConfig - = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); - Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); - - observedCompactionConfig = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); + = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); Assert.assertEquals(observedCompactionConfig, dataSourceCompactionConfig); } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index fe6efcf47e74..48ef5994177b 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Ordering; import com.google.inject.Inject; import org.apache.commons.io.IOUtils; import org.apache.datasketches.hll.TgtHllType; @@ -58,7 +59,9 @@ import org.apache.druid.segment.AutoTypeColumnSchema; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; +import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.coordinator.InlineSchemaDataSourceCompactionConfig; @@ -99,6 +102,7 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; @Test(groups = {TestNGGroup.COMPACTION}) @Guice(moduleFactory = DruidTestModuleFactory.class) @@ -115,6 +119,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest private static final String INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS = "/indexer/wikipedia_index_no_rollup_preserve_metric.json"; private static final int MAX_ROWS_PER_SEGMENT_COMPACTED = 10000; private static final Period NO_SKIP_OFFSET = Period.seconds(0); + private static final FixedIntervalOrderPolicy COMPACT_NOTHING_POLICY = new FixedIntervalOrderPolicy(List.of()); @DataProvider(name = "engine") public static Object[][] engine() @@ -122,6 +127,12 @@ public static Object[][] engine() return new Object[][]{{CompactionEngine.NATIVE}}; } + @DataProvider(name = "useSupervisors") + public static Object[][] useSupervisors() + { + return new Object[][]{{true}, {false}}; + } + @Inject protected CompactionResourceTestClient compactionResource; @@ -621,7 +632,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception checkCompactionIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, Matchers.equalTo(0L), Matchers.greaterThan(0L), Matchers.greaterThan(0L), @@ -639,7 +650,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception checkCompactionIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, Matchers.equalTo(0L), Matchers.greaterThan(0L), Matchers.equalTo(0L), @@ -770,7 +781,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception checkCompactionIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, Matchers.greaterThan(0L), Matchers.greaterThan(0L), Matchers.equalTo(0L), @@ -792,7 +803,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception checkCompactionIntervals(intervalsBeforeCompaction); getAndAssertCompactionStatus( fullDatasourceName, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, Matchers.equalTo(0L), Matchers.greaterThan(0L), Matchers.equalTo(0L), @@ -1586,13 +1597,15 @@ public void testAutoCompactionDutyWithDimensionsSpec(CompactionEngine engine) th } } - @Test - public void testAutoCompactionDutyWithFilter() throws Exception + @Test(dataProvider = "useSupervisors") + public void testAutoCompactionDutyWithFilter(boolean useSupervisors) throws Exception { + updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null)); + loadData(INDEX_TASK_WITH_DIMENSION_SPEC); try (final Closeable ignored = unloader(fullDatasourceName)) { final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + intervalsBeforeCompaction.sort(Ordering.natural().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1616,7 +1629,7 @@ public void testAutoCompactionDutyWithFilter() throws Exception false, CompactionEngine.NATIVE ); - forceTriggerAutoCompaction(2); + forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); // For dim "page", result should only contain value "Striker Eureka" queryAndResultFields = ImmutableMap.of( @@ -1629,19 +1642,21 @@ public void testAutoCompactionDutyWithFilter() throws Exception List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); // Verify compacted segments does not get compacted again - forceTriggerAutoCompaction(2); + forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } } - @Test - public void testAutoCompationDutyWithMetricsSpec() throws Exception + @Test(dataProvider = "useSupervisors") + public void testAutoCompationDutyWithMetricsSpec(boolean useSupervisors) throws Exception { + updateClusterConfig(new ClusterCompactionConfig(0.5, 10, null, useSupervisors, null)); + loadData(INDEX_TASK_WITH_DIMENSION_SPEC); try (final Closeable ignored = unloader(fullDatasourceName)) { final List intervalsBeforeCompaction = coordinator.getSegmentIntervals(fullDatasourceName); - intervalsBeforeCompaction.sort(null); + intervalsBeforeCompaction.sort(Ordering.natural().reversed()); // 4 segments across 2 days (4 total)... verifySegmentsCount(4); @@ -1664,7 +1679,7 @@ public void testAutoCompationDutyWithMetricsSpec() throws Exception false, CompactionEngine.NATIVE ); - forceTriggerAutoCompaction(2); + forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); // Result should be the same with the addition of new metrics, "double_sum_added" and "long_sum_added". // These new metrics should have the same value as the input field "added" @@ -1686,7 +1701,7 @@ public void testAutoCompationDutyWithMetricsSpec() throws Exception List compactTasksBefore = indexer.getCompleteTasksForDataSource(fullDatasourceName); // Verify compacted segments does not get compacted again - forceTriggerAutoCompaction(2); + forceTriggerAutoCompaction(intervalsBeforeCompaction, useSupervisors, 2); List compactTasksAfter = indexer.getCompleteTasksForDataSource(fullDatasourceName); Assert.assertEquals(compactTasksAfter.size(), compactTasksBefore.size()); } @@ -1807,6 +1822,12 @@ private void verifyQuery(String queryResource, Map keyValueToRep queryHelper.testQueriesFromString(queryResponseTemplate); } + private void updateClusterConfig(ClusterCompactionConfig clusterConfig) throws Exception + { + compactionResource.updateClusterConfig(clusterConfig); + LOG.info("Updated cluster config to [%s]", clusterConfig); + } + private void submitCompactionConfig( Integer maxRowsPerSegment, Period skipOffsetFromLatest, @@ -1925,15 +1946,8 @@ private void submitCompactionConfig( Thread.sleep(2000); // Verify that the compaction config is updated correctly. - DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); DataSourceCompactionConfig foundDataSourceCompactionConfig - = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); - Assert.assertNotNull(foundDataSourceCompactionConfig); - Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); - Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), partitionsSpec); - Assert.assertEquals(foundDataSourceCompactionConfig.getSkipOffsetFromLatest(), skipOffsetFromLatest); - - foundDataSourceCompactionConfig = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); + = compactionResource.getDataSourceCompactionConfig(fullDatasourceName); Assert.assertNotNull(foundDataSourceCompactionConfig); Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), partitionsSpec); @@ -1945,15 +1959,56 @@ private void deleteCompactionConfig() throws Exception compactionResource.deleteDataSourceCompactionConfig(fullDatasourceName); // Verify that the compaction config is updated correctly. - DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); + DruidCompactionConfig compactionConfig = DruidCompactionConfig + .empty().withDatasourceConfigs(compactionResource.getAllCompactionConfigs()); DataSourceCompactionConfig foundDataSourceCompactionConfig = compactionConfig.findConfigForDatasource(fullDatasourceName).orNull(); Assert.assertNull(foundDataSourceCompactionConfig); } + /** + * Performs compaction of the given intervals of the test datasource, + * {@link #fullDatasourceName}, and verifies the total number of segments in + * the datasource after compaction. + */ + private void forceTriggerAutoCompaction( + List intervals, + boolean useSupervisors, + int numExpectedSegmentsAfterCompaction + ) throws Exception + { + if (useSupervisors) { + // Enable compaction for the requested intervals + final FixedIntervalOrderPolicy policy = new FixedIntervalOrderPolicy( + intervals.stream().map( + interval -> new FixedIntervalOrderPolicy.Candidate(fullDatasourceName, Intervals.of(interval)) + ).collect(Collectors.toList()) + ); + updateClusterConfig( + new ClusterCompactionConfig(0.5, intervals.size(), policy, true, null) + ); + + // Wait for scheduler to pick up the compaction job + Thread.sleep(30_000); + waitForCompactionToFinish(numExpectedSegmentsAfterCompaction); + + // Disable all compaction + updateClusterConfig( + new ClusterCompactionConfig(0.5, intervals.size(), COMPACT_NOTHING_POLICY, true, null) + ); + } else { + forceTriggerAutoCompaction(numExpectedSegmentsAfterCompaction); + } + } + private void forceTriggerAutoCompaction(int numExpectedSegmentsAfterCompaction) throws Exception { compactionResource.forceTriggerAutoCompaction(); + waitForCompactionToFinish(numExpectedSegmentsAfterCompaction); + } + + private void waitForCompactionToFinish(int numExpectedSegmentsAfterCompaction) + { waitForAllTasksToCompleteForDataSource(fullDatasourceName); ITRetryUtil.retryUntilTrue( () -> coordinator.areSegmentsLoaded(fullDatasourceName), @@ -2042,16 +2097,30 @@ private void verifySegmentsCompactedDimensionSchema(List dimens private void updateCompactionTaskSlot(double compactionTaskSlotRatio, int maxCompactionTaskSlots) throws Exception { - compactionResource.updateCompactionTaskSlot(compactionTaskSlotRatio, maxCompactionTaskSlots); - // Verify that the compaction config is updated correctly. - DruidCompactionConfig compactionConfig = compactionResource.getCompactionConfig(); - Assert.assertEquals(compactionConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); - Assert.assertEquals(compactionConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); + final ClusterCompactionConfig oldConfig = compactionResource.getClusterConfig(); + compactionResource.updateClusterConfig( + new ClusterCompactionConfig( + compactionTaskSlotRatio, + maxCompactionTaskSlots, + oldConfig.getCompactionPolicy(), + oldConfig.isUseSupervisors(), + oldConfig.getEngine() + ) + ); + + // Verify that the compaction config is updated correctly + final ClusterCompactionConfig updatedConfig = compactionResource.getClusterConfig(); + Assert.assertEquals(updatedConfig.getCompactionTaskSlotRatio(), compactionTaskSlotRatio); + Assert.assertEquals(updatedConfig.getMaxCompactionTaskSlots(), maxCompactionTaskSlots); + LOG.info( + "Updated compactionTaskSlotRatio[%s] and maxCompactionTaskSlots[%d]", + compactionTaskSlotRatio, maxCompactionTaskSlots + ); } private void getAndAssertCompactionStatus( String fullDatasourceName, - AutoCompactionSnapshot.AutoCompactionScheduleStatus scheduleStatus, + AutoCompactionSnapshot.ScheduleStatus scheduleStatus, Matcher bytesAwaitingCompactionMatcher, Matcher bytesCompactedMatcher, Matcher bytesSkippedMatcher, diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java index 730d4312aa4a..505e261b2687 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionUpgradeTest.java @@ -57,7 +57,8 @@ public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVers { // Verify that compaction config already exist. This config was inserted manually into the database using SQL script. // This auto compaction configuration payload is from Druid 0.21.0 - DruidCompactionConfig coordinatorCompactionConfig = compactionResource.getCompactionConfig(); + DruidCompactionConfig coordinatorCompactionConfig = DruidCompactionConfig.empty() + .withDatasourceConfigs(compactionResource.getAllCompactionConfigs()); DataSourceCompactionConfig foundDataSourceCompactionConfig = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); Assert.assertNotNull(foundDataSourceCompactionConfig); @@ -100,13 +101,9 @@ public void testUpgradeAutoCompactionConfigurationWhenConfigurationFromOlderVers .build(); compactionResource.submitCompactionConfig(compactionConfig); - // Wait for compaction config to persist - Thread.sleep(2000); - // Verify that compaction was successfully updated - coordinatorCompactionConfig = compactionResource.getCompactionConfig(); foundDataSourceCompactionConfig - = coordinatorCompactionConfig.findConfigForDatasource(UPGRADE_DATASOURCE_NAME).orNull(); + = compactionResource.getDataSourceCompactionConfig(UPGRADE_DATASOURCE_NAME); Assert.assertNotNull(foundDataSourceCompactionConfig); Assert.assertNotNull(foundDataSourceCompactionConfig.getTuningConfig()); Assert.assertEquals(foundDataSourceCompactionConfig.getTuningConfig().getPartitionsSpec(), newPartitionsSpec); diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index edeb16665ba4..bcd3e4348e52 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -25,9 +25,11 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.List; import java.util.Set; @@ -74,4 +76,15 @@ public interface CoordinatorClient * Retrieves list of datasources with used segments. */ ListenableFuture> fetchDataSourcesWithUsedSegments(); + + /** + * Gets the latest compaction snapshots of one or all datasources. + *

+ * API: {@code GET /druid/coordinator/v1/compaction/status} + * + * @param dataSource If passed as non-null, then the returned list contains only + * the snapshot for this datasource. + */ + ListenableFuture getCompactionSnapshots(@Nullable String dataSource); + } diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java index 224a766c719d..50cd58e0eb33 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClientImpl.java @@ -36,11 +36,13 @@ import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordination.LoadableDataSegment; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpMethod; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; import java.util.Set; @@ -199,4 +201,25 @@ public ListenableFuture> fetchDataSourcesWithUsedSegments() holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), new TypeReference<>() {}) ); } + + @Override + public ListenableFuture getCompactionSnapshots(@Nullable String dataSource) + { + final StringBuilder pathBuilder = new StringBuilder("/druid/coordinator/v1/compaction/status"); + if (dataSource != null && !dataSource.isEmpty()) { + pathBuilder.append("?").append("dataSource=").append(StringUtils.urlEncode(dataSource)); + } + + return FutureUtils.transform( + client.asyncRequest( + new RequestBuilder(HttpMethod.GET, pathBuilder.toString()), + new BytesFullResponseHandler() + ), + holder -> JacksonUtils.readValue( + jsonMapper, + holder.getContent(), + CompactionStatusResponse.class + ) + ); + } } diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/NoopOverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/NoopOverlordClient.java index 7ccdcedb2bd3..a1782d33112b 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/NoopOverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/NoopOverlordClient.java @@ -31,8 +31,6 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.server.compaction.CompactionProgressResponse; -import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.http.SegmentsToUpdateFilter; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; @@ -133,12 +131,6 @@ public ListenableFuture getTotalWorkerCapacity( throw new UnsupportedOperationException(); } - @Override - public ListenableFuture getCompactionSnapshots(@Nullable String dataSource) - { - throw new UnsupportedOperationException(); - } - @Override public ListenableFuture markNonOvershadowedSegmentsAsUsed(String dataSource) { @@ -178,12 +170,6 @@ public ListenableFuture markSegmentAsUnused(SegmentId seg throw new UnsupportedOperationException(); } - @Override - public ListenableFuture getBytesAwaitingCompaction(String dataSource) - { - throw new UnsupportedOperationException(); - } - @Override public ListenableFuture isCompactionSupervisorEnabled() { diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java index b14c287e734f..57899844d8b3 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClient.java @@ -35,8 +35,6 @@ import org.apache.druid.java.util.common.parsers.CloseableIterator; import org.apache.druid.metadata.LockFilterPolicy; import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.server.compaction.CompactionProgressResponse; -import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.http.SegmentsToUpdateFilter; import org.apache.druid.timeline.SegmentId; import org.joda.time.DateTime; @@ -224,23 +222,6 @@ ListenableFuture>> findLockedIntervals( */ ListenableFuture isCompactionSupervisorEnabled(); - /** - * Gets the number of bytes yet to be compacted for the given datasource. - *

- * API: {@code /druid/indexer/v1/compaction/progress} - */ - ListenableFuture getBytesAwaitingCompaction(String dataSource); - - /** - * Gets the latest compaction snapshots of one or all datasources. - *

- * API: {@code /druid/indexer/v1/compaction/status} - * - * @param dataSource If passed as non-null, then the returned list contains only - * the snapshot for this datasource. - */ - ListenableFuture getCompactionSnapshots(@Nullable String dataSource); - /** * Marks all non-overshadowed segments of the datasource as used. *

diff --git a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java index ba6a6e76d55f..1b0f21fced49 100644 --- a/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java +++ b/server/src/main/java/org/apache/druid/rpc/indexing/OverlordClientImpl.java @@ -45,8 +45,6 @@ import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.rpc.ServiceClient; import org.apache.druid.rpc.ServiceRetryPolicy; -import org.apache.druid.server.compaction.CompactionProgressResponse; -import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.http.SegmentsToUpdateFilter; import org.apache.druid.timeline.SegmentId; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -316,27 +314,6 @@ public ListenableFuture taskPayload(final String taskId) ); } - @Override - public ListenableFuture getCompactionSnapshots(@Nullable String dataSource) - { - final StringBuilder pathBuilder = new StringBuilder("/druid/indexer/v1/compaction/status"); - if (dataSource != null && !dataSource.isEmpty()) { - pathBuilder.append("?").append("dataSource=").append(dataSource); - } - - return FutureUtils.transform( - client.asyncRequest( - new RequestBuilder(HttpMethod.GET, pathBuilder.toString()), - new BytesFullResponseHandler() - ), - holder -> JacksonUtils.readValue( - jsonMapper, - holder.getContent(), - CompactionStatusResponse.class - ) - ); - } - @Override public ListenableFuture markNonOvershadowedSegmentsAsUsed(String dataSource) { @@ -443,19 +420,6 @@ public ListenableFuture markSegmentAsUnused(SegmentId seg ); } - @Override - public ListenableFuture getBytesAwaitingCompaction(String dataSource) - { - final String path = "/druid/indexer/v1/compaction/progress?dataSource=" + dataSource; - return FutureUtils.transform( - client.asyncRequest( - new RequestBuilder(HttpMethod.GET, path), - new BytesFullResponseHandler() - ), - holder -> JacksonUtils.readValue(jsonMapper, holder.getContent(), CompactionProgressResponse.class) - ); - } - @Override public ListenableFuture isCompactionSupervisorEnabled() { diff --git a/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java index 21f6cafbfaf4..7d7d117f08fc 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/BaseCandidateSearchPolicy.java @@ -30,8 +30,7 @@ * Base implementation of {@link CompactionCandidateSearchPolicy} that can have * a {@code priorityDatasource}. */ -public abstract class BaseCandidateSearchPolicy - implements CompactionCandidateSearchPolicy, Comparator +public abstract class BaseCandidateSearchPolicy implements CompactionCandidateSearchPolicy { private final String priorityDatasource; private final Comparator comparator; @@ -59,7 +58,7 @@ public final String getPriorityDatasource() } @Override - public final int compare(CompactionCandidate o1, CompactionCandidate o2) + public final int compareCandidates(CompactionCandidate o1, CompactionCandidate o2) { return comparator.compare(o1, o2); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java index cf0e016f0540..cc99e03bf214 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionCandidateSearchPolicy.java @@ -23,23 +23,25 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import org.apache.druid.server.coordinator.duty.CompactSegments; -import java.util.Comparator; - /** * Policy used by {@link CompactSegments} duty to pick segments for compaction. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "newestSegmentFirst", value = NewestSegmentFirstPolicy.class) + @JsonSubTypes.Type(name = "newestSegmentFirst", value = NewestSegmentFirstPolicy.class), + @JsonSubTypes.Type(name = "fixedIntervalOrder", value = FixedIntervalOrderPolicy.class) }) -public interface CompactionCandidateSearchPolicy extends Comparator +public interface CompactionCandidateSearchPolicy { /** * Compares between two compaction candidates. Used to determine the * order in which segments and intervals should be picked for compaction. + * + * @return A positive value if {@code candidateA} should be picked first, a + * negative value if {@code candidateB} should be picked first or zero if the + * order does not matter. */ - @Override - int compare(CompactionCandidate o1, CompactionCandidate o2); + int compareCandidates(CompactionCandidate candidateA, CompactionCandidate candidateB); /** * Checks if the given {@link CompactionCandidate} is eligible for compaction diff --git a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusResponse.java b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusResponse.java index 756da7aa9c4b..dee0e5291ccd 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusResponse.java +++ b/server/src/main/java/org/apache/druid/server/compaction/CompactionStatusResponse.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; -import java.util.Collection; +import java.util.List; import java.util.Objects; /** @@ -32,18 +32,18 @@ */ public class CompactionStatusResponse { - private final Collection latestStatus; + private final List latestStatus; @JsonCreator public CompactionStatusResponse( - @JsonProperty("latestStatus") Collection latestStatus + @JsonProperty("latestStatus") List latestStatus ) { this.latestStatus = latestStatus; } @JsonProperty - public Collection getLatestStatus() + public List getLatestStatus() { return latestStatus; } diff --git a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java index 86678e9da9bd..e15d310b33db 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/DataSourceCompactibleSegmentIterator.java @@ -91,7 +91,7 @@ public DataSourceCompactibleSegmentIterator( this.config = config; this.dataSource = config.getDataSource(); this.searchPolicy = searchPolicy; - this.queue = new PriorityQueue<>(searchPolicy); + this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); populateQueue(timeline, skipIntervals); } diff --git a/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java b/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java new file mode 100644 index 000000000000..3e8726471b13 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/compaction/FixedIntervalOrderPolicy.java @@ -0,0 +1,114 @@ +/* + * 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.druid.server.compaction; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.joda.time.Interval; + +import java.util.List; + +/** + * Implementation of {@link CompactionCandidateSearchPolicy} that specifies the + * datasources and intervals eligible for compaction and their order. + *

+ * This policy is primarily used for integration tests. + */ +public class FixedIntervalOrderPolicy implements CompactionCandidateSearchPolicy +{ + private final List eligibleCandidates; + + @JsonCreator + public FixedIntervalOrderPolicy( + @JsonProperty("eligibleCandidates") List eligibleCandidates + ) + { + this.eligibleCandidates = eligibleCandidates; + } + + @JsonProperty + public List getEligibleCandidates() + { + return eligibleCandidates; + } + + @Override + public int compareCandidates(CompactionCandidate candidateA, CompactionCandidate candidateB) + { + return findIndex(candidateA) - findIndex(candidateB); + } + + @Override + public boolean isEligibleForCompaction( + CompactionCandidate candidate, + CompactionStatus currentCompactionStatus, + CompactionTaskStatus latestTaskStatus + ) + { + return findIndex(candidate) < Integer.MAX_VALUE; + } + + private int findIndex(CompactionCandidate candidate) + { + int index = 0; + for (Candidate eligibleCandidate : eligibleCandidates) { + boolean found = eligibleCandidate.datasource.equals(candidate.getDataSource()) + && eligibleCandidate.interval.contains(candidate.getUmbrellaInterval()); + if (found) { + return index; + } else { + index++; + } + } + + return Integer.MAX_VALUE; + } + + /** + * Specifies a datasource-interval eligible for compaction. + */ + public static class Candidate + { + private final String datasource; + private final Interval interval; + + @JsonCreator + public Candidate( + @JsonProperty("datasource") String datasource, + @JsonProperty("interval") Interval interval + ) + { + this.datasource = datasource; + this.interval = interval; + } + + @JsonProperty + public String getDatasource() + { + return datasource; + } + + @JsonProperty + public Interval getInterval() + { + return interval; + } + } +} diff --git a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java index a64c3b1e6410..1e91df7e38a0 100644 --- a/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/compaction/PriorityBasedCompactionSegmentIterator.java @@ -52,7 +52,7 @@ public PriorityBasedCompactionSegmentIterator( CompactionStatusTracker statusTracker ) { - this.queue = new PriorityQueue<>(searchPolicy); + this.queue = new PriorityQueue<>(searchPolicy::compareCandidates); this.datasourceIterators = Maps.newHashMapWithExpectedSize(datasourceToTimeline.size()); compactionConfigs.forEach((datasource, config) -> { if (config == null) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java index 8aa8882b1cb0..e31a7919f24f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java @@ -31,16 +31,17 @@ public class AutoCompactionSnapshot { - public enum AutoCompactionScheduleStatus + public enum ScheduleStatus { NOT_ENABLED, + AWAITING_FIRST_RUN, RUNNING } @JsonProperty private final String dataSource; @JsonProperty - private final AutoCompactionScheduleStatus scheduleStatus; + private final ScheduleStatus scheduleStatus; @JsonProperty private final String message; @JsonProperty @@ -64,13 +65,13 @@ public enum AutoCompactionScheduleStatus public static Builder builder(String dataSource) { - return new Builder(dataSource).withStatus(AutoCompactionScheduleStatus.RUNNING); + return new Builder(dataSource).withStatus(ScheduleStatus.RUNNING); } @JsonCreator public AutoCompactionSnapshot( @JsonProperty("dataSource") @NotNull String dataSource, - @JsonProperty("scheduleStatus") @NotNull AutoCompactionScheduleStatus scheduleStatus, + @JsonProperty("scheduleStatus") @NotNull AutoCompactionSnapshot.ScheduleStatus scheduleStatus, @JsonProperty("message") @Nullable String message, @JsonProperty("bytesAwaitingCompaction") long bytesAwaitingCompaction, @JsonProperty("bytesCompacted") long bytesCompacted, @@ -104,7 +105,7 @@ public String getDataSource() } @NotNull - public AutoCompactionScheduleStatus getScheduleStatus() + public AutoCompactionSnapshot.ScheduleStatus getScheduleStatus() { return scheduleStatus; } @@ -206,7 +207,7 @@ public int hashCode() public static class Builder { private final String dataSource; - private AutoCompactionScheduleStatus scheduleStatus; + private ScheduleStatus scheduleStatus; private String message; private final CompactionStatistics compactedStats = new CompactionStatistics(); @@ -223,7 +224,7 @@ private Builder( this.dataSource = dataSource; } - public Builder withStatus(AutoCompactionScheduleStatus status) + public Builder withStatus(ScheduleStatus status) { this.scheduleStatus = Preconditions.checkNotNull(status, "scheduleStatus cannot be null"); return this; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java index 7334251da76c..437849ad1bf2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ClusterCompactionConfig.java @@ -40,9 +40,9 @@ public class ClusterCompactionConfig private static final CompactionCandidateSearchPolicy DEFAULT_COMPACTION_POLICY = new NewestSegmentFirstPolicy(null); - private final Double compactionTaskSlotRatio; - private final Integer maxCompactionTaskSlots; - private final Boolean useSupervisors; + private final double compactionTaskSlotRatio; + private final int maxCompactionTaskSlots; + private final boolean useSupervisors; private final CompactionEngine engine; private final CompactionCandidateSearchPolicy compactionPolicy; @@ -85,7 +85,7 @@ public CompactionCandidateSearchPolicy getCompactionPolicy() } @JsonProperty - public boolean getUseSupervisors() + public boolean isUseSupervisors() { return useSupervisors; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java index e8265bcdd1af..9313ba1ff15b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorConfigManager.java @@ -19,14 +19,30 @@ package org.apache.druid.server.coordinator; +import com.google.common.base.Optional; import com.google.common.base.Preconditions; +import com.google.common.base.Throwables; import com.google.inject.Inject; +import org.apache.druid.audit.AuditEntry; import org.apache.druid.audit.AuditInfo; +import org.apache.druid.audit.AuditManager; import org.apache.druid.common.config.ConfigManager; import org.apache.druid.common.config.JacksonConfigManager; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.InternalServerError; +import org.apache.druid.error.NotFound; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.joda.time.Interval; +import javax.annotation.Nullable; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.concurrent.ThreadLocalRandom; import java.util.function.UnaryOperator; /** @@ -35,6 +51,12 @@ */ public class CoordinatorConfigManager { + private static final Logger log = new Logger(CoordinatorConfigManager.class); + + private static final long UPDATE_RETRY_DELAY = 1000; + static final int MAX_UPDATE_RETRIES = 5; + + private final AuditManager auditManager; private final JacksonConfigManager jacksonConfigManager; private final MetadataStorageConnector metadataStorageConnector; private final MetadataStorageTablesConfig tablesConfig; @@ -43,12 +65,14 @@ public class CoordinatorConfigManager public CoordinatorConfigManager( JacksonConfigManager jacksonConfigManager, MetadataStorageConnector metadataStorageConnector, - MetadataStorageTablesConfig tablesConfig + MetadataStorageTablesConfig tablesConfig, + AuditManager auditManager ) { this.jacksonConfigManager = jacksonConfigManager; this.metadataStorageConnector = metadataStorageConnector; this.tablesConfig = tablesConfig; + this.auditManager = auditManager; } public CoordinatorDynamicConfig getCurrentDynamicConfig() @@ -127,4 +151,150 @@ public DruidCompactionConfig convertBytesToCompactionConfig(byte[] bytes) DruidCompactionConfig.empty() ); } + + public boolean updateClusterCompactionConfig( + ClusterCompactionConfig config, + AuditInfo auditInfo + ) + { + UnaryOperator operator = current -> current.withClusterConfig(config); + return updateConfigHelper(operator, auditInfo); + } + + public ClusterCompactionConfig getClusterCompactionConfig() + { + return getCurrentCompactionConfig().clusterConfig(); + } + + public boolean updateDatasourceCompactionConfig( + DataSourceCompactionConfig config, + AuditInfo auditInfo + ) + { + UnaryOperator callable = current -> current.withDatasourceConfig(config); + return updateConfigHelper(callable, auditInfo); + } + + public DataSourceCompactionConfig getDatasourceCompactionConfig(String dataSource) + { + final DruidCompactionConfig current = getCurrentCompactionConfig(); + final Optional config = current.findConfigForDatasource(dataSource); + if (config.isPresent()) { + return config.get(); + } else { + throw NotFound.exception("Datasource compaction config does not exist"); + } + } + + public boolean deleteDatasourceCompactionConfig( + String dataSource, + AuditInfo auditInfo + ) + { + UnaryOperator callable = current -> { + final Map configs = current.dataSourceToCompactionConfigMap(); + final DataSourceCompactionConfig config = configs.remove(dataSource); + if (config == null) { + throw NotFound.exception("Datasource compaction config does not exist"); + } + + return current.withDatasourceConfigs(List.copyOf(configs.values())); + }; + return updateConfigHelper(callable, auditInfo); + } + + public List getCompactionConfigHistory( + String dataSource, + @Nullable String interval, + @Nullable Integer count + ) + { + Interval theInterval = interval == null ? null : Intervals.of(interval); + try { + List auditEntries; + if (theInterval == null && count != null) { + auditEntries = auditManager.fetchAuditHistory( + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, + count + ); + } else { + auditEntries = auditManager.fetchAuditHistory( + DruidCompactionConfig.CONFIG_KEY, + DruidCompactionConfig.CONFIG_KEY, + theInterval + ); + } + DataSourceCompactionConfigHistory history = new DataSourceCompactionConfigHistory(dataSource); + for (AuditEntry audit : auditEntries) { + DruidCompactionConfig compactionConfig = convertBytesToCompactionConfig( + audit.getPayload().serialized().getBytes(StandardCharsets.UTF_8) + ); + history.add(compactionConfig, audit.getAuditInfo(), audit.getAuditTime()); + } + return history.getEntries(); + } + catch (Exception e) { + throw InternalServerError.exception( + Throwables.getRootCause(e), + "Could not fetch audit entries" + ); + } + } + + private boolean updateConfigHelper( + UnaryOperator configUpdateOperator, + AuditInfo auditInfo + ) + { + int attemps = 0; + ConfigManager.SetResult setResult = null; + try { + while (attemps < MAX_UPDATE_RETRIES) { + setResult = getAndUpdateCompactionConfig(configUpdateOperator, auditInfo); + if (setResult.isOk() || !setResult.isRetryable()) { + break; + } + attemps++; + updateRetryDelay(); + } + } + catch (DruidException e) { + throw e; + } + catch (Exception e) { + log.warn(e, "Compaction config update failed"); + throw InternalServerError.exception( + Throwables.getRootCause(e), + "Failed to perform operation on compaction config" + ); + } + + if (setResult.isOk()) { + return true; + } else if (setResult.getException() instanceof NoSuchElementException) { + log.warn(setResult.getException(), "Update compaction config failed"); + throw NotFound.exception( + Throwables.getRootCause(setResult.getException()), + "Compaction config does not exist" + ); + } else { + log.warn(setResult.getException(), "Update compaction config failed"); + throw InternalServerError.exception( + Throwables.getRootCause(setResult.getException()), + "Failed to perform operation on compaction config" + ); + } + } + + private void updateRetryDelay() + { + try { + Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); + } + catch (InterruptedException ie) { + throw new RuntimeException(ie); + } + } + } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java index fdbbb57e53ef..0b70ab5b15ba 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigAuditEntry.java @@ -79,4 +79,26 @@ public boolean hasSameConfig(DataSourceCompactionConfigAuditEntry other) return Objects.equals(this.compactionConfig, other.compactionConfig) && Objects.equals(this.globalConfig, other.globalConfig); } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataSourceCompactionConfigAuditEntry that = (DataSourceCompactionConfigAuditEntry) o; + return Objects.equals(globalConfig, that.globalConfig) + && Objects.equals(compactionConfig, that.compactionConfig) + && Objects.equals(auditInfo, that.auditInfo) + && Objects.equals(auditTime, that.auditTime); + } + + @Override + public int hashCode() + { + return Objects.hash(globalConfig, compactionConfig, auditInfo, auditTime); + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java index d9298c6ab637..cc11d9fdf718 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCompactionConfig.java @@ -131,7 +131,7 @@ public int getMaxCompactionTaskSlots() @JsonProperty public boolean isUseSupervisors() { - return clusterConfig.getUseSupervisors(); + return clusterConfig.isUseSupervisors(); } @JsonProperty diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java index f2cad3b455b2..f24aae7baa58 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResource.java @@ -19,29 +19,17 @@ package org.apache.druid.server.http; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.audit.AuditEntry; import org.apache.druid.audit.AuditInfo; -import org.apache.druid.audit.AuditManager; -import org.apache.druid.common.config.ConfigManager.SetResult; -import org.apache.druid.error.DruidException; +import org.apache.druid.common.config.Configs; import org.apache.druid.error.InvalidInput; -import org.apache.druid.error.NotFound; import org.apache.druid.indexer.CompactionEngine; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; -import org.apache.druid.server.coordinator.DataSourceCompactionConfigHistory; -import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.security.ConfigResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; -import org.joda.time.Interval; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.Consumes; @@ -55,97 +43,80 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.concurrent.ThreadLocalRandom; -import java.util.function.UnaryOperator; @Path("/druid/coordinator/v1/config/compaction") @ResourceFilters(ConfigResourceFilter.class) public class CoordinatorCompactionConfigsResource { - private static final Logger LOG = new Logger(CoordinatorCompactionConfigsResource.class); - private static final long UPDATE_RETRY_DELAY = 1000; - static final int MAX_UPDATE_RETRIES = 5; - private final CoordinatorConfigManager configManager; - private final AuditManager auditManager; @Inject public CoordinatorCompactionConfigsResource( - CoordinatorConfigManager configManager, - AuditManager auditManager + CoordinatorConfigManager configManager ) { this.configManager = configManager; - this.auditManager = auditManager; } @GET @Produces(MediaType.APPLICATION_JSON) public Response getCompactionConfig() { - return Response.ok(configManager.getCurrentCompactionConfig()).build(); - } - - @POST - @Path("/cluster") - @Consumes(MediaType.APPLICATION_JSON) - public Response updateClusterCompactionConfig( - ClusterCompactionConfig updatePayload, - @Context HttpServletRequest req - ) - { - UnaryOperator operator = current -> current.withClusterConfig(updatePayload); - return updateConfigHelper(operator, AuthorizationUtils.buildAuditInfo(req)); + return ServletResourceUtils.buildReadResponse( + configManager::getCurrentCompactionConfig + ); } /** - * @deprecated in favor of {@link #updateClusterCompactionConfig}. + * @deprecated Use API {@code GET /druid/indexer/v1/compaction/config/cluster} instead. */ @POST @Deprecated @Path("/taskslots") @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) public Response setCompactionTaskLimit( @QueryParam("ratio") Double compactionTaskSlotRatio, @QueryParam("max") Integer maxCompactionTaskSlots, @Context HttpServletRequest req ) { - return updateClusterCompactionConfig( - new ClusterCompactionConfig( - compactionTaskSlotRatio, - maxCompactionTaskSlots, - null, - null, - null - ), - req + if (compactionTaskSlotRatio == null && maxCompactionTaskSlots == null) { + return ServletResourceUtils.buildUpdateResponse(() -> true); + } + + final ClusterCompactionConfig currentConfig = configManager.getCurrentCompactionConfig().clusterConfig(); + final ClusterCompactionConfig updatedClusterConfig = new ClusterCompactionConfig( + Configs.valueOrDefault(compactionTaskSlotRatio, currentConfig.getCompactionTaskSlotRatio()), + Configs.valueOrDefault(maxCompactionTaskSlots, currentConfig.getMaxCompactionTaskSlots()), + currentConfig.getCompactionPolicy(), + currentConfig.isUseSupervisors(), + currentConfig.getEngine() + ); + + final AuditInfo auditInfo = AuthorizationUtils.buildAuditInfo(req); + return ServletResourceUtils.buildUpdateResponse( + () -> configManager.updateClusterCompactionConfig(updatedClusterConfig, auditInfo) ); } @POST @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) public Response addOrUpdateDatasourceCompactionConfig( final DataSourceCompactionConfig newConfig, @Context HttpServletRequest req ) { - UnaryOperator callable = current -> { + final AuditInfo auditInfo = AuthorizationUtils.buildAuditInfo(req); + return ServletResourceUtils.buildUpdateResponse(() -> { if (newConfig.getEngine() == CompactionEngine.MSQ) { throw InvalidInput.exception( - "MSQ engine in compaction config only supported with supervisor-based compaction on the Overlord."); - } else { - return current.withDatasourceConfig(newConfig); + "MSQ engine is supported only with supervisor-based compaction on the Overlord." + ); } - }; - return updateConfigHelper( - callable, - AuthorizationUtils.buildAuditInfo(req) - ); + return configManager.updateDatasourceCompactionConfig(newConfig, auditInfo); + }); } @GET @@ -153,13 +124,9 @@ public Response addOrUpdateDatasourceCompactionConfig( @Produces(MediaType.APPLICATION_JSON) public Response getDatasourceCompactionConfig(@PathParam("dataSource") String dataSource) { - final DruidCompactionConfig current = configManager.getCurrentCompactionConfig(); - final Optional config = current.findConfigForDatasource(dataSource); - if (config.isPresent()) { - return Response.ok().entity(config.get()).build(); - } else { - return Response.status(Response.Status.NOT_FOUND).build(); - } + return ServletResourceUtils.buildReadResponse( + () -> configManager.getDatasourceCompactionConfig(dataSource) + ); } @GET @@ -171,36 +138,9 @@ public Response getCompactionConfigHistory( @QueryParam("count") Integer count ) { - Interval theInterval = interval == null ? null : Intervals.of(interval); - try { - List auditEntries; - if (theInterval == null && count != null) { - auditEntries = auditManager.fetchAuditHistory( - DruidCompactionConfig.CONFIG_KEY, - DruidCompactionConfig.CONFIG_KEY, - count - ); - } else { - auditEntries = auditManager.fetchAuditHistory( - DruidCompactionConfig.CONFIG_KEY, - DruidCompactionConfig.CONFIG_KEY, - theInterval - ); - } - DataSourceCompactionConfigHistory history = new DataSourceCompactionConfigHistory(dataSource); - for (AuditEntry audit : auditEntries) { - DruidCompactionConfig compactionConfig = configManager.convertBytesToCompactionConfig( - audit.getPayload().serialized().getBytes(StandardCharsets.UTF_8) - ); - history.add(compactionConfig, audit.getAuditInfo(), audit.getAuditTime()); - } - return Response.ok(history.getEntries()).build(); - } - catch (IllegalArgumentException e) { - return Response.status(Response.Status.BAD_REQUEST) - .entity(ServletResourceUtils.sanitizeException(e)) - .build(); - } + return ServletResourceUtils.buildReadResponse( + () -> configManager.getCompactionConfigHistory(dataSource, interval, count) + ); } @DELETE @@ -211,74 +151,9 @@ public Response deleteCompactionConfig( @Context HttpServletRequest req ) { - UnaryOperator callable = current -> { - final Map configs = current.dataSourceToCompactionConfigMap(); - final DataSourceCompactionConfig config = configs.remove(dataSource); - if (config == null) { - throw NotFound.exception("datasource not found"); - } - - return current.withDatasourceConfigs(ImmutableList.copyOf(configs.values())); - }; - return updateConfigHelper(callable, AuthorizationUtils.buildAuditInfo(req)); - } - - private Response updateConfigHelper( - UnaryOperator configOperator, - AuditInfo auditInfo - ) - { - int attemps = 0; - SetResult setResult = null; - try { - while (attemps < MAX_UPDATE_RETRIES) { - setResult = configManager.getAndUpdateCompactionConfig(configOperator, auditInfo); - if (setResult.isOk() || !setResult.isRetryable()) { - break; - } - attemps++; - updateRetryDelay(); - } - } - catch (DruidException e) { - return ServletResourceUtils.buildErrorResponseFrom(e); - } - catch (Exception e) { - LOG.warn(e, "Update compaction config failed"); - return Response.status(Response.Status.INTERNAL_SERVER_ERROR) - .entity(ImmutableMap.of("error", createErrorMessage(e))) - .build(); - } - - if (setResult.isOk()) { - return Response.ok().build(); - } else if (setResult.getException() instanceof NoSuchElementException) { - LOG.warn(setResult.getException(), "Update compaction config failed"); - return Response.status(Response.Status.NOT_FOUND).build(); - } else { - LOG.warn(setResult.getException(), "Update compaction config failed"); - return Response.status(Response.Status.BAD_REQUEST) - .entity(ImmutableMap.of("error", createErrorMessage(setResult.getException()))) - .build(); - } - } - - private void updateRetryDelay() - { - try { - Thread.sleep(ThreadLocalRandom.current().nextLong(UPDATE_RETRY_DELAY)); - } - catch (InterruptedException ie) { - throw new RuntimeException(ie); - } - } - - private String createErrorMessage(Exception e) - { - if (e.getMessage() == null) { - return "Unknown Error"; - } else { - return e.getMessage(); - } + final AuditInfo auditInfo = AuthorizationUtils.buildAuditInfo(req); + return ServletResourceUtils.buildUpdateResponse( + () -> configManager.deleteDatasourceCompactionConfig(dataSource, auditInfo) + ); } } diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java index cb146b7ed955..0e465abbc5c1 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorCompactionResource.java @@ -20,16 +20,10 @@ package org.apache.druid.server.http; import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.common.guava.FutureUtils; -import org.apache.druid.error.InternalServerError; import org.apache.druid.error.InvalidInput; import org.apache.druid.error.NotFound; -import org.apache.druid.rpc.HttpResponseException; -import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.server.compaction.CompactionProgressResponse; import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; @@ -46,22 +40,19 @@ import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.Collection; +import java.util.List; @Path("/druid/coordinator/v1/compaction") public class CoordinatorCompactionResource { private final DruidCoordinator coordinator; - private final OverlordClient overlordClient; @Inject public CoordinatorCompactionResource( - DruidCoordinator coordinator, - OverlordClient overlordClient + DruidCoordinator coordinator ) { this.coordinator = coordinator; - this.overlordClient = overlordClient; } /** @@ -89,10 +80,6 @@ public Response getCompactionProgress( return ServletResourceUtils.buildErrorResponseFrom(InvalidInput.exception("No DataSource specified")); } - if (isCompactionSupervisorEnabled()) { - return buildResponse(overlordClient.getBytesAwaitingCompaction(dataSource)); - } - final AutoCompactionSnapshot snapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); if (snapshot == null) { return ServletResourceUtils.buildErrorResponseFrom(NotFound.exception("Unknown DataSource")); @@ -109,19 +96,15 @@ public Response getCompactionSnapshotForDataSource( @QueryParam("dataSource") String dataSource ) { - if (isCompactionSupervisorEnabled()) { - return buildResponse(overlordClient.getCompactionSnapshots(dataSource)); - } - - final Collection snapshots; + final List snapshots; if (dataSource == null || dataSource.isEmpty()) { - snapshots = coordinator.getAutoCompactionSnapshot().values(); + snapshots = List.copyOf(coordinator.getAutoCompactionSnapshot().values()); } else { AutoCompactionSnapshot autoCompactionSnapshot = coordinator.getAutoCompactionSnapshotForDataSource(dataSource); if (autoCompactionSnapshot == null) { return ServletResourceUtils.buildErrorResponseFrom(NotFound.exception("Unknown DataSource")); } - snapshots = ImmutableList.of(autoCompactionSnapshot); + snapshots = List.of(autoCompactionSnapshot); } return Response.ok(new CompactionStatusResponse(snapshots)).build(); } @@ -129,6 +112,7 @@ public Response getCompactionSnapshotForDataSource( @POST @Path("/simulate") @Consumes(MediaType.APPLICATION_JSON) + @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(StateResourceFilter.class) public Response simulateWithClusterConfigUpdate( ClusterCompactionConfig updatePayload @@ -138,37 +122,4 @@ public Response simulateWithClusterConfigUpdate( coordinator.simulateRunWithConfigUpdate(updatePayload) ).build(); } - - private Response buildResponse(ListenableFuture future) - { - try { - return Response.ok(FutureUtils.getUnchecked(future, true)).build(); - } - catch (Exception e) { - if (e.getCause() instanceof HttpResponseException) { - final HttpResponseException cause = (HttpResponseException) e.getCause(); - return Response.status(cause.getResponse().getStatus().getCode()) - .entity(cause.getResponse().getContent()) - .build(); - } else { - return ServletResourceUtils.buildErrorResponseFrom( - InternalServerError.exception(e.getMessage()) - ); - } - } - } - - /** - * Check if compaction supervisors are enabled on the Overlord. - */ - private boolean isCompactionSupervisorEnabled() - { - try { - return FutureUtils.getUnchecked(overlordClient.isCompactionSupervisorEnabled(), true); - } - catch (Exception e) { - // Overlord is probably on an older version, assume that compaction supervisor is not enabled - return false; - } - } } diff --git a/server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java b/server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java index 58f782587891..04465b677bf8 100644 --- a/server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java +++ b/server/src/main/java/org/apache/druid/server/http/ServletResourceUtils.java @@ -19,18 +19,24 @@ package org.apache.druid.server.http; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import org.apache.druid.error.DruidException; import org.apache.druid.error.ErrorResponse; +import org.apache.druid.error.InternalServerError; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import javax.annotation.Nullable; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import java.util.Map; +import java.util.function.Supplier; public class ServletResourceUtils { + private static final Logger log = new Logger(ServletResourceUtils.class); + /** * Sanitize the exception as a map of "error" to information about the exception. * @@ -61,4 +67,26 @@ public static Response buildErrorResponseFrom(DruidException e) .entity(new ErrorResponse(e)) .build(); } + + public static Response buildUpdateResponse(Supplier updateOperation) + { + return buildReadResponse(() -> Map.of("success", updateOperation.get())); + } + + public static Response buildReadResponse(Supplier readOperation) + { + try { + return Response.ok(readOperation.get()).build(); + } + catch (DruidException e) { + log.error(e, "Error executing HTTP request"); + return ServletResourceUtils.buildErrorResponseFrom(e); + } + catch (Exception e) { + log.error(e, "Error executing HTTP request"); + return ServletResourceUtils.buildErrorResponseFrom( + InternalServerError.exception(Throwables.getRootCause(e), "Unknown error occurred") + ); + } + } } diff --git a/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java b/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java index 28543b1a5c86..741a5f1ae08f 100644 --- a/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java +++ b/server/src/main/java/org/apache/druid/server/security/AuthorizationUtils.java @@ -426,6 +426,47 @@ public static Map> filterAuthorizedRes return filteredResources; } + /** + * Filters the given datasource-related resources on the basis of datasource + * permissions. + * + * @return List of resources to which the user has access, based on whether + * the user has access to the underlying datasource or not. + */ + public static List filterByAuthorizedDatasources( + final HttpServletRequest request, + List resources, + Function getDatasource, + AuthorizerMapper authorizerMapper + ) + { + final Function> raGenerator = + entry -> List.of(createDatasourceResourceAction(getDatasource.apply(entry), request)); + + return Lists.newArrayList( + AuthorizationUtils.filterAuthorizedResources( + request, + resources, + raGenerator, + authorizerMapper + ) + ); + } + + private static ResourceAction createDatasourceResourceAction( + String dataSource, + HttpServletRequest request + ) + { + switch (request.getMethod()) { + case "GET": + case "HEAD": + return AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(dataSource); + default: + return AuthorizationUtils.DATASOURCE_WRITE_RA_GENERATOR.apply(dataSource); + } + } + /** * This method constructs a 'superuser' set of permissions composed of {@link Action#READ} and {@link Action#WRITE} * permissions for all known {@link ResourceType#knownTypes()} for any {@link Authorizer} implementation which is diff --git a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java index 3dde6dda149b..1cf7f86f352c 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/CoordinatorClientImplTest.java @@ -19,6 +19,7 @@ package org.apache.druid.client.coordinator; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -39,8 +40,10 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.PruneLoadSpec; import org.apache.druid.timeline.partition.NumberedShardSpec; @@ -56,7 +59,9 @@ import javax.ws.rs.core.MediaType; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutionException; public class CoordinatorClientImplTest { @@ -374,4 +379,48 @@ public void test_fetchServerViewSegments() throws Exception coordinatorClient.fetchServerViewSegments("xyz", intervals) ); } + + @Test + public void test_getCompactionSnapshots_nullDataSource() + throws JsonProcessingException, ExecutionException, InterruptedException + { + final List compactionSnapshots = List.of( + AutoCompactionSnapshot.builder("ds1") + .withStatus(AutoCompactionSnapshot.ScheduleStatus.RUNNING) + .build(), + AutoCompactionSnapshot.builder("ds2") + .withStatus(AutoCompactionSnapshot.ScheduleStatus.NOT_ENABLED) + .build() + ); + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/compaction/status"), + HttpResponseStatus.OK, + Map.of(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(new CompactionStatusResponse(compactionSnapshots)) + ); + + Assert.assertEquals( + new CompactionStatusResponse(compactionSnapshots), + coordinatorClient.getCompactionSnapshots(null).get() + ); + } + + @Test + public void test_getCompactionSnapshots_nonNullDataSource() throws Exception + { + final List compactionSnapshots = List.of( + AutoCompactionSnapshot.builder("ds1").build() + ); + serviceClient.expectAndRespond( + new RequestBuilder(HttpMethod.GET, "/druid/coordinator/v1/compaction/status?dataSource=ds1"), + HttpResponseStatus.OK, + Map.of(), + DefaultObjectMapper.INSTANCE.writeValueAsBytes(new CompactionStatusResponse(compactionSnapshots)) + ); + + Assert.assertEquals( + new CompactionStatusResponse(compactionSnapshots), + coordinatorClient.getCompactionSnapshots("ds1").get() + ); + } } diff --git a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java index 58f5af58a3e7..ac01db424afa 100644 --- a/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java +++ b/server/src/test/java/org/apache/druid/client/coordinator/NoopCoordinatorClient.java @@ -25,9 +25,11 @@ import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.rpc.ServiceRetryPolicy; import org.apache.druid.segment.metadata.DataSourceInformation; +import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.List; import java.util.Set; @@ -81,4 +83,11 @@ public ListenableFuture> fetchDataSourcesWithUsedSegments() { throw new UnsupportedOperationException(); } + + @Override + public ListenableFuture getCompactionSnapshots(@Nullable String dataSource) + { + throw new UnsupportedOperationException(); + } + } diff --git a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java index 7ef15c931176..2d405e339eb2 100644 --- a/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java +++ b/server/src/test/java/org/apache/druid/rpc/indexing/OverlordClientImplTest.java @@ -48,9 +48,6 @@ import org.apache.druid.rpc.MockServiceClient; import org.apache.druid.rpc.RequestBuilder; import org.apache.druid.segment.TestDataSource; -import org.apache.druid.server.compaction.CompactionProgressResponse; -import org.apache.druid.server.compaction.CompactionStatusResponse; -import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.http.SegmentsToUpdateFilter; import org.apache.druid.timeline.DataSegment; import org.hamcrest.CoreMatchers; @@ -69,7 +66,6 @@ import javax.ws.rs.core.MediaType; import java.net.URI; import java.nio.charset.StandardCharsets; -import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; @@ -473,68 +469,6 @@ public void test_isCompactionSupervisorEnabled() Assert.assertFalse(overlordClient.isCompactionSupervisorEnabled().get()); } - @Test - public void test_getCompactionSnapshots_nullDataSource() - throws JsonProcessingException, ExecutionException, InterruptedException - { - final List compactionSnapshots = Arrays.asList( - AutoCompactionSnapshot.builder("ds1") - .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING) - .build(), - AutoCompactionSnapshot.builder("ds2") - .withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.NOT_ENABLED) - .build() - ); - serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/status"), - HttpResponseStatus.OK, - Collections.emptyMap(), - DefaultObjectMapper.INSTANCE.writeValueAsBytes(new CompactionStatusResponse(compactionSnapshots)) - ); - - Assert.assertEquals( - new CompactionStatusResponse(compactionSnapshots), - overlordClient.getCompactionSnapshots(null).get() - ); - } - - @Test - public void test_getCompactionSnapshots_nonNullDataSource() - throws JsonProcessingException, ExecutionException, InterruptedException - { - final List compactionSnapshots = Collections.singletonList( - AutoCompactionSnapshot.builder("ds1").build() - ); - serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/status?dataSource=ds1"), - HttpResponseStatus.OK, - Collections.emptyMap(), - DefaultObjectMapper.INSTANCE.writeValueAsBytes(new CompactionStatusResponse(compactionSnapshots)) - ); - - Assert.assertEquals( - new CompactionStatusResponse(compactionSnapshots), - overlordClient.getCompactionSnapshots("ds1").get() - ); - } - - @Test - public void test_getBytesAwaitingCompaction() - throws JsonProcessingException, ExecutionException, InterruptedException - { - serviceClient.expectAndRespond( - new RequestBuilder(HttpMethod.GET, "/druid/indexer/v1/compaction/progress?dataSource=ds1"), - HttpResponseStatus.OK, - Collections.emptyMap(), - DefaultObjectMapper.INSTANCE.writeValueAsBytes(new CompactionProgressResponse(100_000L)) - ); - - Assert.assertEquals( - new CompactionProgressResponse(100_000L), - overlordClient.getBytesAwaitingCompaction("ds1").get() - ); - } - @Test public void test_markSegmentAsUsed() throws Exception { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java index 4ba65fe2df8c..a6eb127f854c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java @@ -51,13 +51,13 @@ public void testAutoCompactionSnapshotBuilder() Assert.assertEquals(26, actual.getBytesAwaitingCompaction()); Assert.assertEquals(26, actual.getIntervalCountAwaitingCompaction()); Assert.assertEquals(26, actual.getSegmentCountAwaitingCompaction()); - Assert.assertEquals(AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, actual.getScheduleStatus()); + Assert.assertEquals(AutoCompactionSnapshot.ScheduleStatus.RUNNING, actual.getScheduleStatus()); Assert.assertEquals(expectedDataSource, actual.getDataSource()); Assert.assertEquals(expectedMessage, actual.getMessage()); AutoCompactionSnapshot expected = new AutoCompactionSnapshot( expectedDataSource, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, expectedMessage, 26, 26, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java index abe5dce051f5..fa6a4e9bdc56 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java @@ -175,7 +175,7 @@ private MetadataManager createMetadataManager(JacksonConfigManager configManager { return new MetadataManager( null, - new CoordinatorConfigManager(configManager, null, null), + new CoordinatorConfigManager(configManager, null, null, null), segmentsMetadataManager, null, metadataRuleManager, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java index c177ce73d98a..228229e7b737 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/CompactSegmentsTest.java @@ -81,7 +81,10 @@ import org.apache.druid.segment.incremental.OnheapIncrementalIndex; import org.apache.druid.segment.indexing.BatchIOConfig; import org.apache.druid.segment.transform.CompactionTransformSpec; +import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy; import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.compaction.FixedIntervalOrderPolicy; +import org.apache.druid.server.compaction.NewestSegmentFirstPolicy; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.CatalogDataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; @@ -143,7 +146,7 @@ public class CompactSegmentsTest private static final int TOTAL_INTERVAL_PER_DATASOURCE = 11; private static final int MAXIMUM_CAPACITY_WITH_AUTO_SCALE = 10; - @Parameterized.Parameters(name = "scenario: {0}, engine: {2}") + @Parameterized.Parameters(name = "partitionsSpec:{0}, engine:{2}") public static Collection constructorFeeder() { final MutableInt nextRangePartitionBoundary = new MutableInt(0); @@ -187,11 +190,12 @@ public static Collection constructorFeeder() private final PartitionsSpec partitionsSpec; private final BiFunction shardSpecFactory; private final CompactionEngine engine; + private CompactionCandidateSearchPolicy policy; private final List allSegments = new ArrayList<>(); private DataSourcesSnapshot dataSources; private CompactionStatusTracker statusTracker; - Map> datasourceToSegments = new HashMap<>(); + private final Map> datasourceToSegments = new HashMap<>(); public CompactSegmentsTest( PartitionsSpec partitionsSpec, @@ -224,6 +228,7 @@ public void setup() } dataSources = DataSourcesSnapshot.fromUsedSegments(allSegments); statusTracker = new CompactionStatusTracker(JSON_MAPPER); + policy = new NewestSegmentFirstPolicy(null); } private DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition) @@ -355,6 +360,13 @@ public String get() assertLastSegmentNotCompacted(compactSegments); } + @Test + public void testRun_withFixedIntervalOrderPolicy() + { + policy = new FixedIntervalOrderPolicy(List.of()); + testRun(); + } + @Test public void testMakeStats() { @@ -377,7 +389,7 @@ public void testMakeStats() for (int i = 0; i < 3; i++) { verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, 0, TOTAL_BYTE_PER_DATASOURCE, @@ -399,7 +411,7 @@ public void testMakeStats() for (int i = 1; i < 3; i++) { verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, 0, TOTAL_BYTE_PER_DATASOURCE, @@ -500,7 +512,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, dataSourceName, TOTAL_BYTE_PER_DATASOURCE - 120 - 40 * (compactionRunCount + 1), 120 + 40 * (compactionRunCount + 1), @@ -525,7 +537,7 @@ public void testMakeStatsForDataSourceWithCompactedIntervalBetweenNonCompactedIn ); verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, dataSourceName, 0, TOTAL_BYTE_PER_DATASOURCE, @@ -563,7 +575,7 @@ public void testMakeStatsWithDeactivatedDatasource() for (int i = 0; i < 3; i++) { verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, 0, TOTAL_BYTE_PER_DATASOURCE, @@ -587,7 +599,7 @@ public void testMakeStatsWithDeactivatedDatasource() for (int i = 1; i < 3; i++) { verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, 0, TOTAL_BYTE_PER_DATASOURCE, @@ -654,7 +666,7 @@ public void testMakeStatsForDataSourceWithSkipped() verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, dataSourceName, // Minus 120 bytes accounting for the three skipped segments' original size TOTAL_BYTE_PER_DATASOURCE - 120 - 40 * (compactionRunCount + 1), @@ -677,7 +689,7 @@ public void testMakeStatsForDataSourceWithSkipped() ); verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, dataSourceName, 0, // Minus 120 bytes accounting for the three skipped segments' original size @@ -1522,7 +1534,7 @@ public void testCompactWithoutMetricsSpecShouldSetPreserveExistingMetricsFalse() private void verifySnapshot( CompactSegments compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus scheduleStatus, + AutoCompactionSnapshot.ScheduleStatus scheduleStatus, String dataSourceName, long expectedByteCountAwaitingCompaction, long expectedByteCountCompressed, @@ -1569,7 +1581,7 @@ private void doCompactionAndAssertCompactSegmentStatistics(CompactSegments compa if (i != dataSourceIndex) { verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, TOTAL_BYTE_PER_DATASOURCE - 40L * (compactionRunCount + 1), 40L * (compactionRunCount + 1), @@ -1584,7 +1596,7 @@ private void doCompactionAndAssertCompactSegmentStatistics(CompactSegments compa } else { verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, TOTAL_BYTE_PER_DATASOURCE - 40L * (compactionRunCount + 1), 40L * (compactionRunCount + 1), @@ -1603,7 +1615,7 @@ private void doCompactionAndAssertCompactSegmentStatistics(CompactSegments compa // This verify that dataSource that ran out of slot has correct statistics verifySnapshot( compactSegments, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, DATA_SOURCE_PREFIX + i, TOTAL_BYTE_PER_DATASOURCE - 40L * compactionRunCount, 40L * compactionRunCount, @@ -1651,7 +1663,7 @@ private CoordinatorRunStats doCompactSegments( compactionConfigs, numCompactionTaskSlots == null ? null : 1.0, // 100% when numCompactionTaskSlots is not null numCompactionTaskSlots, - null, + policy, null, null ) @@ -1668,6 +1680,23 @@ private void assertCompactSegments( Supplier expectedVersionSupplier ) { + if (policy instanceof FixedIntervalOrderPolicy) { + // Priority expected intervals + final List eligibleCandidates = new ArrayList<>(); + datasourceToSegments.keySet().forEach( + ds -> eligibleCandidates.add( + new FixedIntervalOrderPolicy.Candidate(ds, expectedInterval) + ) + ); + // Make all other intervals eligible too + datasourceToSegments.keySet().forEach( + ds -> eligibleCandidates.add( + new FixedIntervalOrderPolicy.Candidate(ds, Intervals.ETERNITY) + ) + ); + policy = new FixedIntervalOrderPolicy(eligibleCandidates); + } + for (int i = 0; i < 3; i++) { final CoordinatorRunStats stats = doCompactSegments(compactSegments); Assert.assertEquals( diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java index da69d73ddb2c..3ec3958c4e74 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/KillCompactionConfigTest.java @@ -80,7 +80,8 @@ public void setup() coordinatorConfigManager = new CoordinatorConfigManager( mockJacksonConfigManager, mockConnector, - mockConnectorConfig + mockConnectorConfig, + null ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java index 938b6b09b1a0..ffceedb21f94 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBuilder.java @@ -476,7 +476,7 @@ private Environment( this.metadataManager = new MetadataManager( null, - new CoordinatorConfigManager(jacksonConfigManager, null, null), + new CoordinatorConfigManager(jacksonConfigManager, null, null, null), segmentManager, null, ruleManager, diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index 22b2d1e2c110..75aefc822891 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -38,7 +38,6 @@ import org.apache.druid.metadata.TestMetadataStorageConnector; import org.apache.druid.metadata.TestMetadataStorageTablesConfig; import org.apache.druid.segment.TestDataSource; -import org.apache.druid.server.coordinator.ClusterCompactionConfig; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfigAuditEntry; @@ -86,7 +85,7 @@ public void setup() Mockito.when(mockHttpServletRequest.getRemoteAddr()).thenReturn("123"); final AuditManager auditManager = new TestAuditManager(); configManager = TestCoordinatorConfigManager.create(auditManager); - resource = new CoordinatorCompactionConfigsResource(configManager, auditManager); + resource = new CoordinatorCompactionConfigsResource(configManager); configManager.delegate.start(); } @@ -110,32 +109,15 @@ public void testGetDefaultClusterConfig() Assert.assertEquals(CompactionEngine.NATIVE, defaultConfig.getEngine()); } - @Test - public void testUpdateClusterConfig() - { - Response response = resource.updateClusterCompactionConfig( - new ClusterCompactionConfig(0.5, 10, null, true, CompactionEngine.MSQ), - mockHttpServletRequest - ); - verifyStatus(Response.Status.OK, response); - - final DruidCompactionConfig updatedConfig = verifyAndGetPayload( - resource.getCompactionConfig(), - DruidCompactionConfig.class - ); - - Assert.assertNotNull(updatedConfig); - Assert.assertEquals(0.5, updatedConfig.getCompactionTaskSlotRatio(), DELTA); - Assert.assertEquals(10, updatedConfig.getMaxCompactionTaskSlots()); - Assert.assertTrue(updatedConfig.isUseSupervisors()); - Assert.assertEquals(CompactionEngine.MSQ, updatedConfig.getEngine()); - } - @Test public void testSetCompactionTaskLimit() { - final DruidCompactionConfig defaultConfig + resource.setCompactionTaskLimit(0.1, 100, mockHttpServletRequest); + + final DruidCompactionConfig oldConfig = verifyAndGetPayload(resource.getCompactionConfig(), DruidCompactionConfig.class); + Assert.assertEquals(100, oldConfig.getMaxCompactionTaskSlots()); + Assert.assertEquals(0.1, oldConfig.getCompactionTaskSlotRatio(), 1e-9); Response response = resource.setCompactionTaskLimit(0.5, 9, mockHttpServletRequest); verifyStatus(Response.Status.OK, response); @@ -147,8 +129,13 @@ public void testSetCompactionTaskLimit() Assert.assertEquals(0.5, updatedConfig.getCompactionTaskSlotRatio(), DELTA); Assert.assertEquals(9, updatedConfig.getMaxCompactionTaskSlots()); + // Verify that other cluster config fields are unchanged + Assert.assertEquals(oldConfig.isUseSupervisors(), updatedConfig.isUseSupervisors()); + Assert.assertEquals(oldConfig.getCompactionPolicy(), updatedConfig.getCompactionPolicy()); + Assert.assertEquals(oldConfig.getEngine(), updatedConfig.getEngine()); + // Verify that the other fields are unchanged - Assert.assertEquals(defaultConfig.getCompactionConfigs(), updatedConfig.getCompactionConfigs()); + Assert.assertEquals(oldConfig.getCompactionConfigs(), updatedConfig.getCompactionConfigs()); } @Test @@ -188,7 +175,7 @@ public void testAddDatasourceConfigWithMSQEngineIsInvalid() verifyStatus(Response.Status.BAD_REQUEST, response); Assert.assertTrue(response.getEntity() instanceof ErrorResponse); Assert.assertEquals( - "MSQ engine in compaction config only supported with supervisor-based compaction on the Overlord.", + "MSQ engine is supported only with supervisor-based compaction on the Overlord.", ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() ); } @@ -270,7 +257,7 @@ public void testUpdateIsRetriedIfFailureIsRetryable() ); Assert.assertEquals( - CoordinatorCompactionConfigsResource.MAX_UPDATE_RETRIES, + 5, configManager.numUpdateAttempts ); } @@ -339,7 +326,7 @@ public void testAddInvalidDatasourceConfigThrowsBadRequest() verifyStatus(Response.Status.BAD_REQUEST, response); Assert.assertTrue(response.getEntity() instanceof ErrorResponse); Assert.assertEquals( - "MSQ engine in compaction config only supported with supervisor-based compaction on the Overlord.", + "MSQ engine is supported only with supervisor-based compaction on the Overlord.", ((ErrorResponse) response.getEntity()).getUnderlyingException().getMessage() ); } @@ -454,6 +441,7 @@ public String getConfigTable() return new TestCoordinatorConfigManager( new JacksonConfigManager(configManager, OBJECT_MAPPER, auditManager), configManager, + auditManager, dbConnector, tablesConfig ); @@ -462,11 +450,12 @@ public String getConfigTable() TestCoordinatorConfigManager( JacksonConfigManager jackson, ConfigManager configManager, + AuditManager auditManager, TestDBConnector dbConnector, MetadataStorageTablesConfig tablesConfig ) { - super(jackson, dbConnector, tablesConfig); + super(jackson, dbConnector, tablesConfig, auditManager); this.delegate = configManager; } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java index 002fba124540..943f9a6ccf7d 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionResourceTest.java @@ -20,13 +20,8 @@ package org.apache.druid.server.http; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.error.DruidExceptionMatcher; import org.apache.druid.error.ErrorResponse; -import org.apache.druid.rpc.indexing.NoopOverlordClient; -import org.apache.druid.rpc.indexing.OverlordClient; -import org.apache.druid.server.compaction.CompactionStatistics; import org.apache.druid.server.compaction.CompactionStatusResponse; import org.apache.druid.server.coordinator.AutoCompactionSnapshot; import org.apache.druid.server.coordinator.DruidCoordinator; @@ -37,19 +32,18 @@ import org.junit.Before; import org.junit.Test; -import javax.annotation.Nullable; import javax.ws.rs.core.Response; import java.util.Collections; +import java.util.List; import java.util.Map; public class CoordinatorCompactionResourceTest { private DruidCoordinator mock; - private OverlordClient overlordClient; private final String dataSourceName = "datasource_1"; private final AutoCompactionSnapshot expectedSnapshot = new AutoCompactionSnapshot( dataSourceName, - AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING, + AutoCompactionSnapshot.ScheduleStatus.RUNNING, null, 1, 1, @@ -66,14 +60,6 @@ public class CoordinatorCompactionResourceTest public void setUp() { mock = EasyMock.createStrictMock(DruidCoordinator.class); - overlordClient = new NoopOverlordClient() - { - @Override - public ListenableFuture isCompactionSupervisorEnabled() - { - return Futures.immediateFuture(false); - } - }; } @After @@ -85,7 +71,7 @@ public void tearDown() @Test public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() { - Map expected = ImmutableMap.of( + Map expected = Map.of( dataSourceName, expectedSnapshot ); @@ -93,9 +79,9 @@ public void testGetCompactionSnapshotForDataSourceWithEmptyQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, overlordClient) + final Response response = new CoordinatorCompactionResource(mock) .getCompactionSnapshotForDataSource(""); - Assert.assertEquals(new CompactionStatusResponse(expected.values()), response.getEntity()); + Assert.assertEquals(new CompactionStatusResponse(List.of(expectedSnapshot)), response.getEntity()); Assert.assertEquals(200, response.getStatus()); } @@ -111,9 +97,9 @@ public void testGetCompactionSnapshotForDataSourceWithNullQueryParameter() EasyMock.expect(mock.getAutoCompactionSnapshot()).andReturn(expected).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, overlordClient) + final Response response = new CoordinatorCompactionResource(mock) .getCompactionSnapshotForDataSource(null); - Assert.assertEquals(new CompactionStatusResponse(expected.values()), response.getEntity()); + Assert.assertEquals(new CompactionStatusResponse(List.of(expectedSnapshot)), response.getEntity()); Assert.assertEquals(200, response.getStatus()); } @@ -126,7 +112,7 @@ public void testGetCompactionSnapshotForDataSourceWithValidQueryParameter() .andReturn(expectedSnapshot).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, overlordClient) + final Response response = new CoordinatorCompactionResource(mock) .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals( new CompactionStatusResponse(Collections.singletonList(expectedSnapshot)), @@ -144,7 +130,7 @@ public void testGetCompactionSnapshotForDataSourceWithInvalidQueryParameter() .andReturn(null).once(); EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, overlordClient) + final Response response = new CoordinatorCompactionResource(mock) .getCompactionSnapshotForDataSource(dataSourceName); Assert.assertEquals(404, response.getStatus()); } @@ -154,7 +140,7 @@ public void testGetProgressForNullDatasourceReturnsBadRequest() { EasyMock.replay(mock); - final Response response = new CoordinatorCompactionResource(mock, overlordClient) + final Response response = new CoordinatorCompactionResource(mock) .getCompactionProgress(null); Assert.assertEquals(Response.Status.BAD_REQUEST.getStatusCode(), response.getStatus()); @@ -166,38 +152,4 @@ public void testGetProgressForNullDatasourceReturnsBadRequest() DruidExceptionMatcher.invalidInput().expectMessageIs("No DataSource specified") ); } - - @Test - public void testGetSnapshotRedirectsToOverlordWhenSupervisorIsEnabled() - { - EasyMock.replay(mock); - - AutoCompactionSnapshot.Builder snapshotBuilder = AutoCompactionSnapshot.builder(dataSourceName); - snapshotBuilder.incrementCompactedStats(CompactionStatistics.create(100L, 10L, 1L)); - final AutoCompactionSnapshot snapshotFromOverlord = snapshotBuilder.build(); - - overlordClient = new NoopOverlordClient() { - @Override - public ListenableFuture isCompactionSupervisorEnabled() - { - return Futures.immediateFuture(true); - } - - @Override - public ListenableFuture getCompactionSnapshots(@Nullable String dataSource) - { - return Futures.immediateFuture( - new CompactionStatusResponse(Collections.singletonList(snapshotFromOverlord)) - ); - } - }; - - final Response response = new CoordinatorCompactionResource(mock, overlordClient) - .getCompactionSnapshotForDataSource(dataSourceName); - Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals( - new CompactionStatusResponse(Collections.singletonList(snapshotFromOverlord)), - response.getEntity() - ); - } } diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index e73d87d8bfe4..784be6f38da4 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -119,6 +119,7 @@ import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.appenderator.DummyForInjectionAppenderatorsManager; import org.apache.druid.server.compaction.CompactionStatusTracker; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.coordinator.DruidCompactionConfig; import org.apache.druid.server.http.RedirectFilter; @@ -220,6 +221,7 @@ public void configure(Binder binder) binder.bind(SegmentsMetadataManager.class) .toProvider(SegmentsMetadataManagerProvider.class) .in(ManageLifecycle.class); + binder.bind(CoordinatorConfigManager.class).in(LazySingleton.class); } JsonConfigProvider.bind(binder, "druid.coordinator.asOverlord", CoordinatorOverlordServiceConfig.class); @@ -251,7 +253,7 @@ public void configure(Binder binder) binder.bind(TaskLockbox.class).in(LazySingleton.class); binder.bind(TaskQueryTool.class).in(LazySingleton.class); binder.bind(IndexerMetadataStorageAdapter.class).in(LazySingleton.class); - binder.bind(CompactionScheduler.class).to(OverlordCompactionScheduler.class).in(LazySingleton.class); + binder.bind(CompactionScheduler.class).to(OverlordCompactionScheduler.class).in(ManageLifecycle.class); binder.bind(ScheduledBatchTaskManager.class).in(LazySingleton.class); binder.bind(SupervisorManager.class).in(LazySingleton.class);