Skip to content

Commit

Permalink
Run compaction as a supervisor on Overlord (apache#16768)
Browse files Browse the repository at this point in the history
Description
-----------
Auto-compaction currently poses several challenges as it:
1. may get stuck on a failing interval.
2. may get stuck on the latest interval if more data keeps coming into it.
3. always picks the latest interval regardless of the level of compaction in it.
4. may never pick a datasource if its intervals are not very recent.
5. requires setting an explicit period which does not cater to the changing needs of a Druid cluster.

This PR introduces various improvements to compaction scheduling to tackle the above problems.

Change Summary
--------------
1. Run compaction for a datasource as a supervisor of type `autocompact` on Overlord.
2. Make compaction policy extensible and configurable.
3. Track status of recently submitted compaction tasks and pass this info to policy.
4. Add `/simulate` API on both Coordinator and Overlord to run compaction simulations.
5. Redirect compaction status APIs to the Overlord when compaction supervisors are enabled.
  • Loading branch information
kfaraz authored Sep 2, 2024
1 parent 6eb42e8 commit fe3d589
Show file tree
Hide file tree
Showing 96 changed files with 5,736 additions and 2,138 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -24,9 +24,11 @@
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.server.coordinator.compact.CompactionSegmentIterator;
import org.apache.druid.server.coordinator.compact.CompactionSegmentSearchPolicy;
import org.apache.druid.server.coordinator.compact.NewestSegmentFirstPolicy;
import org.apache.druid.server.compaction.CompactionCandidateSearchPolicy;
import org.apache.druid.server.compaction.CompactionSegmentIterator;
import org.apache.druid.server.compaction.CompactionStatusTracker;
import org.apache.druid.server.compaction.NewestSegmentFirstPolicy;
import org.apache.druid.server.compaction.PriorityBasedCompactionSegmentIterator;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.partition.NumberedShardSpec;
Expand Down Expand Up @@ -61,7 +63,7 @@ public class NewestSegmentFirstPolicyBenchmark
{
private static final String DATA_SOURCE_PREFIX = "dataSource_";

private final CompactionSegmentSearchPolicy policy = new NewestSegmentFirstPolicy(new DefaultObjectMapper());
private final CompactionCandidateSearchPolicy policy = new NewestSegmentFirstPolicy(null);

@Param("100")
private int numDataSources;
Expand Down Expand Up @@ -132,7 +134,13 @@ public void setup()
@Benchmark
public void measureNewestSegmentFirstPolicy(Blackhole blackhole)
{
final CompactionSegmentIterator iterator = policy.createIterator(compactionConfigs, dataSources, Collections.emptyMap());
final CompactionSegmentIterator iterator = new PriorityBasedCompactionSegmentIterator(
policy,
compactionConfigs,
dataSources,
Collections.emptyMap(),
new CompactionStatusTracker(new DefaultObjectMapper())
);
for (int i = 0; i < numCompactionTaskSlots && iterator.hasNext(); i++) {
blackhole.consume(iterator.next());
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.druid.indexer.report.KillTaskReport;
import org.apache.druid.indexer.report.TaskReport;
import org.apache.druid.msq.exec.Controller;
import org.apache.druid.segment.TestDataSource;
import org.apache.druid.server.security.AuthorizerMapper;
import org.junit.Assert;
import org.junit.Test;
Expand All @@ -33,8 +34,6 @@

public class ControllerChatHandlerTest
{
private static final String DATASOURCE = "wiki";

@Test
public void testHttpGetLiveReports()
{
Expand All @@ -47,7 +46,8 @@ public void testHttpGetLiveReports()
.thenReturn(reportMap);

final AuthorizerMapper authorizerMapper = new AuthorizerMapper(null);
ControllerChatHandler chatHandler = new ControllerChatHandler(controller, DATASOURCE, authorizerMapper);
ControllerChatHandler chatHandler
= new ControllerChatHandler(controller, TestDataSource.WIKI, authorizerMapper);

HttpServletRequest httpRequest = Mockito.mock(HttpServletRequest.class);
Mockito.when(httpRequest.getAttribute(ArgumentMatchers.anyString()))
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
/*
* 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.guice;

import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.indexing.compact.CompactionSupervisorSpec;
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.server.coordinator.CompactionSupervisorConfig;

import java.util.List;

public class SupervisorModule implements DruidModule
{
@Override
public void configure(Binder binder)
{
JsonConfigProvider.bind(binder, "druid.supervisor", SupervisorStateManagerConfig.class);
JsonConfigProvider.bind(binder, "druid.supervisor.compaction", CompactionSupervisorConfig.class);
}

@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule(getClass().getSimpleName())
.registerSubtypes(
new NamedType(CompactionSupervisorSpec.class, CompactionSupervisorSpec.TYPE)
)
);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.compact;

import org.apache.druid.server.compaction.CompactionSimulateResult;
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.CompactionSupervisorConfig;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;

import java.util.Map;

/**
* Compaction scheduler that runs on the Overlord if {@link CompactionSupervisorConfig}
* is enabled.
* <p>
* Usage:
* <ul>
* <li>When an active {@link CompactionSupervisor} starts, it should register
* itself by calling {@link #startCompaction}.</li>
* <li>When a suspended {@link CompactionSupervisor} starts, it should stop
* compaction by calling {@link #stopCompaction}.</li>
* <li>When stopping, any {@link CompactionSupervisor} (active or suspended)
* should call {@link #stopCompaction}.</li>
* </ul>
*/
public interface CompactionScheduler
{
void start();

void stop();

boolean isRunning();

CompactionConfigValidationResult validateCompactionConfig(DataSourceCompactionConfig compactionConfig);

void startCompaction(String dataSourceName, DataSourceCompactionConfig compactionConfig);

void stopCompaction(String dataSourceName);

Map<String, AutoCompactionSnapshot> getAllCompactionSnapshots();

AutoCompactionSnapshot getCompactionSnapshot(String dataSource);

CompactionSimulateResult simulateRunWithConfigUpdate(ClusterCompactionConfig updateRequest);

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,156 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.druid.indexing.compact;

import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;

/**
* Supervisor for compaction of a single datasource.
*/
public class CompactionSupervisor implements Supervisor
{
private static final Logger log = new Logger(CompactionSupervisor.class);

private final String dataSource;
private final CompactionScheduler scheduler;
private final CompactionSupervisorSpec supervisorSpec;

public CompactionSupervisor(
CompactionSupervisorSpec supervisorSpec,
CompactionScheduler scheduler
)
{
this.supervisorSpec = supervisorSpec;
this.scheduler = scheduler;
this.dataSource = supervisorSpec.getSpec().getDataSource();
}

@Override
public void start()
{
if (supervisorSpec.isSuspended()) {
log.info("Suspending compaction for dataSource[%s].", dataSource);
scheduler.stopCompaction(dataSource);
} else {
log.info("Starting compaction for dataSource[%s].", dataSource);
scheduler.startCompaction(dataSource, supervisorSpec.getSpec());
}
}

@Override
public void stop(boolean stopGracefully)
{
log.info("Stopping compaction for dataSource[%s].", dataSource);
scheduler.stopCompaction(dataSource);
}

@Override
public SupervisorReport<AutoCompactionSnapshot> getStatus()
{
final AutoCompactionSnapshot snapshot;
if (supervisorSpec.isSuspended()) {
snapshot = AutoCompactionSnapshot.builder(dataSource)
.withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.NOT_ENABLED)
.build();
} else {
snapshot = scheduler.getCompactionSnapshot(dataSource);
}

return new SupervisorReport<>(supervisorSpec.getId(), DateTimes.nowUtc(), snapshot);
}

@Override
public SupervisorStateManager.State getState()
{
if (!scheduler.isRunning()) {
return State.SCHEDULER_STOPPED;
} else if (supervisorSpec.isSuspended()) {
return State.SUSPENDED;
} else {
return State.RUNNING;
}
}

// Un-implemented methods used only by streaming supervisors

@Override
public void reset(DataSourceMetadata dataSourceMetadata)
{
throw new UnsupportedOperationException("Resetting not supported for 'autocompact' supervisors.");
}

@Override
public void resetOffsets(DataSourceMetadata resetDataSourceMetadata)
{
throw new UnsupportedOperationException("Resetting offsets not supported for 'autocompact' supervisors.");
}

@Override
public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata)
{
throw new UnsupportedOperationException("Checkpointing not supported for 'autocompact' supervisors.");
}

@Override
public LagStats computeLagStats()
{
throw new UnsupportedOperationException("Lag stats not supported for 'autocompact' supervisors.");
}

@Override
public int getActiveTaskGroupsCount()
{
throw new UnsupportedOperationException("Task groups not supported for 'autocompact' supervisors.");
}

public enum State implements SupervisorStateManager.State
{
SCHEDULER_STOPPED(true),
RUNNING(true),
SUSPENDED(true),
UNHEALTHY(false);

private final boolean healthy;

State(boolean healthy)
{
this.healthy = healthy;
}

@Override
public boolean isFirstRunOnly()
{
return false;
}

@Override
public boolean isHealthy()
{
return healthy;
}
}
}
Loading

0 comments on commit fe3d589

Please sign in to comment.