From 7bcb35f8ac628c7e4e62dbc7675d1d26f5e704cf Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 12:17:27 +0530 Subject: [PATCH 01/30] Add new dynamic config --- .../coordinator/CoordinatorDynamicConfig.java | 34 ++++++++++++++++--- .../http/CoordinatorDynamicConfigTest.java | 6 ++-- 2 files changed, 33 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index d805bad5e01c..4a6c1e6ca736 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -70,6 +70,8 @@ public class CoordinatorDynamicConfig private final Map debugDimensions; private final Map validDebugDimensions; + private final Set turboLoadHistoricals; + /** * Stale pending segments belonging to the data sources in this list are not killed by {@code * KillStalePendingSegments}. In other words, segments in these data sources are "protected". @@ -118,7 +120,8 @@ public CoordinatorDynamicConfig( @JsonProperty("replicateAfterLoadTimeout") boolean replicateAfterLoadTimeout, @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment, @JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading, - @JsonProperty("debugDimensions") @Nullable Map debugDimensions + @JsonProperty("debugDimensions") @Nullable Map debugDimensions, + @JsonProperty("turboLoadHistoricals") @Nullable Set turboLoadHistoricals ) { this.markSegmentAsUnusedDelayMillis = @@ -162,6 +165,7 @@ public CoordinatorDynamicConfig( ); this.debugDimensions = debugDimensions; this.validDebugDimensions = validateDebugDimensions(debugDimensions); + this.turboLoadHistoricals = parseJsonStringOrArray(turboLoadHistoricals); } private Map validateDebugDimensions(Map debugDimensions) @@ -308,6 +312,12 @@ public boolean getReplicateAfterLoadTimeout() return replicateAfterLoadTimeout; } + @JsonProperty + public Set getTurboLoadHistoricals() + { + return turboLoadHistoricals; + } + @Override public String toString() { @@ -326,6 +336,7 @@ public String toString() ", decommissioningNodes=" + decommissioningNodes + ", pauseCoordination=" + pauseCoordination + ", replicateAfterLoadTimeout=" + replicateAfterLoadTimeout + + ", turboLoadHistoricals=" + turboLoadHistoricals + '}'; } @@ -359,6 +370,7 @@ public boolean equals(Object o) dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn) && Objects.equals(decommissioningNodes, that.decommissioningNodes) + && Objects.equals(turboLoadHistoricals, that.turboLoadHistoricals) && Objects.equals(debugDimensions, that.debugDimensions); } @@ -378,7 +390,8 @@ public int hashCode() dataSourcesToNotKillStalePendingSegmentsIn, decommissioningNodes, pauseCoordination, - debugDimensions + debugDimensions, + turboLoadHistoricals ); } @@ -430,6 +443,7 @@ public static class Builder private Boolean replicateAfterLoadTimeout; private Boolean useRoundRobinSegmentAssignment; private Boolean smartSegmentLoading; + private Set turboLoadHistoricals; public Builder() { @@ -452,7 +466,8 @@ public Builder( @JsonProperty("replicateAfterLoadTimeout") @Nullable Boolean replicateAfterLoadTimeout, @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment, @JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading, - @JsonProperty("debugDimensions") @Nullable Map debugDimensions + @JsonProperty("debugDimensions") @Nullable Map debugDimensions, + @JsonProperty("turboLoadHistoricals") @Nullable Set turboLoadHistoricals ) { this.markSegmentAsUnusedDelayMillis = markSegmentAsUnusedDelayMillis; @@ -471,6 +486,7 @@ public Builder( this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment; this.smartSegmentLoading = smartSegmentLoading; this.debugDimensions = debugDimensions; + this.turboLoadHistoricals = turboLoadHistoricals; } public Builder withMarkSegmentAsUnusedDelayMillis(long leadingTimeMillis) @@ -491,6 +507,12 @@ public Builder withSmartSegmentLoading(boolean smartSegmentLoading) return this; } + public Builder withTurboLoadHistoricals(Set turboLoadHistoricals) + { + this.turboLoadHistoricals = turboLoadHistoricals; + return this; + } + public Builder withReplicantLifetime(int replicantLifetime) { this.replicantLifetime = replicantLifetime; @@ -582,7 +604,8 @@ public CoordinatorDynamicConfig build() valueOrDefault(replicateAfterLoadTimeout, Defaults.REPLICATE_AFTER_LOAD_TIMEOUT), valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT), valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING), - debugDimensions + debugDimensions, + turboLoadHistoricals ); } @@ -612,7 +635,8 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) valueOrDefault(replicateAfterLoadTimeout, defaults.getReplicateAfterLoadTimeout()), valueOrDefault(useRoundRobinSegmentAssignment, defaults.isUseRoundRobinSegmentAssignment()), valueOrDefault(smartSegmentLoading, defaults.isSmartSegmentLoading()), - valueOrDefault(debugDimensions, defaults.getDebugDimensions()) + valueOrDefault(debugDimensions, defaults.getDebugDimensions()), + valueOrDefault(turboLoadHistoricals, defaults.getTurboLoadHistoricals()) ); } } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index eb7fb1992873..41b758376954 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -233,7 +233,8 @@ public void testConstructorWithNullsShouldKillUnusedSegmentsInAllDataSources() true, false, false, - null + null, + ImmutableSet.of("host1") ); Assert.assertTrue(config.getSpecificDataSourcesToKillUnusedSegmentsIn().isEmpty()); } @@ -257,7 +258,8 @@ public void testConstructorWithSpecificDataSourcesToKillShouldNotKillUnusedSegme true, false, false, - null + null, + ImmutableSet.of("host1") ); Assert.assertEquals(ImmutableSet.of("test1"), config.getSpecificDataSourcesToKillUnusedSegmentsIn()); } From c9a9fd56d09bae17c17d0c931a8f13356177d976 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 13:24:55 +0530 Subject: [PATCH 02/30] Inject dynamic config --- .../loading/SegmentLoadQueueManager.java | 6 +++++- .../coordinator/BalanceSegmentsProfiler.java | 5 ++++- .../coordinator/DruidCoordinatorTest.java | 4 ++-- .../coordinator/duty/BalanceSegmentsTest.java | 2 +- .../MarkEternityTombstonesAsUnusedTest.java | 2 +- .../MarkOvershadowedSegmentsAsUnusedTest.java | 3 ++- .../server/coordinator/duty/RunRulesTest.java | 5 ++++- .../duty/UnloadUnusedSegmentsTest.java | 6 +++++- .../rules/BroadcastDistributionRuleTest.java | 5 ++++- .../server/coordinator/rules/LoadRuleTest.java | 5 ++++- .../CoordinatorSimulationBaseTest.java | 18 ++++++++++++++++++ .../simulate/CoordinatorSimulationBuilder.java | 13 ++++++++++--- 12 files changed, 60 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java index 26226a7c0b49..8fc578793132 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java @@ -22,6 +22,7 @@ import com.google.inject.Inject; import org.apache.druid.client.ServerInventoryView; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; @@ -34,16 +35,19 @@ public class SegmentLoadQueueManager private static final Logger log = new Logger(SegmentLoadQueueManager.class); private final LoadQueueTaskMaster taskMaster; + private final CoordinatorConfigManager coordinatorConfigManager; private final ServerInventoryView serverInventoryView; @Inject public SegmentLoadQueueManager( ServerInventoryView serverInventoryView, - LoadQueueTaskMaster taskMaster + LoadQueueTaskMaster taskMaster, + CoordinatorConfigManager coordinatorConfigManager ) { this.serverInventoryView = serverInventoryView; this.taskMaster = taskMaster; + this.coordinatorConfigManager = coordinatorConfigManager; } /** diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java index 505d27a55cef..fd3b475e1c2c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java @@ -35,6 +35,7 @@ import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; import org.apache.druid.server.coordinator.rules.PeriodLoadRule; import org.apache.druid.server.coordinator.rules.Rule; +import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -66,7 +67,9 @@ public class BalanceSegmentsProfiler @Before public void setUp() { - loadQueueManager = new SegmentLoadQueueManager(null, null); + CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); + + loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); druidServer1 = EasyMock.createMock(ImmutableDruidServer.class); druidServer2 = EasyMock.createMock(ImmutableDruidServer.class); emitter = EasyMock.createMock(ServiceEmitter.class); 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 d50e82e7baa0..2f5d499f58f1 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 @@ -159,7 +159,7 @@ public void setUp() throws Exception scheduledExecutorFactory, overlordClient, loadQueueTaskMaster, - new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster), + new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster, new CoordinatorConfigManager(configManager, null, null)), new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, new CoordinatorCustomDutyGroups(ImmutableSet.of()), @@ -680,7 +680,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception scheduledExecutorFactory, overlordClient, loadQueueTaskMaster, - new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster), + new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster, new CoordinatorConfigManager(configManager, null, null)), new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, groups, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java index e4f3c416411b..8855ffc09005 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java @@ -72,7 +72,7 @@ public class BalanceSegmentsTest @Before public void setUp() { - loadQueueManager = new SegmentLoadQueueManager(null, null); + loadQueueManager = new SegmentLoadQueueManager(null, null, null); // Create test segments for multiple datasources final DateTime start1 = DateTimes.of("2012-01-01"); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java index 37fbe1447834..a66af8edd760 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java @@ -472,7 +472,7 @@ private DruidCoordinatorRuntimeParams initializeServerAndGetParams(final Immutab CoordinatorDynamicConfig.builder().withMarkSegmentAsUnusedDelayMillis(0).build() ) .withBalancerStrategy(new RandomBalancerStrategy()) - .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null)) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, null)) .build(); return params; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java index a06f1d3c7322..4e7d6ee1df7a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java @@ -33,6 +33,7 @@ import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategy; import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; +import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; @@ -99,7 +100,7 @@ public void testRun(String serverType) CoordinatorDynamicConfig.builder().withMarkSegmentAsUnusedDelayMillis(0).build() ) .withBalancerStrategy(new RandomBalancerStrategy()) - .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null)) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager())) .build(); SegmentTimeline timeline = segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments() diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index f7e9ececc1ac..7768fb526939 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -35,6 +35,7 @@ import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.segment.IndexIO; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; @@ -50,6 +51,7 @@ import org.apache.druid.server.coordinator.rules.ForeverLoadRule; import org.apache.druid.server.coordinator.rules.IntervalDropRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; +import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; @@ -100,7 +102,8 @@ public void setUp() EmittingLogger.registerEmitter(emitter); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); ruleRunner = new RunRules((ds, set) -> set.size(), databaseRuleManager::getRulesWithDefault); - loadQueueManager = new SegmentLoadQueueManager(null, null); + CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); + loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); balancerExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java index 131750d9581b..cd0f188dcb0a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java @@ -29,6 +29,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -37,6 +38,7 @@ import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; +import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; @@ -84,7 +86,9 @@ public void setUp() brokerServer = EasyMock.createMock(ImmutableDruidServer.class); indexerServer = EasyMock.createMock(ImmutableDruidServer.class); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); - loadQueueManager = new SegmentLoadQueueManager(null, null); + CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); + + loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); DateTime start1 = DateTimes.of("2012-01-01"); DateTime start2 = DateTimes.of("2012-02-01"); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 9932af14adff..7d1be00697a5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -23,6 +23,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -32,6 +33,7 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; +import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; @@ -232,12 +234,13 @@ private DruidCoordinatorRuntimeParams makeParamsWithUsedSegments( DataSegment... usedSegments ) { + CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); return DruidCoordinatorRuntimeParams .builder() .withDruidCluster(druidCluster) .withUsedSegments(usedSegments) .withBalancerStrategy(new RandomBalancerStrategy()) - .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null)) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, coordinatorConfigManager)) .build(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 0dcd43dccc99..90ca95d19816 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -30,6 +30,7 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; @@ -42,6 +43,7 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; +import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; @@ -90,7 +92,8 @@ public void setUp() { exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d")); balancerStrategy = new CostBalancerStrategy(exec); - loadQueueManager = new SegmentLoadQueueManager(null, null); + CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); + loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); } @After diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index 1fd3d54dc456..56ef448da0ee 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -20,10 +20,12 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; +import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; @@ -32,6 +34,7 @@ import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.timeline.DataSegment; +import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -40,6 +43,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; /** * Base test for coordinator simulations. @@ -190,6 +194,20 @@ static Map filter(Dimension dimension, String value) return Collections.singletonMap(dimension.reportedName(), value); } + public static CoordinatorConfigManager createEmptyCoordinatorConfigManager() + { + JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); + EasyMock.expect( + configManager.watch( + EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), + EasyMock.anyObject(Class.class), + EasyMock.anyObject() + ) + ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); + EasyMock.replay(configManager); + return new CoordinatorConfigManager(configManager, null, null); + } + /** * Creates a historical. The {@code uniqueIdInTier} must be correctly specified * as it is used to identify the historical throughout the simulation. 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 ddc360aa0124..3523e29c1aa2 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 @@ -461,8 +461,6 @@ private Environment( coordinatorConfig.getHttpLoadQueuePeonConfig(), httpClient ); - this.loadQueueManager = - new SegmentLoadQueueManager(coordinatorInventoryView, loadQueueTaskMaster); JacksonConfigManager jacksonConfigManager = mockConfigManager(); setDynamicConfig(dynamicConfig); @@ -471,15 +469,24 @@ private Environment( mocks.add(jacksonConfigManager); mocks.add(lookupCoordinatorManager); + CoordinatorConfigManager coordinatorConfigManager = new CoordinatorConfigManager( + jacksonConfigManager, + null, + null + ); + this.metadataManager = new MetadataManager( null, - new CoordinatorConfigManager(jacksonConfigManager, null, null), + coordinatorConfigManager, segmentManager, null, ruleManager, null, null ); + + this.loadQueueManager = + new SegmentLoadQueueManager(coordinatorInventoryView, loadQueueTaskMaster, coordinatorConfigManager); } private void setUp() throws Exception From acb289f41b4d99768a4bf9161f3047a0cb65fe38 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 16:45:22 +0530 Subject: [PATCH 03/30] Add new API --- .../coordination/SegmentLoadDropHandler.java | 7 +- .../loading/HttpLoadQueuePeon.java | 26 +++- .../loading/LoadQueueTaskMaster.java | 8 +- .../http/SegmentChangeRequestPacket.java | 55 ++++++++ .../server/http/SegmentListerResource.java | 119 +++++++++++++++++- .../druid/server/http/SegmentLoadingMode.java | 26 ++++ .../SegmentLoadDropHandlerTest.java | 17 +-- .../loading/HttpLoadQueuePeonTest.java | 23 +++- .../CoordinatorSimulationBuilder.java | 21 ++-- .../org/apache/druid/cli/CliCoordinator.java | 6 +- 10 files changed, 271 insertions(+), 37 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java create mode 100644 server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 12462adab2f6..4dd17e67b2eb 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -33,6 +33,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; @@ -244,14 +245,14 @@ public Collection getSegmentsToDelete() return ImmutableList.copyOf(segmentsToDelete); } - public ListenableFuture> processBatch(List changeRequests) + public ListenableFuture> processBatch(List changeRequests, SegmentLoadingMode segmentLoadingMode) { boolean isAnyRequestDone = false; Map> statuses = Maps.newHashMapWithExpectedSize(changeRequests.size()); for (DataSegmentChangeRequest cr : changeRequests) { - AtomicReference status = processRequest(cr); + AtomicReference status = processRequest(cr, segmentLoadingMode); if (status.get().getState() != SegmentChangeStatus.State.PENDING) { isAnyRequestDone = true; } @@ -271,7 +272,7 @@ public ListenableFuture> processBatch(List processRequest(DataSegmentChangeRequest changeRequest) + private AtomicReference processRequest(DataSegmentChangeRequest changeRequest, SegmentLoadingMode segmentLoadingMode) { synchronized (requestStatusesLock) { AtomicReference status = requestStatuses.getIfPresent(changeRequest); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index cf1a239b59d7..7859e0564408 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -39,12 +39,15 @@ import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeStatus; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.server.http.SegmentChangeRequestPacket; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -75,7 +78,7 @@ */ public class HttpLoadQueuePeon implements LoadQueuePeon { - public static final TypeReference> REQUEST_ENTITY_TYPE_REF = + public static final TypeReference REQUEST_ENTITY_TYPE_REF = new TypeReference<>() {}; public static final TypeReference> RESPONSE_ENTITY_TYPE_REF = @@ -112,6 +115,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final HttpLoadQueuePeonConfig config; + private final String serverName; private final ObjectMapper jsonMapper; private final HttpClient httpClient; private final URL changeRequestURL; @@ -119,16 +123,19 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final AtomicBoolean mainLoopInProgress = new AtomicBoolean(false); private final ExecutorService callBackExecutor; + private final CoordinatorConfigManager coordinatorConfigManager; private final ObjectWriter requestBodyWriter; public HttpLoadQueuePeon( String baseUrl, + String serverName, ObjectMapper jsonMapper, HttpClient httpClient, HttpLoadQueuePeonConfig config, ScheduledExecutorService processingExecutor, - ExecutorService callBackExecutor + ExecutorService callBackExecutor, + CoordinatorConfigManager coordinatorConfigManager ) { this.jsonMapper = jsonMapper; @@ -137,13 +144,15 @@ public HttpLoadQueuePeon( this.config = config; this.processingExecutor = processingExecutor; this.callBackExecutor = callBackExecutor; + this.coordinatorConfigManager = coordinatorConfigManager; this.serverId = baseUrl; + this.serverName = serverName; try { this.changeRequestURL = new URL( new URL(baseUrl), StringUtils.nonStrictFormat( - "druid-internal/v1/segments/changeRequests?timeout=%d", + "druid-internal/v1/segments/changeRequestsV2?timeout=%d", config.getHostTimeout().getMillis() ) ); @@ -200,8 +209,15 @@ private void doSegmentManagement() return; } + Set turboLoadHistoricals = coordinatorConfigManager.getCurrentDynamicConfig().getTurboLoadHistoricals(); + SegmentLoadingMode loadingMode = turboLoadHistoricals.contains(serverName) ? + SegmentLoadingMode.TURBO : + SegmentLoadingMode.NORMAL; + + SegmentChangeRequestPacket segmentChangeRequestPacket = new SegmentChangeRequestPacket(newRequests, loadingMode); + try { - log.trace("Sending [%d] load/drop requests to Server[%s].", newRequests.size(), serverId); + log.trace("Sending [%d] load/drop requests to Server[%s] in loadingMode [%s].", newRequests.size(), serverId, loadingMode); final boolean hasLoadRequests = newRequests.stream().anyMatch(r -> r instanceof SegmentChangeRequestLoad); if (hasLoadRequests && !loadingRateTracker.isLoadingBatch()) { loadingRateTracker.markBatchLoadingStarted(); @@ -212,7 +228,7 @@ private void doSegmentManagement() new Request(HttpMethod.POST, changeRequestURL) .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON) .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON) - .setContent(requestBodyWriter.writeValueAsBytes(newRequests)), + .setContent(requestBodyWriter.writeValueAsBytes(segmentChangeRequestPacket)), responseHandler, new Duration(config.getHostTimeout().getMillis() + 5000) ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java index 647c9e0c4132..ad0be6a2253c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java @@ -25,6 +25,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import java.util.List; @@ -47,6 +48,7 @@ public class LoadQueueTaskMaster private final ExecutorService callbackExec; private final HttpLoadQueuePeonConfig config; private final HttpClient httpClient; + private final CoordinatorConfigManager coordinatorConfigManager; @GuardedBy("this") private final AtomicBoolean isLeader = new AtomicBoolean(false); @@ -58,7 +60,8 @@ public LoadQueueTaskMaster( ScheduledExecutorService peonExec, ExecutorService callbackExec, HttpLoadQueuePeonConfig config, - HttpClient httpClient + HttpClient httpClient, + CoordinatorConfigManager coordinatorConfigManager ) { this.jsonMapper = jsonMapper; @@ -66,11 +69,12 @@ public LoadQueueTaskMaster( this.callbackExec = callbackExec; this.config = config; this.httpClient = httpClient; + this.coordinatorConfigManager = coordinatorConfigManager; } private LoadQueuePeon createPeon(ImmutableDruidServer server) { - return new HttpLoadQueuePeon(server.getURL(), jsonMapper, httpClient, config, peonExec, callbackExec); + return new HttpLoadQueuePeon(server.getURL(), server.getName(), jsonMapper, httpClient, config, peonExec, callbackExec, coordinatorConfigManager); } public Map getAllPeons() diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java b/server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java new file mode 100644 index 000000000000..81174554e292 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java @@ -0,0 +1,55 @@ +/* + * 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.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.server.coordination.DataSegmentChangeRequest; + +import java.util.List; + +public class SegmentChangeRequestPacket // TODO: change name? +{ + private final List changeRequestList; + + private final SegmentLoadingMode segmentLoadingMode; + + @JsonCreator + public SegmentChangeRequestPacket( + @JsonProperty("changeRequestList") List changeRequestList, + @JsonProperty("segmentLoadingMode") SegmentLoadingMode segmentLoadingMode + ) + { + this.changeRequestList = changeRequestList; + this.segmentLoadingMode = segmentLoadingMode; + } + + @JsonProperty("changeRequestList") + public List getChangeRequestList() + { + return changeRequestList; + } + + @JsonProperty("segmentLoadingMode") + public SegmentLoadingMode getSegmentLoadingMode() + { + return segmentLoadingMode; + } +} diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index 7c4392daf4e2..54b1fd0c4cac 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -215,23 +215,131 @@ public void onFailure(Throwable th) return null; } + /** + * Deprecated. + * + * @see SegmentListerResource#applyDataSegmentChangeRequests(long, SegmentChangeRequestPacket, HttpServletRequest) + */ + @Deprecated + @POST + @Path("/changeRequests") + @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + public void applyDataSegmentChangeRequests( + @QueryParam("timeout") long timeout, + List changeRequestList, + @Context final HttpServletRequest req + ) throws IOException + { + if (loadDropRequestHandler == null) { + sendErrorResponse(req, HttpServletResponse.SC_NOT_FOUND, "load/drop handler is not available."); + return; + } + + if (timeout <= 0) { + sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "timeout must be positive."); + return; + } + + if (changeRequestList == null || changeRequestList.isEmpty()) { + sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "No change requests provided."); + return; + } + + final ResponseContext context = createContext(req.getHeader("Accept")); + final ListenableFuture> future = + loadDropRequestHandler.processBatch(changeRequestList, SegmentLoadingMode.NORMAL); + + final AsyncContext asyncContext = req.startAsync(); + + asyncContext.addListener( + new AsyncListener() + { + @Override + public void onComplete(AsyncEvent event) + { + } + + @Override + public void onTimeout(AsyncEvent event) + { + + // HTTP 204 NO_CONTENT is sent to the client. + future.cancel(true); + event.getAsyncContext().complete(); + } + + @Override + public void onError(AsyncEvent event) + { + } + + @Override + public void onStartAsync(AsyncEvent event) + { + } + } + ); + + Futures.addCallback( + future, + new FutureCallback<>() + { + @Override + public void onSuccess(List result) + { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + response.setStatus(HttpServletResponse.SC_OK); + context.inputMapper.writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) + .writeValue(asyncContext.getResponse().getOutputStream(), result); + asyncContext.complete(); + } + catch (Exception ex) { + log.debug(ex, "Request timed out or closed already."); + } + } + + @Override + public void onFailure(Throwable th) + { + try { + HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); + if (th instanceof IllegalArgumentException) { + response.sendError(HttpServletResponse.SC_BAD_REQUEST, th.getMessage()); + } else { + response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, th.getMessage()); + } + asyncContext.complete(); + } + catch (Exception ex) { + log.debug(ex, "Request timed out or closed already."); + } + } + }, + MoreExecutors.directExecutor() + ); + + asyncContext.setTimeout(timeout); + } + /** * This endpoint is used by HttpLoadQueuePeon to assign segment load/drop requests batch. This endpoint makes the * client wait till one of the following events occur. Note that this is implemented using async IO so no jetty * threads are held while in wait. - * + *
* (1) Given timeout elapses. * (2) Some load/drop request completed. - * + *
* It returns a map of "load/drop request -> SUCCESS/FAILED/PENDING status" for each request in the batch. */ @POST - @Path("/changeRequests") + @Path("/changeRequestsV2") @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public void applyDataSegmentChangeRequests( @QueryParam("timeout") long timeout, - List changeRequestList, + SegmentChangeRequestPacket segmentChangeRequestPacket, @Context final HttpServletRequest req ) throws IOException { @@ -244,6 +352,7 @@ public void applyDataSegmentChangeRequests( sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "timeout must be positive."); return; } + List changeRequestList = segmentChangeRequestPacket.getChangeRequestList(); if (changeRequestList == null || changeRequestList.isEmpty()) { sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "No change requests provided."); @@ -252,7 +361,7 @@ public void applyDataSegmentChangeRequests( final ResponseContext context = createContext(req.getHeader("Accept")); final ListenableFuture> future = - loadDropRequestHandler.processBatch(changeRequestList); + loadDropRequestHandler.processBatch(changeRequestList, segmentChangeRequestPacket.getSegmentLoadingMode()); final AsyncContext asyncContext = req.startAsync(); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java new file mode 100644 index 000000000000..a6019ce5a94e --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java @@ -0,0 +1,26 @@ +/* + * 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.http; + +public enum SegmentLoadingMode // TODO: change name? +{ + NORMAL, + TURBO +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index cd2fe2dbd63e..d953fc304b9f 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.SegmentChangeStatus.State; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -230,7 +231,7 @@ public void testProcessBatch() throws Exception new SegmentChangeRequestDrop(segment2) ); - ListenableFuture> future = handler.processBatch(batch); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); Map expectedStatusMap = new HashMap<>(); expectedStatusMap.put(batch.get(0), SegmentChangeStatus.PENDING); @@ -244,7 +245,7 @@ public void testProcessBatch() throws Exception runnable.run(); } - result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1))).get(); + result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadingMode.NORMAL).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); @@ -271,7 +272,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - ListenableFuture> future = handler.processBatch(batch); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); @@ -280,7 +281,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState()); Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); - future = handler.processBatch(batch); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -343,7 +344,7 @@ public int getDropSegmentDelayMillis() List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); // Request 1: Load the segment - ListenableFuture> future = handler.processBatch(batch); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -354,7 +355,7 @@ public int getDropSegmentDelayMillis() // Request 2: Drop the segment batch = ImmutableList.of(new SegmentChangeRequestDrop(segment1)); - future = handler.processBatch(batch); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -372,7 +373,7 @@ public int getDropSegmentDelayMillis() // Request 3: Reload the segment batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = handler.processBatch(batch); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -389,7 +390,7 @@ public int getDropSegmentDelayMillis() // Request 4: Try to reload the segment - segment is loaded and announced again batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = handler.processBatch(batch); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 1dcf384d2e0d..19b924d76de5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.http.client.HttpClient; @@ -33,11 +34,15 @@ import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeStatus; +import org.apache.druid.server.coordinator.CoordinatorConfigManager; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.apache.druid.server.http.SegmentChangeRequestPacket; import org.apache.druid.timeline.DataSegment; +import org.easymock.EasyMock; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -56,6 +61,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -76,8 +82,18 @@ public class HttpLoadQueuePeonTest public void setUp() { httpClient = new TestHttpClient(); + JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); + EasyMock.expect( + configManager.watch( + EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), + EasyMock.anyObject(Class.class), + EasyMock.anyObject() + ) + ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); + EasyMock.replay(configManager); httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", + "dummy:4000", MAPPER, httpClient, new HttpLoadQueuePeonConfig(null, null, 10), @@ -86,7 +102,8 @@ public void setUp() httpClient.processingExecutor, true ), - httpClient.callbackExecutor + httpClient.callbackExecutor, + new CoordinatorConfigManager(configManager, null, null) ); httpLoadQueuePeon.start(); } @@ -348,11 +365,13 @@ public ListenableFuture go( httpResponse.setContent(ChannelBuffers.buffer(0)); httpResponseHandler.handleResponse(httpResponse, null); try { - List changeRequests = MAPPER.readValue( + SegmentChangeRequestPacket packet = MAPPER.readValue( request.getContent().array(), HttpLoadQueuePeon.REQUEST_ENTITY_TYPE_REF ); + List changeRequests = packet.getChangeRequestList(); + List statuses = new ArrayList<>(changeRequests.size()); for (DataSegmentChangeRequest cr : changeRequests) { cr.go(this, 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 3523e29c1aa2..3a007c92269b 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 @@ -454,27 +454,28 @@ private Environment( createBalancerStrategy(balancerStrategy), new HttpLoadQueuePeonConfig(null, null, null) ); + + JacksonConfigManager jacksonConfigManager = mockConfigManager(); + setDynamicConfig(dynamicConfig); + CoordinatorConfigManager coordinatorConfigManager = new CoordinatorConfigManager( + jacksonConfigManager, + null, + null + ); + this.loadQueueTaskMaster = new LoadQueueTaskMaster( OBJECT_MAPPER, executorFactory.create(1, ExecutorFactory.LOAD_QUEUE_EXECUTOR), executorFactory.create(1, ExecutorFactory.LOAD_CALLBACK_EXECUTOR), coordinatorConfig.getHttpLoadQueuePeonConfig(), - httpClient + httpClient, + coordinatorConfigManager ); - JacksonConfigManager jacksonConfigManager = mockConfigManager(); - setDynamicConfig(dynamicConfig); - this.lookupCoordinatorManager = EasyMock.createNiceMock(LookupCoordinatorManager.class); mocks.add(jacksonConfigManager); mocks.add(lookupCoordinatorManager); - CoordinatorConfigManager coordinatorConfigManager = new CoordinatorConfigManager( - jacksonConfigManager, - null, - null - ); - this.metadataManager = new MetadataManager( null, coordinatorConfigManager, diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index b1fc5f634047..8fb5b9b1e43d 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -306,7 +306,8 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( ScheduledExecutorFactory factory, DruidCoordinatorConfig config, @EscalatedGlobal HttpClient httpClient, - Lifecycle lifecycle + Lifecycle lifecycle, + CoordinatorConfigManager coordinatorConfigManager ) { final ExecutorService callBackExec = Execs.singleThreaded("LoadQueuePeon-callbackexec--%d"); @@ -316,7 +317,8 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( factory.create(1, "Master-PeonExec--%d"), callBackExec, config.getHttpLoadQueuePeonConfig(), - httpClient + httpClient, + coordinatorConfigManager ); } } From 386e3895ceb478f0ef491422f01b6ce3591ed062 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 18:14:40 +0530 Subject: [PATCH 04/30] Add new loading pool --- .../DataSegmentChangeHandler.java | 1 + .../coordination/SegmentBootstrapper.java | 5 +- .../coordination/SegmentLoadDropHandler.java | 75 ++++++++++++++----- .../SegmentBootstrapperCacheTest.java | 7 +- .../SegmentLoadDropHandlerTest.java | 3 +- 5 files changed, 67 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeHandler.java b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeHandler.java index cd2a8c3740f2..70758cd63abd 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/DataSegmentChangeHandler.java @@ -28,5 +28,6 @@ public interface DataSegmentChangeHandler { void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback); + void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback); } diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java index 7eec82e80b1c..14836e838237 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java @@ -39,6 +39,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; @@ -207,11 +208,11 @@ private void loadSegmentsOnStartup() throws IOException try { segmentManager.loadSegmentOnBootstrap( segment, - () -> loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false) + () -> loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false, SegmentLoadingMode.NORMAL) ); } catch (Exception e) { - loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false); + loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false, SegmentLoadingMode.NORMAL); throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); } try { diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 4dd17e67b2eb..ff1b31d3d7d9 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -65,6 +65,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final DataSegmentAnnouncer announcer; private final SegmentManager segmentManager; private final ScheduledExecutorService exec; + private final ScheduledExecutorService turboExec; private final ConcurrentSkipListSet segmentsToDelete; @@ -92,6 +93,10 @@ public SegmentLoadDropHandler( Executors.newScheduledThreadPool( config.getNumLoadingThreads(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") + ), + Executors.newScheduledThreadPool( + config.getNumBootstrapThreads(), + Execs.makeThreadFactory("TurboDataSegmentChangeHandler-%s") ) ); } @@ -101,13 +106,15 @@ public SegmentLoadDropHandler( SegmentLoaderConfig config, DataSegmentAnnouncer announcer, SegmentManager segmentManager, - ScheduledExecutorService exec + ScheduledExecutorService exec, + ScheduledExecutorService turboExec ) { this.config = config; this.announcer = announcer; this.segmentManager = segmentManager; this.exec = exec; + this.turboExec = turboExec; this.segmentsToDelete = new ConcurrentSkipListSet<>(); requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); @@ -126,6 +133,10 @@ public Map getRowCountDistributionPerDataso @Override public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) { + addSegment(segment, callback, SegmentLoadingMode.NORMAL); + } + + public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback, SegmentLoadingMode loadingMode) { SegmentChangeStatus result = null; try { log.info("Loading segment[%s]", segment.getId()); @@ -150,7 +161,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m segmentManager.loadSegment(segment); } catch (Exception e) { - removeSegment(segment, DataSegmentChangeCallback.NOOP, false); + removeSegment(segment, DataSegmentChangeCallback.NOOP, false, loadingMode); throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); } try { @@ -180,14 +191,15 @@ each time when addSegment() is called, it has to wait for the lock in order to m @Override public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) { - removeSegment(segment, callback, true); + removeSegment(segment, callback, true, SegmentLoadingMode.NORMAL); } @VisibleForTesting void removeSegment( final DataSegment segment, @Nullable final DataSegmentChangeCallback callback, - final boolean scheduleDrop + final boolean scheduleDrop, + final SegmentLoadingMode segmentLoadingMode ) { SegmentChangeStatus result = null; @@ -215,11 +227,20 @@ void removeSegment( "Completely removing segment[%s] in [%,d]ms.", segment.getId(), config.getDropSegmentDelayMillis() ); - exec.schedule( - runnable, - config.getDropSegmentDelayMillis(), - TimeUnit.MILLISECONDS - ); + if (SegmentLoadingMode.TURBO.equals(segmentLoadingMode)) { + turboExec.schedule( + runnable, + config.getDropSegmentDelayMillis(), + TimeUnit.MILLISECONDS + ); + } else { + exec.schedule( + runnable, + config.getDropSegmentDelayMillis(), + TimeUnit.MILLISECONDS + ); + + } } else { runnable.run(); } @@ -283,25 +304,43 @@ private AtomicReference processRequest(DataSegmentChangeReq new DataSegmentChangeHandler() { @Override - public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) + public void addSegment( + DataSegment segment, + @Nullable DataSegmentChangeCallback callback + ) { requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); - exec.submit( - () -> SegmentLoadDropHandler.this.addSegment( - ((SegmentChangeRequestLoad) changeRequest).getSegment(), - () -> resolveWaitingFutures() - ) - ); + if (SegmentLoadingMode.TURBO.equals(segmentLoadingMode)) { + turboExec.submit( + () -> SegmentLoadDropHandler.this.addSegment( + ((SegmentChangeRequestLoad) changeRequest).getSegment(), + () -> resolveWaitingFutures(), + segmentLoadingMode + ) + ); + } else { + exec.submit( + () -> SegmentLoadDropHandler.this.addSegment( + ((SegmentChangeRequestLoad) changeRequest).getSegment(), + () -> resolveWaitingFutures(), + segmentLoadingMode + ) + ); + } } @Override - public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) + public void removeSegment( + DataSegment segment, + @Nullable DataSegmentChangeCallback callback + ) { requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); SegmentLoadDropHandler.this.removeSegment( ((SegmentChangeRequestDrop) changeRequest).getSegment(), () -> resolveWaitingFutures(), - true + true, + segmentLoadingMode ); } }, diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java index 187725317a21..194016b5164e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -219,12 +220,12 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException // Make sure adding segments beyond allowed size fails DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE); - loadDropHandler.addSegment(newSegment, null); + loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment)); // Clearing some segment should allow for new segments - loadDropHandler.removeSegment(expectedSegments.get(0), null, false); - loadDropHandler.addSegment(newSegment, null); + loadDropHandler.removeSegment(expectedSegments.get(0), null, false, SegmentLoadingMode.NORMAL); + loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); bootstrapper.stop(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index d953fc304b9f..8932d3c82a0f 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -421,7 +421,8 @@ private SegmentLoadDropHandler initSegmentLoadDropHandler( config, segmentAnnouncer, segmentManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]") + scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + scheduledExecutorFactory.create(5, "TurboSegmentLoadDropHandlerTest-[%d]") ); } } From 48e81dca710929da8f8796ed669305e86ae7079e Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 18:16:17 +0530 Subject: [PATCH 05/30] Revert "Inject dynamic config" This reverts commit c9a9fd56d09bae17c17d0c931a8f13356177d976. --- .../loading/SegmentLoadQueueManager.java | 6 +----- .../coordinator/BalanceSegmentsProfiler.java | 5 +---- .../coordinator/DruidCoordinatorTest.java | 4 ++-- .../coordinator/duty/BalanceSegmentsTest.java | 2 +- .../MarkEternityTombstonesAsUnusedTest.java | 2 +- .../MarkOvershadowedSegmentsAsUnusedTest.java | 3 +-- .../server/coordinator/duty/RunRulesTest.java | 5 +---- .../duty/UnloadUnusedSegmentsTest.java | 6 +----- .../rules/BroadcastDistributionRuleTest.java | 5 +---- .../server/coordinator/rules/LoadRuleTest.java | 5 +---- .../CoordinatorSimulationBaseTest.java | 18 ------------------ .../simulate/CoordinatorSimulationBuilder.java | 8 ++++---- 12 files changed, 15 insertions(+), 54 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java index 8fc578793132..26226a7c0b49 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/SegmentLoadQueueManager.java @@ -22,7 +22,6 @@ import com.google.inject.Inject; import org.apache.druid.client.ServerInventoryView; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; @@ -35,19 +34,16 @@ public class SegmentLoadQueueManager private static final Logger log = new Logger(SegmentLoadQueueManager.class); private final LoadQueueTaskMaster taskMaster; - private final CoordinatorConfigManager coordinatorConfigManager; private final ServerInventoryView serverInventoryView; @Inject public SegmentLoadQueueManager( ServerInventoryView serverInventoryView, - LoadQueueTaskMaster taskMaster, - CoordinatorConfigManager coordinatorConfigManager + LoadQueueTaskMaster taskMaster ) { this.serverInventoryView = serverInventoryView; this.taskMaster = taskMaster; - this.coordinatorConfigManager = coordinatorConfigManager; } /** diff --git a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java index fd3b475e1c2c..505d27a55cef 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/BalanceSegmentsProfiler.java @@ -35,7 +35,6 @@ import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; import org.apache.druid.server.coordinator.rules.PeriodLoadRule; import org.apache.druid.server.coordinator.rules.Rule; -import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; import org.easymock.EasyMock; @@ -67,9 +66,7 @@ public class BalanceSegmentsProfiler @Before public void setUp() { - CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); - - loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); + loadQueueManager = new SegmentLoadQueueManager(null, null); druidServer1 = EasyMock.createMock(ImmutableDruidServer.class); druidServer2 = EasyMock.createMock(ImmutableDruidServer.class); emitter = EasyMock.createMock(ServiceEmitter.class); 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 2f5d499f58f1..d50e82e7baa0 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 @@ -159,7 +159,7 @@ public void setUp() throws Exception scheduledExecutorFactory, overlordClient, loadQueueTaskMaster, - new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster, new CoordinatorConfigManager(configManager, null, null)), + new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster), new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, new CoordinatorCustomDutyGroups(ImmutableSet.of()), @@ -680,7 +680,7 @@ public void testCoordinatorCustomDutyGroupsRunAsExpected() throws Exception scheduledExecutorFactory, overlordClient, loadQueueTaskMaster, - new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster, new CoordinatorConfigManager(configManager, null, null)), + new SegmentLoadQueueManager(serverInventoryView, loadQueueTaskMaster), new LatchableServiceAnnouncer(leaderAnnouncerLatch, leaderUnannouncerLatch), druidNode, groups, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java index 8855ffc09005..e4f3c416411b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/BalanceSegmentsTest.java @@ -72,7 +72,7 @@ public class BalanceSegmentsTest @Before public void setUp() { - loadQueueManager = new SegmentLoadQueueManager(null, null, null); + loadQueueManager = new SegmentLoadQueueManager(null, null); // Create test segments for multiple datasources final DateTime start1 = DateTimes.of("2012-01-01"); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java index a66af8edd760..37fbe1447834 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkEternityTombstonesAsUnusedTest.java @@ -472,7 +472,7 @@ private DruidCoordinatorRuntimeParams initializeServerAndGetParams(final Immutab CoordinatorDynamicConfig.builder().withMarkSegmentAsUnusedDelayMillis(0).build() ) .withBalancerStrategy(new RandomBalancerStrategy()) - .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, null)) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null)) .build(); return params; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java index 4e7d6ee1df7a..a06f1d3c7322 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/MarkOvershadowedSegmentsAsUnusedTest.java @@ -33,7 +33,6 @@ import org.apache.druid.server.coordinator.balancer.RandomBalancerStrategy; import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; -import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.simulate.TestSegmentsMetadataManager; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; @@ -100,7 +99,7 @@ public void testRun(String serverType) CoordinatorDynamicConfig.builder().withMarkSegmentAsUnusedDelayMillis(0).build() ) .withBalancerStrategy(new RandomBalancerStrategy()) - .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager())) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null)) .build(); SegmentTimeline timeline = segmentsMetadataManager.getSnapshotOfDataSourcesWithAllUsedSegments() diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java index 7768fb526939..f7e9ececc1ac 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/RunRulesTest.java @@ -35,7 +35,6 @@ import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.segment.IndexIO; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; @@ -51,7 +50,6 @@ import org.apache.druid.server.coordinator.rules.ForeverLoadRule; import org.apache.druid.server.coordinator.rules.IntervalDropRule; import org.apache.druid.server.coordinator.rules.IntervalLoadRule; -import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; @@ -102,8 +100,7 @@ public void setUp() EmittingLogger.registerEmitter(emitter); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); ruleRunner = new RunRules((ds, set) -> set.size(), databaseRuleManager::getRulesWithDefault); - CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); - loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); + loadQueueManager = new SegmentLoadQueueManager(null, null); balancerExecutor = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "RunRulesTest-%d")); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java index cd0f188dcb0a..131750d9581b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/duty/UnloadUnusedSegmentsTest.java @@ -29,7 +29,6 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -38,7 +37,6 @@ import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; import org.apache.druid.server.coordinator.rules.ForeverLoadRule; -import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; @@ -86,9 +84,7 @@ public void setUp() brokerServer = EasyMock.createMock(ImmutableDruidServer.class); indexerServer = EasyMock.createMock(ImmutableDruidServer.class); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); - CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); - - loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); + loadQueueManager = new SegmentLoadQueueManager(null, null); DateTime start1 = DateTimes.of("2012-01-01"); DateTime start2 = DateTimes.of("2012-02-01"); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java index 7d1be00697a5..9932af14adff 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java @@ -23,7 +23,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -33,7 +32,6 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; -import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; @@ -234,13 +232,12 @@ private DruidCoordinatorRuntimeParams makeParamsWithUsedSegments( DataSegment... usedSegments ) { - CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); return DruidCoordinatorRuntimeParams .builder() .withDruidCluster(druidCluster) .withUsedSegments(usedSegments) .withBalancerStrategy(new RandomBalancerStrategy()) - .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null, coordinatorConfigManager)) + .withSegmentAssignerUsing(new SegmentLoadQueueManager(null, null)) .build(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java index 90ca95d19816..0dcd43dccc99 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/LoadRuleTest.java @@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.DruidCluster; @@ -43,7 +42,6 @@ import org.apache.druid.server.coordinator.loading.SegmentLoadQueueManager; import org.apache.druid.server.coordinator.loading.StrategicSegmentAssigner; import org.apache.druid.server.coordinator.loading.TestLoadQueuePeon; -import org.apache.druid.server.coordinator.simulate.CoordinatorSimulationBaseTest; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.timeline.DataSegment; @@ -92,8 +90,7 @@ public void setUp() { exec = MoreExecutors.listeningDecorator(Execs.multiThreaded(1, "LoadRuleTest-%d")); balancerStrategy = new CostBalancerStrategy(exec); - CoordinatorConfigManager coordinatorConfigManager = CoordinatorSimulationBaseTest.createEmptyCoordinatorConfigManager(); - loadQueueManager = new SegmentLoadQueueManager(null, null, coordinatorConfigManager); + loadQueueManager = new SegmentLoadQueueManager(null, null); } @After diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java index 56ef448da0ee..1fd3d54dc456 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/CoordinatorSimulationBaseTest.java @@ -20,12 +20,10 @@ package org.apache.druid.server.coordinator.simulate; import org.apache.druid.client.DruidServer; -import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.metrics.MetricsVerifier; import org.apache.druid.segment.TestDataSource; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.rules.ForeverBroadcastDistributionRule; @@ -34,7 +32,6 @@ import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.timeline.DataSegment; -import org.easymock.EasyMock; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -43,7 +40,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; /** * Base test for coordinator simulations. @@ -194,20 +190,6 @@ static Map filter(Dimension dimension, String value) return Collections.singletonMap(dimension.reportedName(), value); } - public static CoordinatorConfigManager createEmptyCoordinatorConfigManager() - { - JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); - EasyMock.expect( - configManager.watch( - EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), - EasyMock.anyObject(Class.class), - EasyMock.anyObject() - ) - ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); - EasyMock.replay(configManager); - return new CoordinatorConfigManager(configManager, null, null); - } - /** * Creates a historical. The {@code uniqueIdInTier} must be correctly specified * as it is used to identify the historical throughout the simulation. 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 3a007c92269b..053afa6466cf 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 @@ -472,22 +472,22 @@ private Environment( coordinatorConfigManager ); + this.loadQueueManager = + new SegmentLoadQueueManager(coordinatorInventoryView, loadQueueTaskMaster); + this.lookupCoordinatorManager = EasyMock.createNiceMock(LookupCoordinatorManager.class); mocks.add(jacksonConfigManager); mocks.add(lookupCoordinatorManager); this.metadataManager = new MetadataManager( null, - coordinatorConfigManager, + new CoordinatorConfigManager(jacksonConfigManager, null, null), segmentManager, null, ruleManager, null, null ); - - this.loadQueueManager = - new SegmentLoadQueueManager(coordinatorInventoryView, loadQueueTaskMaster, coordinatorConfigManager); } private void setUp() throws Exception From 270acdf84ed0a695e6122c6aa2c192601e578667 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 18:27:42 +0530 Subject: [PATCH 06/30] Cleanup --- .../server/coordination/SegmentLoadDropHandler.java | 3 ++- .../server/coordinator/loading/HttpLoadQueuePeon.java | 8 ++++---- ...Packet.java => HistoricalSegmentChangeRequest.java} | 4 ++-- .../druid/server/http/SegmentListerResource.java | 10 +++++----- .../apache/druid/server/http/SegmentLoadingMode.java | 2 +- .../coordinator/loading/HttpLoadQueuePeonTest.java | 6 +++--- 6 files changed, 17 insertions(+), 16 deletions(-) rename server/src/main/java/org/apache/druid/server/http/{SegmentChangeRequestPacket.java => HistoricalSegmentChangeRequest.java} (94%) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index ff1b31d3d7d9..6292c4ef46bc 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -136,7 +136,8 @@ public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback addSegment(segment, callback, SegmentLoadingMode.NORMAL); } - public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback, SegmentLoadingMode loadingMode) { + public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback, SegmentLoadingMode loadingMode) + { SegmentChangeStatus result = null; try { log.info("Loading segment[%s]", segment.getId()); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 7859e0564408..71f598038b2b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -46,7 +46,7 @@ import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; -import org.apache.druid.server.http.SegmentChangeRequestPacket; +import org.apache.druid.server.http.HistoricalSegmentChangeRequest; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -78,7 +78,7 @@ */ public class HttpLoadQueuePeon implements LoadQueuePeon { - public static final TypeReference REQUEST_ENTITY_TYPE_REF = + public static final TypeReference REQUEST_ENTITY_TYPE_REF = new TypeReference<>() {}; public static final TypeReference> RESPONSE_ENTITY_TYPE_REF = @@ -214,7 +214,7 @@ private void doSegmentManagement() SegmentLoadingMode.TURBO : SegmentLoadingMode.NORMAL; - SegmentChangeRequestPacket segmentChangeRequestPacket = new SegmentChangeRequestPacket(newRequests, loadingMode); + HistoricalSegmentChangeRequest request = new HistoricalSegmentChangeRequest(newRequests, loadingMode); try { log.trace("Sending [%d] load/drop requests to Server[%s] in loadingMode [%s].", newRequests.size(), serverId, loadingMode); @@ -228,7 +228,7 @@ private void doSegmentManagement() new Request(HttpMethod.POST, changeRequestURL) .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON) .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON) - .setContent(requestBodyWriter.writeValueAsBytes(segmentChangeRequestPacket)), + .setContent(requestBodyWriter.writeValueAsBytes(request)), responseHandler, new Duration(config.getHostTimeout().getMillis() + 5000) ); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java similarity index 94% rename from server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java rename to server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java index 81174554e292..d77ac153b77a 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentChangeRequestPacket.java +++ b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java @@ -25,14 +25,14 @@ import java.util.List; -public class SegmentChangeRequestPacket // TODO: change name? +public class HistoricalSegmentChangeRequest { private final List changeRequestList; private final SegmentLoadingMode segmentLoadingMode; @JsonCreator - public SegmentChangeRequestPacket( + public HistoricalSegmentChangeRequest( @JsonProperty("changeRequestList") List changeRequestList, @JsonProperty("segmentLoadingMode") SegmentLoadingMode segmentLoadingMode ) diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index 54b1fd0c4cac..a0a5148bd686 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -218,7 +218,7 @@ public void onFailure(Throwable th) /** * Deprecated. * - * @see SegmentListerResource#applyDataSegmentChangeRequests(long, SegmentChangeRequestPacket, HttpServletRequest) + * @see SegmentListerResource#applyDataSegmentChangeRequests(long, HistoricalSegmentChangeRequest, HttpServletRequest) */ @Deprecated @POST @@ -291,7 +291,7 @@ public void onSuccess(List result) try { HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); response.setStatus(HttpServletResponse.SC_OK); - context.inputMapper.writerWithType(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) + context.inputMapper.writerFor(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) .writeValue(asyncContext.getResponse().getOutputStream(), result); asyncContext.complete(); } @@ -339,7 +339,7 @@ public void onFailure(Throwable th) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public void applyDataSegmentChangeRequests( @QueryParam("timeout") long timeout, - SegmentChangeRequestPacket segmentChangeRequestPacket, + HistoricalSegmentChangeRequest historicalSegmentChangeRequest, @Context final HttpServletRequest req ) throws IOException { @@ -352,7 +352,7 @@ public void applyDataSegmentChangeRequests( sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "timeout must be positive."); return; } - List changeRequestList = segmentChangeRequestPacket.getChangeRequestList(); + List changeRequestList = historicalSegmentChangeRequest.getChangeRequestList(); if (changeRequestList == null || changeRequestList.isEmpty()) { sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "No change requests provided."); @@ -361,7 +361,7 @@ public void applyDataSegmentChangeRequests( final ResponseContext context = createContext(req.getHeader("Accept")); final ListenableFuture> future = - loadDropRequestHandler.processBatch(changeRequestList, segmentChangeRequestPacket.getSegmentLoadingMode()); + loadDropRequestHandler.processBatch(changeRequestList, historicalSegmentChangeRequest.getSegmentLoadingMode()); final AsyncContext asyncContext = req.startAsync(); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java index a6019ce5a94e..647c390394f5 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java @@ -19,7 +19,7 @@ package org.apache.druid.server.http; -public enum SegmentLoadingMode // TODO: change name? +public enum SegmentLoadingMode { NORMAL, TURBO diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 19b924d76de5..c923e071b31a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -40,7 +40,7 @@ import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; -import org.apache.druid.server.http.SegmentChangeRequestPacket; +import org.apache.druid.server.http.HistoricalSegmentChangeRequest; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.jboss.netty.buffer.ChannelBuffers; @@ -365,12 +365,12 @@ public ListenableFuture go( httpResponse.setContent(ChannelBuffers.buffer(0)); httpResponseHandler.handleResponse(httpResponse, null); try { - SegmentChangeRequestPacket packet = MAPPER.readValue( + HistoricalSegmentChangeRequest segmentChangeRequest = MAPPER.readValue( request.getContent().array(), HttpLoadQueuePeon.REQUEST_ENTITY_TYPE_REF ); - List changeRequests = packet.getChangeRequestList(); + List changeRequests = segmentChangeRequest.getChangeRequestList(); List statuses = new ArrayList<>(changeRequests.size()); for (DataSegmentChangeRequest cr : changeRequests) { From 85a1ae84c85a11ea57183de3d32ee45a9c20b71d Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 3 Mar 2025 20:33:42 +0530 Subject: [PATCH 07/30] Clean up --- .../coordination/SegmentBootstrapper.java | 5 +- .../coordination/SegmentLoadDropHandler.java | 79 +++++++++++-------- .../loading/HttpLoadQueuePeon.java | 10 +-- .../http/HistoricalSegmentChangeRequest.java | 7 +- .../server/http/SegmentListerResource.java | 4 +- .../druid/server/http/SegmentLoadingMode.java | 8 +- .../SegmentBootstrapperCacheTest.java | 7 +- .../SegmentLoadDropHandlerTest.java | 17 ++-- 8 files changed, 73 insertions(+), 64 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java index 14836e838237..7eec82e80b1c 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentBootstrapper.java @@ -39,7 +39,6 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; -import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; @@ -208,11 +207,11 @@ private void loadSegmentsOnStartup() throws IOException try { segmentManager.loadSegmentOnBootstrap( segment, - () -> loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false, SegmentLoadingMode.NORMAL) + () -> loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false) ); } catch (Exception e) { - loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false, SegmentLoadingMode.NORMAL); + loadDropHandler.removeSegment(segment, DataSegmentChangeCallback.NOOP, false); throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); } try { diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 6292c4ef46bc..7b49010b19b2 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -27,13 +27,14 @@ import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; +import org.apache.druid.error.DruidException; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; -import org.apache.druid.server.http.SegmentLoadingMode; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; @@ -44,6 +45,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -195,6 +197,12 @@ public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallba removeSegment(segment, callback, true, SegmentLoadingMode.NORMAL); } + @VisibleForTesting + void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback, boolean scheduleDrop) + { + removeSegment(segment, callback, scheduleDrop, SegmentLoadingMode.NORMAL); + } + @VisibleForTesting void removeSegment( final DataSegment segment, @@ -228,20 +236,11 @@ void removeSegment( "Completely removing segment[%s] in [%,d]ms.", segment.getId(), config.getDropSegmentDelayMillis() ); - if (SegmentLoadingMode.TURBO.equals(segmentLoadingMode)) { - turboExec.schedule( - runnable, - config.getDropSegmentDelayMillis(), - TimeUnit.MILLISECONDS - ); - } else { - exec.schedule( - runnable, - config.getDropSegmentDelayMillis(), - TimeUnit.MILLISECONDS - ); - - } + getExecutorService(segmentLoadingMode).schedule( + runnable, + config.getDropSegmentDelayMillis(), + TimeUnit.MILLISECONDS + ); } else { runnable.run(); } @@ -311,23 +310,13 @@ public void addSegment( ) { requestStatuses.put(changeRequest, new AtomicReference<>(SegmentChangeStatus.PENDING)); - if (SegmentLoadingMode.TURBO.equals(segmentLoadingMode)) { - turboExec.submit( - () -> SegmentLoadDropHandler.this.addSegment( - ((SegmentChangeRequestLoad) changeRequest).getSegment(), - () -> resolveWaitingFutures(), - segmentLoadingMode - ) - ); - } else { - exec.submit( - () -> SegmentLoadDropHandler.this.addSegment( - ((SegmentChangeRequestLoad) changeRequest).getSegment(), - () -> resolveWaitingFutures(), - segmentLoadingMode - ) - ); - } + getExecutorService(segmentLoadingMode).submit( + () -> SegmentLoadDropHandler.this.addSegment( + ((SegmentChangeRequestLoad) changeRequest).getSegment(), + () -> resolveWaitingFutures(), + segmentLoadingMode + ) + ); } @Override @@ -427,5 +416,31 @@ public boolean cancel(boolean interruptIfRunning) return true; } } + + public ScheduledExecutorService getExecutorService(SegmentLoadingMode loadingMode) + { + switch (loadingMode) { + case TURBO: + return turboExec; + case NORMAL: + return exec; + default: + throw DruidException.defensive("Unknown execution mode [%s]", loadingMode); + } + } + + public static SegmentLoadingMode getLoadingMode(CoordinatorDynamicConfig coordinatorDynamicConfig, String serverName) + { + final Set turboLoadHistoricals = coordinatorDynamicConfig.getTurboLoadHistoricals(); + return turboLoadHistoricals.contains(serverName) ? + SegmentLoadingMode.TURBO : + SegmentLoadingMode.NORMAL; + } + + public enum SegmentLoadingMode + { + NORMAL, + TURBO + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 71f598038b2b..7423cc7f5e24 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -38,6 +38,7 @@ import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeStatus; +import org.apache.druid.server.coordination.SegmentLoadDropHandler; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; @@ -47,7 +48,6 @@ import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.server.http.HistoricalSegmentChangeRequest; -import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -152,7 +152,7 @@ public HttpLoadQueuePeon( this.changeRequestURL = new URL( new URL(baseUrl), StringUtils.nonStrictFormat( - "druid-internal/v1/segments/changeRequestsV2?timeout=%d", + "druid-internal/v1/segments/segmentChangeRequests?timeout=%d", config.getHostTimeout().getMillis() ) ); @@ -209,10 +209,8 @@ private void doSegmentManagement() return; } - Set turboLoadHistoricals = coordinatorConfigManager.getCurrentDynamicConfig().getTurboLoadHistoricals(); - SegmentLoadingMode loadingMode = turboLoadHistoricals.contains(serverName) ? - SegmentLoadingMode.TURBO : - SegmentLoadingMode.NORMAL; + SegmentLoadDropHandler.SegmentLoadingMode loadingMode = + SegmentLoadDropHandler.getLoadingMode(coordinatorConfigManager.getCurrentDynamicConfig(), serverName); HistoricalSegmentChangeRequest request = new HistoricalSegmentChangeRequest(newRequests, loadingMode); diff --git a/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java index d77ac153b77a..b47bde369214 100644 --- a/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java +++ b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.server.coordination.DataSegmentChangeRequest; +import org.apache.druid.server.coordination.SegmentLoadDropHandler; import java.util.List; @@ -29,12 +30,12 @@ public class HistoricalSegmentChangeRequest { private final List changeRequestList; - private final SegmentLoadingMode segmentLoadingMode; + private final SegmentLoadDropHandler.SegmentLoadingMode segmentLoadingMode; @JsonCreator public HistoricalSegmentChangeRequest( @JsonProperty("changeRequestList") List changeRequestList, - @JsonProperty("segmentLoadingMode") SegmentLoadingMode segmentLoadingMode + @JsonProperty("segmentLoadingMode") SegmentLoadDropHandler.SegmentLoadingMode segmentLoadingMode ) { this.changeRequestList = changeRequestList; @@ -48,7 +49,7 @@ public List getChangeRequestList() } @JsonProperty("segmentLoadingMode") - public SegmentLoadingMode getSegmentLoadingMode() + public SegmentLoadDropHandler.SegmentLoadingMode getSegmentLoadingMode() { return segmentLoadingMode; } diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index a0a5148bd686..e4287be4309a 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -248,7 +248,7 @@ public void applyDataSegmentChangeRequests( final ResponseContext context = createContext(req.getHeader("Accept")); final ListenableFuture> future = - loadDropRequestHandler.processBatch(changeRequestList, SegmentLoadingMode.NORMAL); + loadDropRequestHandler.processBatch(changeRequestList, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); final AsyncContext asyncContext = req.startAsync(); @@ -334,7 +334,7 @@ public void onFailure(Throwable th) * It returns a map of "load/drop request -> SUCCESS/FAILED/PENDING status" for each request in the batch. */ @POST - @Path("/changeRequestsV2") + @Path("/segmentChangeRequests") @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public void applyDataSegmentChangeRequests( diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java index 647c390394f5..e758ed285611 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java @@ -19,8 +19,6 @@ package org.apache.druid.server.http; -public enum SegmentLoadingMode -{ - NORMAL, - TURBO -} +/** + * + */ diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java index 194016b5164e..d6903fccf7b1 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java @@ -36,7 +36,6 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; -import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -220,12 +219,12 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException // Make sure adding segments beyond allowed size fails DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE); - loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); + loadDropHandler.addSegment(newSegment, null, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment)); // Clearing some segment should allow for new segments - loadDropHandler.removeSegment(expectedSegments.get(0), null, false, SegmentLoadingMode.NORMAL); - loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); + loadDropHandler.removeSegment(expectedSegments.get(0), null, false, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + loadDropHandler.addSegment(newSegment, null, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); bootstrapper.stop(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 8932d3c82a0f..a093d32c3f9e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -30,7 +30,6 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.SegmentChangeStatus.State; -import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -231,7 +230,7 @@ public void testProcessBatch() throws Exception new SegmentChangeRequestDrop(segment2) ); - ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); Map expectedStatusMap = new HashMap<>(); expectedStatusMap.put(batch.get(0), SegmentChangeStatus.PENDING); @@ -245,7 +244,7 @@ public void testProcessBatch() throws Exception runnable.run(); } - result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadingMode.NORMAL).get(); + result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadDropHandler.SegmentLoadingMode.NORMAL).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); @@ -272,7 +271,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); @@ -281,7 +280,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState()); Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); - future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -344,7 +343,7 @@ public int getDropSegmentDelayMillis() List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); // Request 1: Load the segment - ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -355,7 +354,7 @@ public int getDropSegmentDelayMillis() // Request 2: Drop the segment batch = ImmutableList.of(new SegmentChangeRequestDrop(segment1)); - future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -373,7 +372,7 @@ public int getDropSegmentDelayMillis() // Request 3: Reload the segment batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -390,7 +389,7 @@ public int getDropSegmentDelayMillis() // Request 4: Try to reload the segment - segment is loaded and announced again batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } From d0c62fde698988c799bb8f4b661699367bf4baab Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 4 Mar 2025 12:22:11 +0530 Subject: [PATCH 08/30] Address review comments --- .../coordination/SegmentLoadDropHandler.java | 17 +---------------- .../coordinator/CoordinatorDynamicConfig.java | 9 +++++++++ .../coordinator/loading/HttpLoadQueuePeon.java | 7 ++++--- .../http/HistoricalSegmentChangeRequest.java | 7 +++---- .../server/http/SegmentListerResource.java | 2 +- .../druid/server/http/SegmentLoadingMode.java | 8 +++++--- .../SegmentBootstrapperCacheTest.java | 7 ++++--- .../SegmentLoadDropHandlerTest.java | 17 +++++++++-------- 8 files changed, 36 insertions(+), 38 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 7b49010b19b2..d9c6c938b37e 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -34,7 +34,7 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.segment.loading.SegmentLoadingException; import org.apache.druid.server.SegmentManager; -import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.SegmentRowCountDistribution; import org.apache.druid.timeline.DataSegment; @@ -45,7 +45,6 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; @@ -428,19 +427,5 @@ public ScheduledExecutorService getExecutorService(SegmentLoadingMode loadingMod throw DruidException.defensive("Unknown execution mode [%s]", loadingMode); } } - - public static SegmentLoadingMode getLoadingMode(CoordinatorDynamicConfig coordinatorDynamicConfig, String serverName) - { - final Set turboLoadHistoricals = coordinatorDynamicConfig.getTurboLoadHistoricals(); - return turboLoadHistoricals.contains(serverName) ? - SegmentLoadingMode.TURBO : - SegmentLoadingMode.NORMAL; - } - - public enum SegmentLoadingMode - { - NORMAL, - TURBO - } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 4a6c1e6ca736..29183332f889 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -27,6 +27,7 @@ import org.apache.druid.error.InvalidInput; import org.apache.druid.server.coordinator.duty.KillUnusedSegments; import org.apache.druid.server.coordinator.stats.Dimension; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.utils.JvmUtils; import javax.annotation.Nullable; @@ -204,6 +205,14 @@ private static Set parseJsonStringOrArray(Object jsonStringOrArray) } } + public static SegmentLoadingMode getLoadingMode(CoordinatorDynamicConfig coordinatorDynamicConfig, String serverName) + { + final Set turboLoadHistoricals = coordinatorDynamicConfig.getTurboLoadHistoricals(); + return turboLoadHistoricals.contains(serverName) ? + SegmentLoadingMode.TURBO : + SegmentLoadingMode.NORMAL; + } + @JsonProperty("millisToWaitBeforeDeleting") public long getMarkSegmentAsUnusedDelayMillis() { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 7423cc7f5e24..630875df6a01 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -38,9 +38,9 @@ import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeStatus; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.coordinator.CoordinatorConfigManager; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; @@ -48,6 +48,7 @@ import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; import org.apache.druid.server.http.HistoricalSegmentChangeRequest; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; import org.jboss.netty.handler.codec.http.HttpMethod; @@ -209,8 +210,8 @@ private void doSegmentManagement() return; } - SegmentLoadDropHandler.SegmentLoadingMode loadingMode = - SegmentLoadDropHandler.getLoadingMode(coordinatorConfigManager.getCurrentDynamicConfig(), serverName); + SegmentLoadingMode loadingMode = + CoordinatorDynamicConfig.getLoadingMode(coordinatorConfigManager.getCurrentDynamicConfig(), serverName); HistoricalSegmentChangeRequest request = new HistoricalSegmentChangeRequest(newRequests, loadingMode); diff --git a/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java index b47bde369214..d77ac153b77a 100644 --- a/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java +++ b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.server.coordination.DataSegmentChangeRequest; -import org.apache.druid.server.coordination.SegmentLoadDropHandler; import java.util.List; @@ -30,12 +29,12 @@ public class HistoricalSegmentChangeRequest { private final List changeRequestList; - private final SegmentLoadDropHandler.SegmentLoadingMode segmentLoadingMode; + private final SegmentLoadingMode segmentLoadingMode; @JsonCreator public HistoricalSegmentChangeRequest( @JsonProperty("changeRequestList") List changeRequestList, - @JsonProperty("segmentLoadingMode") SegmentLoadDropHandler.SegmentLoadingMode segmentLoadingMode + @JsonProperty("segmentLoadingMode") SegmentLoadingMode segmentLoadingMode ) { this.changeRequestList = changeRequestList; @@ -49,7 +48,7 @@ public List getChangeRequestList() } @JsonProperty("segmentLoadingMode") - public SegmentLoadDropHandler.SegmentLoadingMode getSegmentLoadingMode() + public SegmentLoadingMode getSegmentLoadingMode() { return segmentLoadingMode; } diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index e4287be4309a..7167b41ced52 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -248,7 +248,7 @@ public void applyDataSegmentChangeRequests( final ResponseContext context = createContext(req.getHeader("Accept")); final ListenableFuture> future = - loadDropRequestHandler.processBatch(changeRequestList, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + loadDropRequestHandler.processBatch(changeRequestList, SegmentLoadingMode.NORMAL); final AsyncContext asyncContext = req.startAsync(); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java index e758ed285611..818bac574825 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java @@ -19,6 +19,8 @@ package org.apache.druid.server.http; -/** - * - */ +public enum SegmentLoadingMode +{ + NORMAL, + TURBO; +} diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java index d6903fccf7b1..194016b5164e 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java @@ -36,6 +36,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -219,12 +220,12 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException // Make sure adding segments beyond allowed size fails DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE); - loadDropHandler.addSegment(newSegment, null, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment)); // Clearing some segment should allow for new segments - loadDropHandler.removeSegment(expectedSegments.get(0), null, false, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); - loadDropHandler.addSegment(newSegment, null, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + loadDropHandler.removeSegment(expectedSegments.get(0), null, false, SegmentLoadingMode.NORMAL); + loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); bootstrapper.stop(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index a093d32c3f9e..8932d3c82a0f 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -30,6 +30,7 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.coordination.SegmentChangeStatus.State; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; import org.junit.Before; @@ -230,7 +231,7 @@ public void testProcessBatch() throws Exception new SegmentChangeRequestDrop(segment2) ); - ListenableFuture> future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); Map expectedStatusMap = new HashMap<>(); expectedStatusMap.put(batch.get(0), SegmentChangeStatus.PENDING); @@ -244,7 +245,7 @@ public void testProcessBatch() throws Exception runnable.run(); } - result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadDropHandler.SegmentLoadingMode.NORMAL).get(); + result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadingMode.NORMAL).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); @@ -271,7 +272,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ DataSegment segment1 = makeSegment("batchtest1", "1", Intervals.of("P1d/2011-04-01")); List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - ListenableFuture> future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); @@ -280,7 +281,7 @@ public void testProcessBatchDuplicateLoadRequestsWhenFirstRequestFailsSecondRequ Assert.assertEquals(State.FAILED, result.get(0).getStatus().getState()); Assert.assertEquals(ImmutableList.of(), segmentAnnouncer.getObservedSegments()); - future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -343,7 +344,7 @@ public int getDropSegmentDelayMillis() List batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); // Request 1: Load the segment - ListenableFuture> future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + ListenableFuture> future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -354,7 +355,7 @@ public int getDropSegmentDelayMillis() // Request 2: Drop the segment batch = ImmutableList.of(new SegmentChangeRequestDrop(segment1)); - future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -372,7 +373,7 @@ public int getDropSegmentDelayMillis() // Request 3: Reload the segment batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } @@ -389,7 +390,7 @@ public int getDropSegmentDelayMillis() // Request 4: Try to reload the segment - segment is loaded and announced again batch = ImmutableList.of(new SegmentChangeRequestLoad(segment1)); - future = handler.processBatch(batch, SegmentLoadDropHandler.SegmentLoadingMode.NORMAL); + future = handler.processBatch(batch, SegmentLoadingMode.NORMAL); for (Runnable runnable : scheduledRunnable) { runnable.run(); } From cf389fe018efe04a2916988f47a10fdc27f80615 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 4 Mar 2025 13:40:10 +0530 Subject: [PATCH 09/30] Address review comments --- .../loading/HttpLoadQueuePeon.java | 15 +++++---------- .../loading/LoadQueueTaskMaster.java | 19 ++++++++++++++----- .../SegmentBootstrapperCacheTest.java | 7 +++---- .../loading/HttpLoadQueuePeonTest.java | 5 ++--- .../CoordinatorSimulationBuilder.java | 2 +- .../org/apache/druid/cli/CliCoordinator.java | 2 +- 6 files changed, 26 insertions(+), 24 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 630875df6a01..91b599b3512d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -39,8 +39,6 @@ import org.apache.druid.server.coordination.SegmentChangeRequestLoad; import org.apache.druid.server.coordination.SegmentChangeStatus; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; -import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.stats.CoordinatorRunStats; import org.apache.druid.server.coordinator.stats.CoordinatorStat; @@ -73,6 +71,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; /** * @@ -116,7 +115,6 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final HttpLoadQueuePeonConfig config; - private final String serverName; private final ObjectMapper jsonMapper; private final HttpClient httpClient; private final URL changeRequestURL; @@ -124,19 +122,18 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final AtomicBoolean mainLoopInProgress = new AtomicBoolean(false); private final ExecutorService callBackExecutor; - private final CoordinatorConfigManager coordinatorConfigManager; + private final Supplier loadingModeSupplier; private final ObjectWriter requestBodyWriter; public HttpLoadQueuePeon( String baseUrl, - String serverName, ObjectMapper jsonMapper, HttpClient httpClient, HttpLoadQueuePeonConfig config, ScheduledExecutorService processingExecutor, ExecutorService callBackExecutor, - CoordinatorConfigManager coordinatorConfigManager + Supplier loadingModeSupplier ) { this.jsonMapper = jsonMapper; @@ -145,10 +142,9 @@ public HttpLoadQueuePeon( this.config = config; this.processingExecutor = processingExecutor; this.callBackExecutor = callBackExecutor; - this.coordinatorConfigManager = coordinatorConfigManager; this.serverId = baseUrl; - this.serverName = serverName; + this.loadingModeSupplier = loadingModeSupplier; try { this.changeRequestURL = new URL( new URL(baseUrl), @@ -210,8 +206,7 @@ private void doSegmentManagement() return; } - SegmentLoadingMode loadingMode = - CoordinatorDynamicConfig.getLoadingMode(coordinatorConfigManager.getCurrentDynamicConfig(), serverName); + SegmentLoadingMode loadingMode = loadingModeSupplier.get(); HistoricalSegmentChangeRequest request = new HistoricalSegmentChangeRequest(newRequests, loadingMode); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java index ad0be6a2253c..8134350c2406 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java @@ -25,7 +25,7 @@ import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; +import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import java.util.List; @@ -35,6 +35,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; /** * Provides LoadQueuePeons @@ -48,7 +49,7 @@ public class LoadQueueTaskMaster private final ExecutorService callbackExec; private final HttpLoadQueuePeonConfig config; private final HttpClient httpClient; - private final CoordinatorConfigManager coordinatorConfigManager; + private final Supplier coordinatorDynamicConfigSupplier; @GuardedBy("this") private final AtomicBoolean isLeader = new AtomicBoolean(false); @@ -61,7 +62,7 @@ public LoadQueueTaskMaster( ExecutorService callbackExec, HttpLoadQueuePeonConfig config, HttpClient httpClient, - CoordinatorConfigManager coordinatorConfigManager + Supplier coordinatorDynamicConfigSupplier ) { this.jsonMapper = jsonMapper; @@ -69,12 +70,20 @@ public LoadQueueTaskMaster( this.callbackExec = callbackExec; this.config = config; this.httpClient = httpClient; - this.coordinatorConfigManager = coordinatorConfigManager; + this.coordinatorDynamicConfigSupplier = coordinatorDynamicConfigSupplier; } private LoadQueuePeon createPeon(ImmutableDruidServer server) { - return new HttpLoadQueuePeon(server.getURL(), server.getName(), jsonMapper, httpClient, config, peonExec, callbackExec, coordinatorConfigManager); + return new HttpLoadQueuePeon( + server.getURL(), + jsonMapper, + httpClient, + config, + peonExec, + callbackExec, + () -> CoordinatorDynamicConfig.getLoadingMode(coordinatorDynamicConfigSupplier.get(), server.getName()) + ); } public Map getAllPeons() diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java index 194016b5164e..187725317a21 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentBootstrapperCacheTest.java @@ -36,7 +36,6 @@ import org.apache.druid.segment.loading.StorageLocationConfig; import org.apache.druid.server.SegmentManager; import org.apache.druid.server.TestSegmentUtils; -import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.server.metrics.DataSourceTaskIdHolder; import org.apache.druid.timeline.DataSegment; import org.junit.Assert; @@ -220,12 +219,12 @@ public void testLoadLocalCache() throws IOException, SegmentLoadingException // Make sure adding segments beyond allowed size fails DataSegment newSegment = TestSegmentUtils.makeSegment("test", "new-segment", SEGMENT_SIZE); - loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); + loadDropHandler.addSegment(newSegment, null); Assert.assertFalse(segmentAnnouncer.getObservedSegments().contains(newSegment)); // Clearing some segment should allow for new segments - loadDropHandler.removeSegment(expectedSegments.get(0), null, false, SegmentLoadingMode.NORMAL); - loadDropHandler.addSegment(newSegment, null, SegmentLoadingMode.NORMAL); + loadDropHandler.removeSegment(expectedSegments.get(0), null, false); + loadDropHandler.addSegment(newSegment, null); Assert.assertTrue(segmentAnnouncer.getObservedSegments().contains(newSegment)); bootstrapper.stop(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index c923e071b31a..2ab1d7aa2133 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -34,13 +34,13 @@ import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeStatus; -import org.apache.druid.server.coordinator.CoordinatorConfigManager; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; import org.apache.druid.server.http.HistoricalSegmentChangeRequest; +import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.jboss.netty.buffer.ChannelBuffers; @@ -93,7 +93,6 @@ public void setUp() EasyMock.replay(configManager); httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", - "dummy:4000", MAPPER, httpClient, new HttpLoadQueuePeonConfig(null, null, 10), @@ -103,7 +102,7 @@ public void setUp() true ), httpClient.callbackExecutor, - new CoordinatorConfigManager(configManager, null, null) + () -> SegmentLoadingMode.NORMAL ); httpLoadQueuePeon.start(); } 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 053afa6466cf..3bc22d3da02d 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 @@ -469,7 +469,7 @@ private Environment( executorFactory.create(1, ExecutorFactory.LOAD_CALLBACK_EXECUTOR), coordinatorConfig.getHttpLoadQueuePeonConfig(), httpClient, - coordinatorConfigManager + coordinatorConfigManager::getCurrentDynamicConfig ); this.loadQueueManager = diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 8fb5b9b1e43d..1f61d9716eba 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -318,7 +318,7 @@ public LoadQueueTaskMaster getLoadQueueTaskMaster( callBackExec, config.getHttpLoadQueuePeonConfig(), httpClient, - coordinatorConfigManager + coordinatorConfigManager::getCurrentDynamicConfig ); } } From a1af8c3163aa0f69d8d6b782b2e200c13c514832 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 4 Mar 2025 14:01:30 +0530 Subject: [PATCH 10/30] Revert new API --- .../loading/HttpLoadQueuePeon.java | 9 +- .../http/HistoricalSegmentChangeRequest.java | 55 --------- .../server/http/SegmentListerResource.java | 116 +----------------- .../loading/HttpLoadQueuePeonTest.java | 5 +- 4 files changed, 8 insertions(+), 177 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 91b599b3512d..e6c93ed078cd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -45,7 +45,6 @@ import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; -import org.apache.druid.server.http.HistoricalSegmentChangeRequest; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -78,7 +77,7 @@ */ public class HttpLoadQueuePeon implements LoadQueuePeon { - public static final TypeReference REQUEST_ENTITY_TYPE_REF = + public static final TypeReference> REQUEST_ENTITY_TYPE_REF = new TypeReference<>() {}; public static final TypeReference> RESPONSE_ENTITY_TYPE_REF = @@ -149,7 +148,7 @@ public HttpLoadQueuePeon( this.changeRequestURL = new URL( new URL(baseUrl), StringUtils.nonStrictFormat( - "druid-internal/v1/segments/segmentChangeRequests?timeout=%d", + "druid-internal/v1/segments/changeRequests?timeout=%d", config.getHostTimeout().getMillis() ) ); @@ -208,8 +207,6 @@ private void doSegmentManagement() SegmentLoadingMode loadingMode = loadingModeSupplier.get(); - HistoricalSegmentChangeRequest request = new HistoricalSegmentChangeRequest(newRequests, loadingMode); - try { log.trace("Sending [%d] load/drop requests to Server[%s] in loadingMode [%s].", newRequests.size(), serverId, loadingMode); final boolean hasLoadRequests = newRequests.stream().anyMatch(r -> r instanceof SegmentChangeRequestLoad); @@ -222,7 +219,7 @@ private void doSegmentManagement() new Request(HttpMethod.POST, changeRequestURL) .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON) .addHeader(HttpHeaders.Names.CONTENT_TYPE, MediaType.APPLICATION_JSON) - .setContent(requestBodyWriter.writeValueAsBytes(request)), + .setContent(requestBodyWriter.writeValueAsBytes(newRequests)), responseHandler, new Duration(config.getHostTimeout().getMillis() + 5000) ); diff --git a/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java b/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java deleted file mode 100644 index d77ac153b77a..000000000000 --- a/server/src/main/java/org/apache/druid/server/http/HistoricalSegmentChangeRequest.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * 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.http; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.server.coordination.DataSegmentChangeRequest; - -import java.util.List; - -public class HistoricalSegmentChangeRequest -{ - private final List changeRequestList; - - private final SegmentLoadingMode segmentLoadingMode; - - @JsonCreator - public HistoricalSegmentChangeRequest( - @JsonProperty("changeRequestList") List changeRequestList, - @JsonProperty("segmentLoadingMode") SegmentLoadingMode segmentLoadingMode - ) - { - this.changeRequestList = changeRequestList; - this.segmentLoadingMode = segmentLoadingMode; - } - - @JsonProperty("changeRequestList") - public List getChangeRequestList() - { - return changeRequestList; - } - - @JsonProperty("segmentLoadingMode") - public SegmentLoadingMode getSegmentLoadingMode() - { - return segmentLoadingMode; - } -} diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index 7167b41ced52..434e6c3ef4c5 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -215,114 +215,6 @@ public void onFailure(Throwable th) return null; } - /** - * Deprecated. - * - * @see SegmentListerResource#applyDataSegmentChangeRequests(long, HistoricalSegmentChangeRequest, HttpServletRequest) - */ - @Deprecated - @POST - @Path("/changeRequests") - @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) - public void applyDataSegmentChangeRequests( - @QueryParam("timeout") long timeout, - List changeRequestList, - @Context final HttpServletRequest req - ) throws IOException - { - if (loadDropRequestHandler == null) { - sendErrorResponse(req, HttpServletResponse.SC_NOT_FOUND, "load/drop handler is not available."); - return; - } - - if (timeout <= 0) { - sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "timeout must be positive."); - return; - } - - if (changeRequestList == null || changeRequestList.isEmpty()) { - sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "No change requests provided."); - return; - } - - final ResponseContext context = createContext(req.getHeader("Accept")); - final ListenableFuture> future = - loadDropRequestHandler.processBatch(changeRequestList, SegmentLoadingMode.NORMAL); - - final AsyncContext asyncContext = req.startAsync(); - - asyncContext.addListener( - new AsyncListener() - { - @Override - public void onComplete(AsyncEvent event) - { - } - - @Override - public void onTimeout(AsyncEvent event) - { - - // HTTP 204 NO_CONTENT is sent to the client. - future.cancel(true); - event.getAsyncContext().complete(); - } - - @Override - public void onError(AsyncEvent event) - { - } - - @Override - public void onStartAsync(AsyncEvent event) - { - } - } - ); - - Futures.addCallback( - future, - new FutureCallback<>() - { - @Override - public void onSuccess(List result) - { - try { - HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); - response.setStatus(HttpServletResponse.SC_OK); - context.inputMapper.writerFor(HttpLoadQueuePeon.RESPONSE_ENTITY_TYPE_REF) - .writeValue(asyncContext.getResponse().getOutputStream(), result); - asyncContext.complete(); - } - catch (Exception ex) { - log.debug(ex, "Request timed out or closed already."); - } - } - - @Override - public void onFailure(Throwable th) - { - try { - HttpServletResponse response = (HttpServletResponse) asyncContext.getResponse(); - if (th instanceof IllegalArgumentException) { - response.sendError(HttpServletResponse.SC_BAD_REQUEST, th.getMessage()); - } else { - response.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR, th.getMessage()); - } - asyncContext.complete(); - } - catch (Exception ex) { - log.debug(ex, "Request timed out or closed already."); - } - } - }, - MoreExecutors.directExecutor() - ); - - asyncContext.setTimeout(timeout); - } - /** * This endpoint is used by HttpLoadQueuePeon to assign segment load/drop requests batch. This endpoint makes the * client wait till one of the following events occur. Note that this is implemented using async IO so no jetty @@ -334,12 +226,13 @@ public void onFailure(Throwable th) * It returns a map of "load/drop request -> SUCCESS/FAILED/PENDING status" for each request in the batch. */ @POST - @Path("/segmentChangeRequests") + @Path("/changeRequests") @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) @Consumes({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public void applyDataSegmentChangeRequests( @QueryParam("timeout") long timeout, - HistoricalSegmentChangeRequest historicalSegmentChangeRequest, + @QueryParam("loadingMode") @Nullable SegmentLoadingMode loadingMode, + List changeRequestList, @Context final HttpServletRequest req ) throws IOException { @@ -352,7 +245,6 @@ public void applyDataSegmentChangeRequests( sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "timeout must be positive."); return; } - List changeRequestList = historicalSegmentChangeRequest.getChangeRequestList(); if (changeRequestList == null || changeRequestList.isEmpty()) { sendErrorResponse(req, HttpServletResponse.SC_BAD_REQUEST, "No change requests provided."); @@ -361,7 +253,7 @@ public void applyDataSegmentChangeRequests( final ResponseContext context = createContext(req.getHeader("Accept")); final ListenableFuture> future = - loadDropRequestHandler.processBatch(changeRequestList, historicalSegmentChangeRequest.getSegmentLoadingMode()); + loadDropRequestHandler.processBatch(changeRequestList, loadingMode == null ? SegmentLoadingMode.NORMAL : loadingMode); final AsyncContext asyncContext = req.startAsync(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 2ab1d7aa2133..89351cd8163d 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -39,7 +39,6 @@ import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; -import org.apache.druid.server.http.HistoricalSegmentChangeRequest; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; @@ -364,13 +363,11 @@ public ListenableFuture go( httpResponse.setContent(ChannelBuffers.buffer(0)); httpResponseHandler.handleResponse(httpResponse, null); try { - HistoricalSegmentChangeRequest segmentChangeRequest = MAPPER.readValue( + List changeRequests = MAPPER.readValue( request.getContent().array(), HttpLoadQueuePeon.REQUEST_ENTITY_TYPE_REF ); - List changeRequests = segmentChangeRequest.getChangeRequestList(); - List statuses = new ArrayList<>(changeRequests.size()); for (DataSegmentChangeRequest cr : changeRequests) { cr.go(this, null); From 2ec0dd16cb3b07f96b84640ca70f82e7f3b2faa9 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 7 Mar 2025 11:39:51 +0530 Subject: [PATCH 11/30] Address review comments --- .../loading/HttpLoadQueuePeon.java | 25 +++++++++---------- .../druid/server/http/SegmentLoadingMode.java | 2 +- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index e6c93ed078cd..34b37241ae6c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -116,7 +116,6 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final ObjectMapper jsonMapper; private final HttpClient httpClient; - private final URL changeRequestURL; private final String serverId; private final AtomicBoolean mainLoopInProgress = new AtomicBoolean(false); @@ -144,18 +143,6 @@ public HttpLoadQueuePeon( this.serverId = baseUrl; this.loadingModeSupplier = loadingModeSupplier; - try { - this.changeRequestURL = new URL( - new URL(baseUrl), - StringUtils.nonStrictFormat( - "druid-internal/v1/segments/changeRequests?timeout=%d", - config.getHostTimeout().getMillis() - ) - ); - } - catch (MalformedURLException ex) { - throw new RuntimeException(ex); - } } private void doSegmentManagement() @@ -214,6 +201,15 @@ private void doSegmentManagement() loadingRateTracker.markBatchLoadingStarted(); } + final URL changeRequestURL = new URL( + new URL(serverId), + StringUtils.nonStrictFormat( + "druid-internal/v1/segments/changeRequests?timeout=%d&loadingMode=%s", + config.getHostTimeout().getMillis(), + loadingMode + ) + ); + BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); ListenableFuture future = httpClient.go( new Request(HttpMethod.POST, changeRequestURL) @@ -315,6 +311,9 @@ private void logRequestFailure(Throwable t) processingExecutor ); } + catch (MalformedURLException ex) { + throw new RuntimeException(ex); + } catch (Throwable th) { log.error(th, "Error sending load/drop request to [%s].", serverId); mainLoopInProgress.set(false); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java index 818bac574825..647c390394f5 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java @@ -22,5 +22,5 @@ public enum SegmentLoadingMode { NORMAL, - TURBO; + TURBO } From 117ae34eee7c3a88ff265f939378e48e3d622209 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 7 Mar 2025 13:27:59 +0530 Subject: [PATCH 12/30] Address review comments --- .../druid/server/coordination/SegmentLoadDropHandler.java | 2 +- .../druid/server/coordinator/CoordinatorDynamicConfig.java | 5 ++++- .../server/coordinator/loading/LoadQueueTaskMaster.java | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index d9c6c938b37e..a59f698588e3 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -416,7 +416,7 @@ public boolean cancel(boolean interruptIfRunning) } } - public ScheduledExecutorService getExecutorService(SegmentLoadingMode loadingMode) + private ScheduledExecutorService getExecutorService(SegmentLoadingMode loadingMode) { switch (loadingMode) { case TURBO: diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 29183332f889..185066b4704b 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -205,7 +205,10 @@ private static Set parseJsonStringOrArray(Object jsonStringOrArray) } } - public static SegmentLoadingMode getLoadingMode(CoordinatorDynamicConfig coordinatorDynamicConfig, String serverName) + public static SegmentLoadingMode getLoadingModeForServer( + CoordinatorDynamicConfig coordinatorDynamicConfig, + String serverName + ) { final Set turboLoadHistoricals = coordinatorDynamicConfig.getTurboLoadHistoricals(); return turboLoadHistoricals.contains(serverName) ? diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java index 8134350c2406..8e077a6a55ca 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java @@ -82,7 +82,7 @@ private LoadQueuePeon createPeon(ImmutableDruidServer server) config, peonExec, callbackExec, - () -> CoordinatorDynamicConfig.getLoadingMode(coordinatorDynamicConfigSupplier.get(), server.getName()) + () -> CoordinatorDynamicConfig.getLoadingModeForServer(coordinatorDynamicConfigSupplier.get(), server.getName()) ); } From 426def6e86a17a276bfb4a811a058e8bab19f1a4 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 7 Mar 2025 13:34:03 +0530 Subject: [PATCH 13/30] Rename config --- .../coordinator/CoordinatorDynamicConfig.java | 34 +++++++++---------- 1 file changed, 17 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 185066b4704b..f65c0322b93c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -71,7 +71,7 @@ public class CoordinatorDynamicConfig private final Map debugDimensions; private final Map validDebugDimensions; - private final Set turboLoadHistoricals; + private final Set turboLoadingNodes; /** * Stale pending segments belonging to the data sources in this list are not killed by {@code @@ -122,7 +122,7 @@ public CoordinatorDynamicConfig( @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment, @JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading, @JsonProperty("debugDimensions") @Nullable Map debugDimensions, - @JsonProperty("turboLoadHistoricals") @Nullable Set turboLoadHistoricals + @JsonProperty("turboLoadingNodes") @Nullable Set turboLoadingNodes ) { this.markSegmentAsUnusedDelayMillis = @@ -166,7 +166,7 @@ public CoordinatorDynamicConfig( ); this.debugDimensions = debugDimensions; this.validDebugDimensions = validateDebugDimensions(debugDimensions); - this.turboLoadHistoricals = parseJsonStringOrArray(turboLoadHistoricals); + this.turboLoadingNodes = parseJsonStringOrArray(turboLoadingNodes); } private Map validateDebugDimensions(Map debugDimensions) @@ -210,8 +210,8 @@ public static SegmentLoadingMode getLoadingModeForServer( String serverName ) { - final Set turboLoadHistoricals = coordinatorDynamicConfig.getTurboLoadHistoricals(); - return turboLoadHistoricals.contains(serverName) ? + final Set turboLoadingNodes = coordinatorDynamicConfig.getTurboLoadingNodes(); + return turboLoadingNodes.contains(serverName) ? SegmentLoadingMode.TURBO : SegmentLoadingMode.NORMAL; } @@ -325,9 +325,9 @@ public boolean getReplicateAfterLoadTimeout() } @JsonProperty - public Set getTurboLoadHistoricals() + public Set getTurboLoadingNodes() { - return turboLoadHistoricals; + return turboLoadingNodes; } @Override @@ -348,7 +348,7 @@ public String toString() ", decommissioningNodes=" + decommissioningNodes + ", pauseCoordination=" + pauseCoordination + ", replicateAfterLoadTimeout=" + replicateAfterLoadTimeout + - ", turboLoadHistoricals=" + turboLoadHistoricals + + ", turboLoadingNodes=" + turboLoadingNodes + '}'; } @@ -382,7 +382,7 @@ public boolean equals(Object o) dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn) && Objects.equals(decommissioningNodes, that.decommissioningNodes) - && Objects.equals(turboLoadHistoricals, that.turboLoadHistoricals) + && Objects.equals(turboLoadingNodes, that.turboLoadingNodes) && Objects.equals(debugDimensions, that.debugDimensions); } @@ -403,7 +403,7 @@ public int hashCode() decommissioningNodes, pauseCoordination, debugDimensions, - turboLoadHistoricals + turboLoadingNodes ); } @@ -455,7 +455,7 @@ public static class Builder private Boolean replicateAfterLoadTimeout; private Boolean useRoundRobinSegmentAssignment; private Boolean smartSegmentLoading; - private Set turboLoadHistoricals; + private Set turboLoadingNodes; public Builder() { @@ -479,7 +479,7 @@ public Builder( @JsonProperty("useRoundRobinSegmentAssignment") @Nullable Boolean useRoundRobinSegmentAssignment, @JsonProperty("smartSegmentLoading") @Nullable Boolean smartSegmentLoading, @JsonProperty("debugDimensions") @Nullable Map debugDimensions, - @JsonProperty("turboLoadHistoricals") @Nullable Set turboLoadHistoricals + @JsonProperty("turboLoadingNodes") @Nullable Set turboLoadingNodes ) { this.markSegmentAsUnusedDelayMillis = markSegmentAsUnusedDelayMillis; @@ -498,7 +498,7 @@ public Builder( this.useRoundRobinSegmentAssignment = useRoundRobinSegmentAssignment; this.smartSegmentLoading = smartSegmentLoading; this.debugDimensions = debugDimensions; - this.turboLoadHistoricals = turboLoadHistoricals; + this.turboLoadingNodes = turboLoadingNodes; } public Builder withMarkSegmentAsUnusedDelayMillis(long leadingTimeMillis) @@ -519,9 +519,9 @@ public Builder withSmartSegmentLoading(boolean smartSegmentLoading) return this; } - public Builder withTurboLoadHistoricals(Set turboLoadHistoricals) + public Builder withTurboLoadingNodes(Set turboLoadingNodes) { - this.turboLoadHistoricals = turboLoadHistoricals; + this.turboLoadingNodes = turboLoadingNodes; return this; } @@ -617,7 +617,7 @@ public CoordinatorDynamicConfig build() valueOrDefault(useRoundRobinSegmentAssignment, Defaults.USE_ROUND_ROBIN_ASSIGNMENT), valueOrDefault(smartSegmentLoading, Defaults.SMART_SEGMENT_LOADING), debugDimensions, - turboLoadHistoricals + turboLoadingNodes ); } @@ -648,7 +648,7 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) valueOrDefault(useRoundRobinSegmentAssignment, defaults.isUseRoundRobinSegmentAssignment()), valueOrDefault(smartSegmentLoading, defaults.isSmartSegmentLoading()), valueOrDefault(debugDimensions, defaults.getDebugDimensions()), - valueOrDefault(turboLoadHistoricals, defaults.getTurboLoadHistoricals()) + valueOrDefault(turboLoadingNodes, defaults.getTurboLoadingNodes()) ); } } From 09f679a679d86a3ec5a3327f29ee2ce5703792b8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 7 Mar 2025 14:34:15 +0530 Subject: [PATCH 14/30] Improve coverage --- .../coordinator/CoordinatorDynamicConfig.java | 6 +- .../loading/LoadQueueTaskMaster.java | 2 +- .../http/CoordinatorDynamicConfigTest.java | 72 ++++++++++++++----- 3 files changed, 55 insertions(+), 25 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index f65c0322b93c..373369e9ef00 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -205,12 +205,8 @@ private static Set parseJsonStringOrArray(Object jsonStringOrArray) } } - public static SegmentLoadingMode getLoadingModeForServer( - CoordinatorDynamicConfig coordinatorDynamicConfig, - String serverName - ) + public SegmentLoadingMode getLoadingModeForServer(String serverName) { - final Set turboLoadingNodes = coordinatorDynamicConfig.getTurboLoadingNodes(); return turboLoadingNodes.contains(serverName) ? SegmentLoadingMode.TURBO : SegmentLoadingMode.NORMAL; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java index 8e077a6a55ca..ea6f7183dd1e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java @@ -82,7 +82,7 @@ private LoadQueuePeon createPeon(ImmutableDruidServer server) config, peonExec, callbackExec, - () -> CoordinatorDynamicConfig.getLoadingModeForServer(coordinatorDynamicConfigSupplier.get(), server.getName()) + () -> coordinatorDynamicConfigSupplier.get().getLoadingModeForServer(server.getName()) ); } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index 41b758376954..d06d1b40f483 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -51,7 +51,8 @@ public void testSerde() throws Exception + " \"maxSegmentsInNodeLoadingQueue\": 1,\n" + " \"decommissioningNodes\": [\"host1\", \"host2\"],\n" + " \"pauseCoordination\": false,\n" - + " \"replicateAfterLoadTimeout\": false\n" + + " \"replicateAfterLoadTimeout\": false,\n" + + " \"turboLoadingNodes\":[\"host1\", \"host3\"]\n" + "}\n"; CoordinatorDynamicConfig actual = mapper.readValue( @@ -65,6 +66,7 @@ public void testSerde() throws Exception ); ImmutableSet decommissioning = ImmutableSet.of("host1", "host2"); ImmutableSet whitelist = ImmutableSet.of("test1", "test2"); + ImmutableSet turboLoading = ImmutableSet.of("host1", "host3"); assertConfig( actual, 1, @@ -78,7 +80,8 @@ public void testSerde() throws Exception 1, decommissioning, false, - false + false, + turboLoading ); actual = CoordinatorDynamicConfig.builder().withDecommissioningNodes(ImmutableSet.of("host1")).build(actual); @@ -95,7 +98,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), false, - false + false, + turboLoading ); actual = CoordinatorDynamicConfig.builder().build(actual); @@ -112,7 +116,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), false, - false + false, + turboLoading ); actual = CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(actual); @@ -129,7 +134,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), true, - false + false, + turboLoading ); actual = CoordinatorDynamicConfig.builder().withReplicateAfterLoadTimeout(true).build(actual); @@ -146,7 +152,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), true, - true + true, + turboLoading ); actual = CoordinatorDynamicConfig.builder().build(actual); @@ -163,7 +170,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), true, - true + true, + turboLoading ); actual = CoordinatorDynamicConfig.builder().withKillTaskSlotRatio(0.1).build(actual); @@ -180,7 +188,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), true, - true + true, + turboLoading ); actual = CoordinatorDynamicConfig.builder().withMaxKillTaskSlots(5).build(actual); @@ -197,7 +206,8 @@ public void testSerde() throws Exception 1, ImmutableSet.of("host1"), true, - true + true, + turboLoading ); } @@ -274,7 +284,8 @@ public void testDecommissioningParametersBackwardCompatibility() throws Exceptio + " \"replicationThrottleLimit\": 1,\n" + " \"balancerComputeThreads\": 2, \n" + " \"killDataSourceWhitelist\": [\"test1\",\"test2\"],\n" - + " \"maxSegmentsInNodeLoadingQueue\": 1\n" + + " \"maxSegmentsInNodeLoadingQueue\": 1,\n" + + " \"turboLoadingNodes\": [\"host3\",\"host4\"]\n" + "}\n"; CoordinatorDynamicConfig actual = mapper.readValue( @@ -288,6 +299,7 @@ public void testDecommissioningParametersBackwardCompatibility() throws Exceptio ); ImmutableSet decommissioning = ImmutableSet.of(); ImmutableSet whitelist = ImmutableSet.of("test1", "test2"); + ImmutableSet turboLoading = ImmutableSet.of("host3", "host4"); assertConfig( actual, 1, @@ -301,7 +313,8 @@ public void testDecommissioningParametersBackwardCompatibility() throws Exceptio 1, decommissioning, false, - false + false, + turboLoading ); actual = CoordinatorDynamicConfig.builder().withDecommissioningNodes(ImmutableSet.of("host1")).build(actual); @@ -318,7 +331,8 @@ public void testDecommissioningParametersBackwardCompatibility() throws Exceptio 1, ImmutableSet.of("host1"), false, - false + false, + turboLoading ); actual = CoordinatorDynamicConfig.builder().build(actual); @@ -335,7 +349,8 @@ public void testDecommissioningParametersBackwardCompatibility() throws Exceptio 1, ImmutableSet.of("host1"), false, - false + false, + turboLoading ); } @@ -374,7 +389,8 @@ public void testSerdeWithStringInKillDataSourceWhitelist() throws Exception 1, ImmutableSet.of(), false, - false + false, + ImmutableSet.of() ); } @@ -413,7 +429,8 @@ public void testHandleMissingPercentOfSegmentsToConsiderPerMove() throws Excepti 1, decommissioning, false, - false + false, + ImmutableSet.of() ); } @@ -448,7 +465,8 @@ public void testDeserializeWithoutMaxSegmentsInNodeLoadingQueue() throws Excepti EXPECTED_DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE, ImmutableSet.of(), false, - false + false, + ImmutableSet.of() ); } @@ -470,7 +488,8 @@ public void testBuilderDefaults() EXPECTED_DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE, emptyList, false, - false + false, + ImmutableSet.of() ); } @@ -495,7 +514,8 @@ public void testBuilderWithDefaultSpecificDataSourcesToKillUnusedSegmentsInSpeci EXPECTED_DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE, ImmutableSet.of(), false, - false + false, + ImmutableSet.of() ); } @@ -513,6 +533,18 @@ public void testUpdate() ); } + @Test + public void testTurboLoadingNodes() + { + CoordinatorDynamicConfig config = CoordinatorDynamicConfig + .builder() + .withTurboLoadingNodes(ImmutableSet.of("localhost:8083")) + .build(); + + Assert.assertEquals(SegmentLoadingMode.NORMAL, config.getLoadingModeForServer("localhost:8082")); + Assert.assertEquals(SegmentLoadingMode.TURBO, config.getLoadingModeForServer("localhost:8083")); + } + @Test public void testEqualsAndHashCode() { @@ -535,7 +567,8 @@ private void assertConfig( int expectedMaxSegmentsInNodeLoadingQueue, Set decommissioningNodes, boolean pauseCoordination, - boolean replicateAfterLoadTimeout + boolean replicateAfterLoadTimeout, + Set turboLoadingNodes ) { Assert.assertEquals( @@ -556,6 +589,7 @@ private void assertConfig( Assert.assertEquals(decommissioningNodes, config.getDecommissioningNodes()); Assert.assertEquals(pauseCoordination, config.getPauseCoordination()); Assert.assertEquals(replicateAfterLoadTimeout, config.getReplicateAfterLoadTimeout()); + Assert.assertEquals(turboLoadingNodes, config.getTurboLoadingNodes()); } private static int getDefaultNumBalancerThreads() From d2b4e9328b47bd33c6bdb734de4cf7c8f58cdc50 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sat, 8 Mar 2025 15:42:18 +0530 Subject: [PATCH 15/30] Add dynamic configuration of batch size --- .../coordination/SegmentLoadDropHandler.java | 5 ++ .../config/HttpLoadQueuePeonConfig.java | 12 ++-- .../loading/HttpLoadQueuePeon.java | 52 ++++++++++++++++-- .../http/HistoricalLoadingCapabilities.java | 55 +++++++++++++++++++ .../server/http/SegmentListerResource.java | 21 +++++++ .../DruidCoordinatorConfigTest.java | 4 +- 6 files changed, 139 insertions(+), 10 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index a59f698588e3..f0672187b0ca 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -427,5 +427,10 @@ private ScheduledExecutorService getExecutorService(SegmentLoadingMode loadingMo throw DruidException.defensive("Unknown execution mode [%s]", loadingMode); } } + + public SegmentLoaderConfig getSegmentLoaderConfig() + { + return config; + } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java index f6f402037d1f..d77943d61017 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java @@ -24,6 +24,8 @@ import org.apache.druid.common.config.Configs; import org.joda.time.Duration; +import javax.annotation.Nullable; + public class HttpLoadQueuePeonConfig { private static final Duration DEFAULT_LOAD_TIMEOUT = Duration.standardMinutes(15); @@ -35,21 +37,23 @@ public class HttpLoadQueuePeonConfig private final Duration repeatDelay; @JsonProperty - private final int batchSize; + @Nullable + private final Integer batchSize; @JsonCreator public HttpLoadQueuePeonConfig( @JsonProperty("hostTimeout") Duration hostTimeout, @JsonProperty("repeatDelay") Duration repeatDelay, - @JsonProperty("batchSize") Integer batchSize + @JsonProperty("batchSize") @Nullable Integer batchSize ) { this.hostTimeout = Configs.valueOrDefault(hostTimeout, Duration.standardMinutes(5)); this.repeatDelay = Configs.valueOrDefault(repeatDelay, Duration.standardMinutes(1)); - this.batchSize = Configs.valueOrDefault(batchSize, 1); + this.batchSize = batchSize; } - public int getBatchSize() + @Nullable + public Integer getBatchSize() { return batchSize; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 34b37241ae6c..da6c23b689a3 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -25,6 +25,7 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -45,6 +46,7 @@ import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; +import org.apache.druid.server.http.HistoricalLoadingCapabilities; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -123,6 +125,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final Supplier loadingModeSupplier; private final ObjectWriter requestBodyWriter; + private final HistoricalLoadingCapabilities serverCapabilities; public HttpLoadQueuePeon( String baseUrl, @@ -143,6 +146,39 @@ public HttpLoadQueuePeon( this.serverId = baseUrl; this.loadingModeSupplier = loadingModeSupplier; + this.serverCapabilities = fetchSegmentLoadingCapabilities(); + } + + private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() + { + try { + log.trace("Fetching historical capabilities from Server[%s].", new URL(serverId)); + final URL segmentLoadingCapabilitiesURL = new URL( + new URL(serverId), + "druid-internal/v1/segments/segmentLoadingCapabilities" + ); + + BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); + ListenableFuture future = httpClient.go( + new Request(HttpMethod.GET, segmentLoadingCapabilitiesURL) + .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON), + responseHandler, + new Duration(10000) + ); + + if (HttpServletResponse.SC_OK != responseHandler.getStatus()) { + throw new RuntimeException(); + } + + return jsonMapper.readValue( + future.get(), + HistoricalLoadingCapabilities.class + ); + } + catch (Throwable th) { + log.error("Received error while fetching historical capabilities from Server[%s].", serverId); + throw new RuntimeException(th); + } } private void doSegmentManagement() @@ -152,7 +188,17 @@ private void doSegmentManagement() return; } - final int batchSize = config.getBatchSize(); + final SegmentLoadingMode loadingMode = loadingModeSupplier.get(); + int batchSize; + if (config.getBatchSize() != null) { + batchSize = config.getBatchSize(); + } else if (SegmentLoadingMode.TURBO.equals(loadingMode)) { + batchSize = serverCapabilities.getNumTurboLoadingThreads(); + } else if (SegmentLoadingMode.NORMAL.equals(loadingMode)) { + batchSize = serverCapabilities.getNumLoadingThreads(); + } else { + throw DruidException.defensive().build("unsupported loading mode"); + } final List newRequests = new ArrayList<>(batchSize); @@ -186,14 +232,12 @@ private void doSegmentManagement() if (newRequests.isEmpty()) { log.trace( "[%s]Found no load/drop requests. SegmentsToLoad[%d], SegmentsToDrop[%d], batchSize[%d].", - serverId, segmentsToLoad.size(), segmentsToDrop.size(), config.getBatchSize() + serverId, segmentsToLoad.size(), segmentsToDrop.size(), batchSize ); mainLoopInProgress.set(false); return; } - SegmentLoadingMode loadingMode = loadingModeSupplier.get(); - try { log.trace("Sending [%d] load/drop requests to Server[%s] in loadingMode [%s].", newRequests.size(), serverId, loadingMode); final boolean hasLoadRequests = newRequests.stream().anyMatch(r -> r instanceof SegmentChangeRequestLoad); diff --git a/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java b/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java new file mode 100644 index 000000000000..e7a1a38dca2d --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java @@ -0,0 +1,55 @@ +/* + * 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.http; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.error.DruidException; + +public class HistoricalLoadingCapabilities +{ + private final int numLoadingThreads; + private final int numTurboThreads; + + @JsonCreator + public HistoricalLoadingCapabilities( + @JsonProperty("numLoadingThreads") int numLoadingThreads, + @JsonProperty("numTurboThreads") int numTurboThreads + ) + { + if (numLoadingThreads < 1 || numTurboThreads < 1) { + throw DruidException.defensive().build("numLoadingThreads and numTurboThreads must be greater than 0"); + } + this.numLoadingThreads = numLoadingThreads; + this.numTurboThreads = numTurboThreads; + } + + @JsonProperty("numLoadingThreads") + public int getNumLoadingThreads() + { + return numLoadingThreads; + } + + @JsonProperty("numTurboThreads") + public int getNumTurboLoadingThreads() + { + return numTurboThreads; + } +} diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index 434e6c3ef4c5..caad5f86648e 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -31,6 +31,7 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.java.util.emitter.EmittingLogger; +import org.apache.druid.segment.loading.SegmentLoaderConfig; import org.apache.druid.server.coordination.BatchDataSegmentAnnouncer; import org.apache.druid.server.coordination.ChangeRequestHistory; import org.apache.druid.server.coordination.ChangeRequestsSnapshot; @@ -54,6 +55,7 @@ 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.io.IOException; import java.util.List; @@ -328,6 +330,25 @@ public void onFailure(Throwable th) asyncContext.setTimeout(timeout); } + @GET + @Path("/segmentLoadingCapabilities") + @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) + public Response getSegmentLoadingCapabilities( + @Context final HttpServletRequest req + ) throws IOException + { + if (loadDropRequestHandler == null) { + sendErrorResponse(req, HttpServletResponse.SC_NOT_FOUND, "load/drop handler is not available."); + return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); + } + + SegmentLoaderConfig config = loadDropRequestHandler.getSegmentLoaderConfig(); + HistoricalLoadingCapabilities capabilitiesResponse = + new HistoricalLoadingCapabilities(config.getNumLoadingThreads(), config.getNumBootstrapThreads()); + + return Response.status(Response.Status.OK).entity(capabilitiesResponse).build(); + } + private void sendErrorResponse(HttpServletRequest req, int code, String error) throws IOException { AsyncContext asyncContext = req.startAsync(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java index 36cb9c8439f4..d91cb62050a1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorConfigTest.java @@ -101,7 +101,7 @@ public void testLoadQueuePeonConfigDefaultValues() Assert.assertEquals(Duration.standardMinutes(1), config.getRepeatDelay()); Assert.assertEquals(Duration.standardMinutes(5), config.getHostTimeout()); Assert.assertEquals(Duration.standardMinutes(15), config.getLoadTimeout()); - Assert.assertEquals(1, config.getBatchSize()); + Assert.assertNull(config.getBatchSize()); } @Test @@ -118,7 +118,7 @@ public void testLoadQueuePeonConfigOverrideValues() Assert.assertEquals(Duration.standardMinutes(20), config.getRepeatDelay()); Assert.assertEquals(Duration.standardMinutes(10), config.getHostTimeout()); Assert.assertEquals(Duration.standardMinutes(15), config.getLoadTimeout()); - Assert.assertEquals(100, config.getBatchSize()); + Assert.assertEquals(Integer.valueOf(100), config.getBatchSize()); } @Test From 6884fc319f7857997698eb246a4f811c43006071 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sat, 8 Mar 2025 20:29:14 +0530 Subject: [PATCH 16/30] Add test --- .../coordination/SegmentLoadDropHandler.java | 21 ++++- .../coordinator/CoordinatorDynamicConfig.java | 7 ++ .../http/HistoricalLoadingCapabilities.java | 16 ++-- .../HistoricalLoadingCapabilitiesTest.java | 84 +++++++++++++++++++ 4 files changed, 119 insertions(+), 9 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index f0672187b0ca..d45c187c16ec 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -265,7 +265,15 @@ public Collection getSegmentsToDelete() return ImmutableList.copyOf(segmentsToDelete); } - public ListenableFuture> processBatch(List changeRequests, SegmentLoadingMode segmentLoadingMode) + /** + * Process a list of {@link DataSegmentChangeRequest}, invoking + * {@link #processRequest(DataSegmentChangeRequest, SegmentLoadingMode)} for each one. Handles the computation + * asynchronously and returns a future to the result. + */ + public ListenableFuture> processBatch( + List changeRequests, + SegmentLoadingMode segmentLoadingMode + ) { boolean isAnyRequestDone = false; @@ -292,7 +300,16 @@ public ListenableFuture> processBatch(List processRequest(DataSegmentChangeRequest changeRequest, SegmentLoadingMode segmentLoadingMode) + /** + * Process a {@link DataSegmentChangeRequest}, invoking the request's + * {@link DataSegmentChangeRequest#go(DataSegmentChangeHandler, DataSegmentChangeCallback)}. + * The segmentLoadingMode parameter determines the thread pool to use. + * Returns an atomic reference to the segment status. + */ + private AtomicReference processRequest( + DataSegmentChangeRequest changeRequest, + SegmentLoadingMode segmentLoadingMode + ) { synchronized (requestStatusesLock) { AtomicReference status = requestStatuses.getIfPresent(changeRequest); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 373369e9ef00..44c471899cf2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -320,6 +320,13 @@ public boolean getReplicateAfterLoadTimeout() return replicateAfterLoadTimeout; } + /** + * List of historical servers to put into turboloading mode. These historicals will use a larger thread pool to load + * segments. This causes decreases the average time taken to load segments. However, this also means less resources + * given to query threads which causes a drop in query performance. + * + * @return Set of host:port entries + */ @JsonProperty public Set getTurboLoadingNodes() { diff --git a/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java b/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java index e7a1a38dca2d..be16d995c8bb 100644 --- a/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java +++ b/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java @@ -26,19 +26,21 @@ public class HistoricalLoadingCapabilities { private final int numLoadingThreads; - private final int numTurboThreads; + private final int numTurboLoadingThreads; @JsonCreator public HistoricalLoadingCapabilities( @JsonProperty("numLoadingThreads") int numLoadingThreads, - @JsonProperty("numTurboThreads") int numTurboThreads + @JsonProperty("numTurboLoadingThreads") int numTurboLoadingThreads ) { - if (numLoadingThreads < 1 || numTurboThreads < 1) { - throw DruidException.defensive().build("numLoadingThreads and numTurboThreads must be greater than 0"); + if (numLoadingThreads < 1 || numTurboLoadingThreads < 1) { + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("numLoadingThreads and numTurboLoadingThreads must be greater than 0."); } this.numLoadingThreads = numLoadingThreads; - this.numTurboThreads = numTurboThreads; + this.numTurboLoadingThreads = numTurboLoadingThreads; } @JsonProperty("numLoadingThreads") @@ -47,9 +49,9 @@ public int getNumLoadingThreads() return numLoadingThreads; } - @JsonProperty("numTurboThreads") + @JsonProperty("numTurboLoadingThreads") public int getNumTurboLoadingThreads() { - return numTurboThreads; + return numTurboLoadingThreads; } } diff --git a/server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java b/server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java new file mode 100644 index 000000000000..5b82daed8683 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java @@ -0,0 +1,84 @@ +/* + * 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.http; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.exc.ValueInstantiationException; +import org.apache.druid.jackson.DefaultObjectMapper; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +public class HistoricalLoadingCapabilitiesTest +{ + private final ObjectMapper jsonMapper = new DefaultObjectMapper(); + + @Test + public void testSerde() throws Exception + { + HistoricalLoadingCapabilities capabilities = new HistoricalLoadingCapabilities(1, 4); + + HistoricalLoadingCapabilities reread = jsonMapper.readValue(jsonMapper.writeValueAsString(capabilities), HistoricalLoadingCapabilities.class); + + Assert.assertEquals(capabilities.getNumLoadingThreads(), reread.getNumLoadingThreads()); + Assert.assertEquals(capabilities.getNumTurboLoadingThreads(), reread.getNumTurboLoadingThreads()); + } + + @Test + public void testSerdeFromJson() throws JsonProcessingException + { + String json = "{\"numLoadingThreads\":3,\"numTurboLoadingThreads\":5}"; + HistoricalLoadingCapabilities reread = jsonMapper.readValue(json, HistoricalLoadingCapabilities.class); + + Assert.assertEquals(3, reread.getNumLoadingThreads()); + Assert.assertEquals(5, reread.getNumTurboLoadingThreads()); + } + + @Test + public void testSerdeFromEmptyJson() + { + MatcherAssert.assertThat( + Assert.assertThrows(ValueInstantiationException.class, () -> + jsonMapper.readValue("{}", HistoricalLoadingCapabilities.class) + ), + CoreMatchers.allOf( + CoreMatchers.instanceOf(ValueInstantiationException.class), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString("numLoadingThreads and numTurboLoadingThreads must be greater than 0.") + ) + ) + ); + + MatcherAssert.assertThat( + Assert.assertThrows(ValueInstantiationException.class, () -> + jsonMapper.readValue("{\"numLoadingThreads\":3}", HistoricalLoadingCapabilities.class) + ), + CoreMatchers.allOf( + CoreMatchers.instanceOf(ValueInstantiationException.class), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString("numLoadingThreads and numTurboLoadingThreads must be greater than 0.") + ) + ) + ); + } +} From 96f89b6d5c2f77b683765dfcd63957397153c2de Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 9 Mar 2025 20:14:18 +0530 Subject: [PATCH 17/30] Add test --- .../dynamic-configuration-api.md | 11 ++++--- docs/configuration/index.md | 3 +- .../loading/HttpLoadQueuePeon.java | 29 ++++++++++++++++++- .../druid/server/http/SegmentLoadingMode.java | 3 ++ .../loading/HttpLoadQueuePeonTest.java | 4 ++- .../http/CoordinatorDynamicConfigTest.java | 12 ++++++++ 6 files changed, 55 insertions(+), 7 deletions(-) diff --git a/docs/api-reference/dynamic-configuration-api.md b/docs/api-reference/dynamic-configuration-api.md index 90b7028f2478..971aa81d206a 100644 --- a/docs/api-reference/dynamic-configuration-api.md +++ b/docs/api-reference/dynamic-configuration-api.md @@ -105,7 +105,8 @@ Host: http://ROUTER_IP:ROUTER_PORT "maxNonPrimaryReplicantsToLoad": 2147483647, "useRoundRobinSegmentAssignment": true, "smartSegmentLoading": true, - "debugDimensions": null + "debugDimensions": null, + "turboLoadingNodes": [] } ``` @@ -172,7 +173,8 @@ curl "http://ROUTER_IP:ROUTER_PORT/druid/coordinator/v1/config" \ "pauseCoordination": false, "replicateAfterLoadTimeout": false, "maxNonPrimaryReplicantsToLoad": 2147483647, - "useRoundRobinSegmentAssignment": true + "useRoundRobinSegmentAssignment": true, + "turboLoadingNodes": [] }' ``` @@ -203,7 +205,8 @@ Content-Length: 683 "pauseCoordination": false, "replicateAfterLoadTimeout": false, "maxNonPrimaryReplicantsToLoad": 2147483647, - "useRoundRobinSegmentAssignment": true + "useRoundRobinSegmentAssignment": true, + "turboLoadingNodes": [] } ``` @@ -289,7 +292,7 @@ Host: http://ROUTER_IP:ROUTER_PORT "comment": "", "ip": "127.0.0.1" }, - "payload": "{\"millisToWaitBeforeDeleting\":900000,\"maxSegmentsToMove\":5,\"replicantLifetime\":15,\"replicationThrottleLimit\":10,\"balancerComputeThreads\":1,\"killDataSourceWhitelist\":[],\"killPendingSegmentsSkipList\":[],\"maxSegmentsInNodeLoadingQueue\":100,\"decommissioningNodes\":[],\"decommissioningMaxPercentOfMaxSegmentsToMove\":70,\"pauseCoordination\":false,\"replicateAfterLoadTimeout\":false,\"maxNonPrimaryReplicantsToLoad\":2147483647,\"useRoundRobinSegmentAssignment\":true,\"smartSegmentLoading\":true,\"debugDimensions\":null}", + "payload": "{\"millisToWaitBeforeDeleting\":900000,\"maxSegmentsToMove\":5,\"replicantLifetime\":15,\"replicationThrottleLimit\":10,\"balancerComputeThreads\":1,\"killDataSourceWhitelist\":[],\"killPendingSegmentsSkipList\":[],\"maxSegmentsInNodeLoadingQueue\":100,\"decommissioningNodes\":[],\"decommissioningMaxPercentOfMaxSegmentsToMove\":70,\"pauseCoordination\":false,\"replicateAfterLoadTimeout\":false,\"maxNonPrimaryReplicantsToLoad\":2147483647,\"useRoundRobinSegmentAssignment\":true,\"smartSegmentLoading\":true,\"debugDimensions\":null,\"decommissioningNodes\":[]}", "auditTime": "2023-10-03T20:59:51.622Z" } ] diff --git a/docs/configuration/index.md b/docs/configuration/index.md index cd1b8dfc1f9e..8ba138788955 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -885,7 +885,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.coordinator.kill.maxInterval`|The largest interval, as an [ISO 8601 duration](https://en.wikipedia.org/wiki/ISO_8601#Durations), of segments to delete per kill task. Set to zero, e.g. `PT0S`, for unlimited. This only applies when `druid.coordinator.kill.on=true`.|`P30D`| |`druid.coordinator.balancer.strategy`|The [balancing strategy](../design/coordinator.md#balancing-segments-in-a-tier) used by the Coordinator to distribute segments among the Historical servers in a tier. The `cost` strategy distributes segments by minimizing a cost function, `diskNormalized` weights these costs with the disk usage ratios of the servers and `random` distributes segments randomly.|`cost`| |`druid.coordinator.loadqueuepeon.http.repeatDelay`|The start and repeat delay (in milliseconds) for the load queue peon, which manages the load/drop queue of segments for any server.|1 minute| -|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on Historical service.|1| +|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on Historical service. If the value is not provided, automatically sets the value to the `numLoadingThreads` on the historical. | `druid.segmentCache.numLoadingThreads` | |`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator service should act like an Overlord as well. This configuration allows users to simplify a Druid cluster by not having to deploy any standalone Overlord services. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also.|false| |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord services and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL| @@ -953,6 +953,7 @@ The following table shows the dynamic configuration properties for the Coordinat |`decommissioningNodes`|List of Historical servers to decommission. Coordinator will not assign new segments to decommissioning servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `maxSegmentsToMove`.|none| |`pauseCoordination`|Boolean flag for whether or not the Coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` interface. Such duties include: segment balancing, segment compaction, submitting kill tasks for unused segments (if enabled), logging of used segments in the cluster, marking of newly unused or overshadowed segments, matching and execution of load/drop rules for used segments, unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS name nodes with downtime and don't want the Coordinator to be directing Historical nodes to hit the name node with API requests until maintenance is done and the deep store is declared healthy for use again.|false| |`replicateAfterLoadTimeout`|Boolean flag for whether or not additional replication is needed for segments that have failed to load due to the expiry of `druid.coordinator.load.timeout`. If this is set to true, the Coordinator will attempt to replicate the failed segment on a different historical server. This helps improve the segment availability if there are a few slow Historicals in the cluster. However, the slow Historical may still load the segment later and the Coordinator may issue drop requests if the segment is over-replicated.|false| +|`turboLoadingNodes`|List of Historical servers to place in turbo loading mode. This causes the historical to load segments faster at the cost of query performance. For any performance increase, the runtime parameter `druid.coordinator.loadqueuepeon.http.batchSize` must not be configured. |none| ##### Smart segment loading diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index da6c23b689a3..b27a17fb518e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectWriter; +import com.google.common.annotations.VisibleForTesting; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; @@ -149,6 +150,30 @@ public HttpLoadQueuePeon( this.serverCapabilities = fetchSegmentLoadingCapabilities(); } + @VisibleForTesting + HttpLoadQueuePeon( + String baseUrl, + ObjectMapper jsonMapper, + HttpClient httpClient, + HttpLoadQueuePeonConfig config, + ScheduledExecutorService processingExecutor, + ExecutorService callBackExecutor, + Supplier loadingModeSupplier, + HistoricalLoadingCapabilities serverCapabilities + ) + { + this.jsonMapper = jsonMapper; + this.requestBodyWriter = jsonMapper.writerFor(REQUEST_ENTITY_TYPE_REF); + this.httpClient = httpClient; + this.config = config; + this.processingExecutor = processingExecutor; + this.callBackExecutor = callBackExecutor; + + this.serverId = baseUrl; + this.loadingModeSupplier = loadingModeSupplier; + this.serverCapabilities = serverCapabilities; + } + private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() { try { @@ -177,7 +202,9 @@ private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() } catch (Throwable th) { log.error("Received error while fetching historical capabilities from Server[%s].", serverId); - throw new RuntimeException(th); + throw DruidException.forPersona(DruidException.Persona.OPERATOR) + .ofCategory(DruidException.Category.RUNTIME_FAILURE) + .build(th, "Failed to fetch historical capabilities"); } } diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java index 647c390394f5..b3896a540bda 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingMode.java @@ -19,6 +19,9 @@ package org.apache.druid.server.http; +/** + * Determines the threadpool used by the historical to load segments. + */ public enum SegmentLoadingMode { NORMAL, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 89351cd8163d..c4343516a0bb 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -39,6 +39,7 @@ import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; +import org.apache.druid.server.http.HistoricalLoadingCapabilities; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.easymock.EasyMock; @@ -101,7 +102,8 @@ public void setUp() true ), httpClient.callbackExecutor, - () -> SegmentLoadingMode.NORMAL + () -> SegmentLoadingMode.NORMAL, + new HistoricalLoadingCapabilities(1, 1) ); httpLoadQueuePeon.start(); } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index d06d1b40f483..b24f90baaa93 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -545,6 +545,18 @@ public void testTurboLoadingNodes() Assert.assertEquals(SegmentLoadingMode.TURBO, config.getLoadingModeForServer("localhost:8083")); } + @Test + public void testTurboLoadingNodes() + { + CoordinatorDynamicConfig config = CoordinatorDynamicConfig + .builder() + .withTurboLoadingNodes(ImmutableSet.of("localhost:8083")) + .build(); + + Assert.assertEquals(SegmentLoadingMode.NORMAL, config.getLoadingModeForServer("localhost:8082")); + Assert.assertEquals(SegmentLoadingMode.TURBO, config.getLoadingModeForServer("localhost:8083")); + } + @Test public void testEqualsAndHashCode() { From f81e20f2461ce80624b94e97361ccafdfd35c44b Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 9 Mar 2025 20:58:35 +0530 Subject: [PATCH 18/30] Add test --- .../loading/HttpLoadQueuePeon.java | 28 +++++++++++-------- .../SegmentLoadDropHandlerTest.java | 2 +- .../loading/HttpLoadQueuePeonTest.java | 26 ++++++++++++++++- .../http/CoordinatorDynamicConfigTest.java | 12 -------- 4 files changed, 42 insertions(+), 26 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index b27a17fb518e..8339225f312c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -201,10 +201,9 @@ private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() ); } catch (Throwable th) { - log.error("Received error while fetching historical capabilities from Server[%s].", serverId); throw DruidException.forPersona(DruidException.Persona.OPERATOR) .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(th, "Failed to fetch historical capabilities"); + .build(th, "Received error while fetching historical capabilities from Server[%s].", serverId); } } @@ -216,16 +215,7 @@ private void doSegmentManagement() } final SegmentLoadingMode loadingMode = loadingModeSupplier.get(); - int batchSize; - if (config.getBatchSize() != null) { - batchSize = config.getBatchSize(); - } else if (SegmentLoadingMode.TURBO.equals(loadingMode)) { - batchSize = serverCapabilities.getNumTurboLoadingThreads(); - } else if (SegmentLoadingMode.NORMAL.equals(loadingMode)) { - batchSize = serverCapabilities.getNumLoadingThreads(); - } else { - throw DruidException.defensive().build("unsupported loading mode"); - } + int batchSize = calculateBatchSize(loadingMode); final List newRequests = new ArrayList<>(batchSize); @@ -391,6 +381,20 @@ private void logRequestFailure(Throwable t) } } + @VisibleForTesting + int calculateBatchSize(SegmentLoadingMode loadingMode) + { + if (config.getBatchSize() != null) { + return config.getBatchSize(); + } else if (SegmentLoadingMode.TURBO.equals(loadingMode)) { + return serverCapabilities.getNumTurboLoadingThreads(); + } else if (SegmentLoadingMode.NORMAL.equals(loadingMode)) { + return serverCapabilities.getNumLoadingThreads(); + } else { + throw DruidException.defensive().build("unsupported loading mode"); + } + } + private void handleResponseStatus(DataSegmentChangeRequest changeRequest, SegmentChangeStatus status) { changeRequest.go( diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 8932d3c82a0f..0b61b01effc7 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -245,7 +245,7 @@ public void testProcessBatch() throws Exception runnable.run(); } - result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadingMode.NORMAL).get(); + result = handler.processBatch(ImmutableList.of(new SegmentChangeRequestLoad(segment1)), SegmentLoadingMode.TURBO).get(); Assert.assertEquals(SegmentChangeStatus.SUCCESS, result.get(0).getStatus()); Assert.assertEquals(ImmutableList.of(segment1), segmentAnnouncer.getObservedSegments()); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index c4343516a0bb..20073411c223 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -103,7 +103,7 @@ public void setUp() ), httpClient.callbackExecutor, () -> SegmentLoadingMode.NORMAL, - new HistoricalLoadingCapabilities(1, 1) + new HistoricalLoadingCapabilities(1, 3) ); httpLoadQueuePeon.start(); } @@ -333,6 +333,30 @@ public void testLoadRateIsChangedWhenLoadSucceeds() throws InterruptedException ); } + @Test + public void testBatchSize() + { + Assert.assertEquals(10, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.NORMAL)); + + httpLoadQueuePeon = new HttpLoadQueuePeon( + "http://dummy:4000", + MAPPER, + httpClient, + new HttpLoadQueuePeonConfig(null, null, null), + new WrappingScheduledExecutorService( + "HttpLoadQueuePeonTest-%s", + httpClient.processingExecutor, + true + ), + httpClient.callbackExecutor, + () -> SegmentLoadingMode.NORMAL, + new HistoricalLoadingCapabilities(1, 3) + ); + + Assert.assertEquals(1, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.NORMAL)); + Assert.assertEquals(3, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.TURBO)); + } + private LoadPeonCallback markSegmentProcessed(DataSegment segment) { return success -> httpClient.processedSegments.add(segment); diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index b24f90baaa93..d06d1b40f483 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -545,18 +545,6 @@ public void testTurboLoadingNodes() Assert.assertEquals(SegmentLoadingMode.TURBO, config.getLoadingModeForServer("localhost:8083")); } - @Test - public void testTurboLoadingNodes() - { - CoordinatorDynamicConfig config = CoordinatorDynamicConfig - .builder() - .withTurboLoadingNodes(ImmutableSet.of("localhost:8083")) - .build(); - - Assert.assertEquals(SegmentLoadingMode.NORMAL, config.getLoadingModeForServer("localhost:8082")); - Assert.assertEquals(SegmentLoadingMode.TURBO, config.getLoadingModeForServer("localhost:8083")); - } - @Test public void testEqualsAndHashCode() { From 4a14236229e60ee627540fac985b4280907c9da2 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Sun, 9 Mar 2025 21:21:47 +0530 Subject: [PATCH 19/30] Fix tests --- .../TestSegmentLoadingHttpClient.java | 27 +++++++++++++++++++ 1 file changed, 27 insertions(+) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java index 5caa90d8dfdb..28677d57fc08 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java @@ -24,6 +24,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.SettableFuture; 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.HttpResponseHandler; @@ -32,6 +33,7 @@ import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeStatus; +import org.apache.druid.server.http.HistoricalLoadingCapabilities; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -79,12 +81,16 @@ public ListenableFuture go( } @Override + @SuppressWarnings("unchecked") public ListenableFuture go( Request request, HttpResponseHandler handler, Duration readTimeout ) { + if (request.getUrl().toString().contains("/segmentLoadingCapabilities")) { + return getCapabilities(handler); + } return executorService.submit(() -> processRequest(request, handler)); } @@ -143,6 +149,27 @@ private List processRequest( .collect(Collectors.toList()); } + private ListenableFuture getCapabilities(HttpResponseHandler handler) + { + try { + // Set response content and status + final HttpResponse response = new DefaultHttpResponse(HttpVersion.HTTP_1_1, HttpResponseStatus.OK); + response.setContent(ChannelBuffers.EMPTY_BUFFER); + handler.handleResponse(response, NOOP_TRAFFIC_COP); + + // Serialize + SettableFuture future = SettableFuture.create(); + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + objectMapper.writeValue(baos, new HistoricalLoadingCapabilities(1, 1)); + future.set(new ByteArrayInputStream(baos.toByteArray())); + } + return future; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + /** * Processes each DataSegmentChangeRequest using the handler. */ From a502cef450b7e36e294046b752dfdfe9bba31065 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 17 Mar 2025 12:21:49 +0530 Subject: [PATCH 20/30] Fix ITs --- .../coordinator/loading/HttpLoadQueuePeon.java | 16 ++++++++++++---- 1 file changed, 12 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 8339225f312c..e725dbaacc69 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -184,19 +184,27 @@ private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() ); BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); - ListenableFuture future = httpClient.go( + InputStream stream = httpClient.go( new Request(HttpMethod.GET, segmentLoadingCapabilitiesURL) .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON), responseHandler, new Duration(10000) - ); + ).get(); + + if (HttpServletResponse.SC_NOT_FOUND == responseHandler.getStatus()) { + log.info( + "Historical capabilities endpoint not found at server[%s]. Using default values.", + new URL(serverId) + ); + return new HistoricalLoadingCapabilities(1, 1); + } if (HttpServletResponse.SC_OK != responseHandler.getStatus()) { - throw new RuntimeException(); + throw new RE("Error when fetching capabilities from server[%s]. Received [%s]", new URL(serverId), responseHandler.getStatus()); } return jsonMapper.readValue( - future.get(), + stream, HistoricalLoadingCapabilities.class ); } From ed7815a9b17e0b30fabf3a917543078b3ddc64de Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 20 Mar 2025 11:58:23 +0530 Subject: [PATCH 21/30] Update docs --- docs/configuration/index.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 1ab1678e2f37..6ceb09284866 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -885,7 +885,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.coordinator.kill.maxInterval`|The largest interval, as an [ISO 8601 duration](https://en.wikipedia.org/wiki/ISO_8601#Durations), of segments to delete per kill task. Set to zero, e.g. `PT0S`, for unlimited. This only applies when `druid.coordinator.kill.on=true`.|`P30D`| |`druid.coordinator.balancer.strategy`|The [balancing strategy](../design/coordinator.md#balancing-segments-in-a-tier) used by the Coordinator to distribute segments among the Historical servers in a tier. The `cost` strategy distributes segments by minimizing a cost function, `diskNormalized` weights these costs with the disk usage ratios of the servers and `random` distributes segments randomly.|`cost`| |`druid.coordinator.loadqueuepeon.http.repeatDelay`|The start and repeat delay (in milliseconds) for the load queue peon, which manages the load/drop queue of segments for any server.|1 minute| -|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than `druid.segmentCache.numLoadingThreads` config on Historical service. If the value is not provided, automatically sets the value to the `numLoadingThreads` on the historical. | `druid.segmentCache.numLoadingThreads` | +|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than or equal to the `druid.segmentCache.numLoadingThreads` config on Historical service. If the value is not provided, the coordinator automatically dynamically configures the value to the `numLoadingThreads` available on the historical on the historical. | `druid.segmentCache.numLoadingThreads` | |`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator service should act like an Overlord as well. This configuration allows users to simplify a Druid cluster by not having to deploy any standalone Overlord services. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also.|false| |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord services and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL| @@ -953,7 +953,7 @@ The following table shows the dynamic configuration properties for the Coordinat |`decommissioningNodes`|List of Historical servers to decommission. Coordinator will not assign new segments to decommissioning servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `maxSegmentsToMove`.|none| |`pauseCoordination`|Boolean flag for whether or not the Coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` interface. Such duties include: segment balancing, segment compaction, submitting kill tasks for unused segments (if enabled), logging of used segments in the cluster, marking of newly unused or overshadowed segments, matching and execution of load/drop rules for used segments, unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS name nodes with downtime and don't want the Coordinator to be directing Historical nodes to hit the name node with API requests until maintenance is done and the deep store is declared healthy for use again.|false| |`replicateAfterLoadTimeout`|Boolean flag for whether or not additional replication is needed for segments that have failed to load due to the expiry of `druid.coordinator.load.timeout`. If this is set to true, the Coordinator will attempt to replicate the failed segment on a different historical server. This helps improve the segment availability if there are a few slow Historicals in the cluster. However, the slow Historical may still load the segment later and the Coordinator may issue drop requests if the segment is over-replicated.|false| -|`turboLoadingNodes`|List of Historical servers to place in turbo loading mode. This causes the historical to load segments faster at the cost of query performance. For any performance increase, the runtime parameter `druid.coordinator.loadqueuepeon.http.batchSize` must not be configured. |none| +|`turboLoadingNodes`| List of Historical servers to place in turbo loading mode. These Historicals will load segments more quickly but at the cost of query performance. `turboLoadingNodes` requires dynamic configuration of batchSize to provide performance improvements. Hence, the runtime parameter `druid.coordinator.loadqueuepeon.http.batchSize` must not be configured. |none| ##### Smart segment loading From cf71484b77dc7327f62d848c5d801fe7a5974a5c Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 20 Mar 2025 12:30:09 +0530 Subject: [PATCH 22/30] Split executors --- .../coordination/SegmentLoadDropHandler.java | 41 +++++++++++++------ .../SegmentLoadDropHandlerTest.java | 8 ++-- 2 files changed, 34 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index d45c187c16ec..6336036cef04 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -46,8 +46,11 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentSkipListSet; +import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -65,8 +68,9 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final SegmentLoaderConfig config; private final DataSegmentAnnouncer announcer; private final SegmentManager segmentManager; - private final ScheduledExecutorService exec; - private final ScheduledExecutorService turboExec; + private final ScheduledExecutorService scheduledExecutorService; + private final ThreadPoolExecutor standardExec; + private final ThreadPoolExecutor turboExec; private final ConcurrentSkipListSet segmentsToDelete; @@ -91,13 +95,21 @@ public SegmentLoadDropHandler( config, announcer, segmentManager, - Executors.newScheduledThreadPool( - config.getNumLoadingThreads(), + new ThreadPoolExecutor( + config.getNumLoadingThreads(), config.getNumLoadingThreads(), + 60L, TimeUnit.SECONDS, + new SynchronousQueue<>(), Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") ), - Executors.newScheduledThreadPool( - config.getNumBootstrapThreads(), + new ThreadPoolExecutor( + config.getNumBootstrapThreads(), config.getNumBootstrapThreads(), + 60L, TimeUnit.SECONDS, + new SynchronousQueue<>(), Execs.makeThreadFactory("TurboDataSegmentChangeHandler-%s") + ), + Executors.newScheduledThreadPool( + config.getNumLoadingThreads(), + Execs.makeThreadFactory("ScheduledDataSegmentChangeHandler-%s") ) ); } @@ -107,15 +119,20 @@ public SegmentLoadDropHandler( SegmentLoaderConfig config, DataSegmentAnnouncer announcer, SegmentManager segmentManager, - ScheduledExecutorService exec, - ScheduledExecutorService turboExec + ThreadPoolExecutor standardExec, + ThreadPoolExecutor turboExec, + ScheduledExecutorService scheduledExecutorService ) { this.config = config; this.announcer = announcer; this.segmentManager = segmentManager; - this.exec = exec; + this.standardExec = standardExec; this.turboExec = turboExec; + this.scheduledExecutorService = scheduledExecutorService; + + this.standardExec.allowCoreThreadTimeOut(true); + this.turboExec.allowCoreThreadTimeOut(true); this.segmentsToDelete = new ConcurrentSkipListSet<>(); requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); @@ -235,7 +252,7 @@ void removeSegment( "Completely removing segment[%s] in [%,d]ms.", segment.getId(), config.getDropSegmentDelayMillis() ); - getExecutorService(segmentLoadingMode).schedule( + scheduledExecutorService.schedule( runnable, config.getDropSegmentDelayMillis(), TimeUnit.MILLISECONDS @@ -433,13 +450,13 @@ public boolean cancel(boolean interruptIfRunning) } } - private ScheduledExecutorService getExecutorService(SegmentLoadingMode loadingMode) + private ExecutorService getExecutorService(SegmentLoadingMode loadingMode) { switch (loadingMode) { case TURBO: return turboExec; case NORMAL: - return exec; + return standardExec; default: throw DruidException.defensive("Unknown execution mode [%s]", loadingMode); } diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index 0b61b01effc7..b7cb1831329b 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -49,6 +49,7 @@ import java.util.Map; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import static org.apache.druid.server.TestSegmentUtils.makeSegment; @@ -107,7 +108,7 @@ public int getDropSegmentDelayMillis() scheduledExecutorFactory = (corePoolSize, nameFormat) -> { // Override normal behavior by adding the runnable to a list so that you can make sure - // all the shceduled runnables are executed by explicitly calling run() on each item in the list + // all the scheduled runnables are executed by explicitly calling run() on each item in the list return new ScheduledThreadPoolExecutor(corePoolSize, Execs.makeThreadFactory(nameFormat)) { @Override @@ -421,8 +422,9 @@ private SegmentLoadDropHandler initSegmentLoadDropHandler( config, segmentAnnouncer, segmentManager, - scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - scheduledExecutorFactory.create(5, "TurboSegmentLoadDropHandlerTest-[%d]") + (ThreadPoolExecutor) scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), + (ThreadPoolExecutor) scheduledExecutorFactory.create(5, "TurboSegmentLoadDropHandlerTest-[%d]"), + scheduledExecutorFactory.create(5, "ScheduledSegmentLoadDropHandlerTest-[%d]") ); } } From e24c5e028a4562490bbbffbe15bd629a0fea77cf Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Thu, 20 Mar 2025 12:41:54 +0530 Subject: [PATCH 23/30] Fix typo --- docs/configuration/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 6ceb09284866..8104b840cd5a 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -885,7 +885,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.coordinator.kill.maxInterval`|The largest interval, as an [ISO 8601 duration](https://en.wikipedia.org/wiki/ISO_8601#Durations), of segments to delete per kill task. Set to zero, e.g. `PT0S`, for unlimited. This only applies when `druid.coordinator.kill.on=true`.|`P30D`| |`druid.coordinator.balancer.strategy`|The [balancing strategy](../design/coordinator.md#balancing-segments-in-a-tier) used by the Coordinator to distribute segments among the Historical servers in a tier. The `cost` strategy distributes segments by minimizing a cost function, `diskNormalized` weights these costs with the disk usage ratios of the servers and `random` distributes segments randomly.|`cost`| |`druid.coordinator.loadqueuepeon.http.repeatDelay`|The start and repeat delay (in milliseconds) for the load queue peon, which manages the load/drop queue of segments for any server.|1 minute| -|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than or equal to the `druid.segmentCache.numLoadingThreads` config on Historical service. If the value is not provided, the coordinator automatically dynamically configures the value to the `numLoadingThreads` available on the historical on the historical. | `druid.segmentCache.numLoadingThreads` | +|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than or equal to the `druid.segmentCache.numLoadingThreads` config on Historical service. If the value is not provided, the coordinator automatically dynamically configures the value to the `numLoadingThreads` available on the Historical. | `druid.segmentCache.numLoadingThreads` | |`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator service should act like an Overlord as well. This configuration allows users to simplify a Druid cluster by not having to deploy any standalone Overlord services. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also.|false| |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord services and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL| From 844c2f990d6f31217fd9f08d653af1f83f71bd25 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 21 Mar 2025 13:28:01 +0530 Subject: [PATCH 24/30] Address review comments --- docs/configuration/index.md | 4 +- .../coordination/SegmentLoadDropHandler.java | 66 +++++++------------ .../coordinator/CoordinatorDynamicConfig.java | 7 +- .../loading/HttpLoadQueuePeon.java | 36 ++++------ .../server/http/SegmentListerResource.java | 17 +++-- ...s.java => SegmentLoadingCapabilities.java} | 4 +- .../loading/HttpLoadQueuePeonTest.java | 23 ++----- .../CoordinatorSimulationBuilder.java | 7 +- .../TestSegmentLoadingHttpClient.java | 4 +- ...va => SegmentLoadingCapabilitiesTest.java} | 12 ++-- 10 files changed, 68 insertions(+), 112 deletions(-) rename server/src/main/java/org/apache/druid/server/http/{HistoricalLoadingCapabilities.java => SegmentLoadingCapabilities.java} (95%) rename server/src/test/java/org/apache/druid/server/http/{HistoricalLoadingCapabilitiesTest.java => SegmentLoadingCapabilitiesTest.java} (82%) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 8104b840cd5a..687e1d688a67 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -885,7 +885,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |`druid.coordinator.kill.maxInterval`|The largest interval, as an [ISO 8601 duration](https://en.wikipedia.org/wiki/ISO_8601#Durations), of segments to delete per kill task. Set to zero, e.g. `PT0S`, for unlimited. This only applies when `druid.coordinator.kill.on=true`.|`P30D`| |`druid.coordinator.balancer.strategy`|The [balancing strategy](../design/coordinator.md#balancing-segments-in-a-tier) used by the Coordinator to distribute segments among the Historical servers in a tier. The `cost` strategy distributes segments by minimizing a cost function, `diskNormalized` weights these costs with the disk usage ratios of the servers and `random` distributes segments randomly.|`cost`| |`druid.coordinator.loadqueuepeon.http.repeatDelay`|The start and repeat delay (in milliseconds) for the load queue peon, which manages the load/drop queue of segments for any server.|1 minute| -|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than or equal to the `druid.segmentCache.numLoadingThreads` config on Historical service. If the value is not provided, the coordinator automatically dynamically configures the value to the `numLoadingThreads` available on the Historical. | `druid.segmentCache.numLoadingThreads` | +|`druid.coordinator.loadqueuepeon.http.batchSize`|Number of segment load/drop requests to batch in one HTTP request. Note that it must be smaller than or equal to the `druid.segmentCache.numLoadingThreads` config on Historical service. If this value is not configured, the coordinator uses the value of the `numLoadingThreads` for the respective server. | `druid.segmentCache.numLoadingThreads` | |`druid.coordinator.asOverlord.enabled`|Boolean value for whether this Coordinator service should act like an Overlord as well. This configuration allows users to simplify a Druid cluster by not having to deploy any standalone Overlord services. If set to true, then Overlord console is available at `http://coordinator-host:port/console.html` and be sure to set `druid.coordinator.asOverlord.overlordService` also.|false| |`druid.coordinator.asOverlord.overlordService`| Required, if `druid.coordinator.asOverlord.enabled` is `true`. This must be same value as `druid.service` on standalone Overlord services and `druid.selectors.indexing.serviceName` on Middle Managers.|NULL| @@ -953,7 +953,7 @@ The following table shows the dynamic configuration properties for the Coordinat |`decommissioningNodes`|List of Historical servers to decommission. Coordinator will not assign new segments to decommissioning servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by `maxSegmentsToMove`.|none| |`pauseCoordination`|Boolean flag for whether or not the Coordinator should execute its various duties of coordinating the cluster. Setting this to true essentially pauses all coordination work while allowing the API to remain up. Duties that are paused include all classes that implement the `CoordinatorDuty` interface. Such duties include: segment balancing, segment compaction, submitting kill tasks for unused segments (if enabled), logging of used segments in the cluster, marking of newly unused or overshadowed segments, matching and execution of load/drop rules for used segments, unloading segments that are no longer marked as used from Historical servers. An example of when an admin may want to pause coordination would be if they are doing deep storage maintenance on HDFS name nodes with downtime and don't want the Coordinator to be directing Historical nodes to hit the name node with API requests until maintenance is done and the deep store is declared healthy for use again.|false| |`replicateAfterLoadTimeout`|Boolean flag for whether or not additional replication is needed for segments that have failed to load due to the expiry of `druid.coordinator.load.timeout`. If this is set to true, the Coordinator will attempt to replicate the failed segment on a different historical server. This helps improve the segment availability if there are a few slow Historicals in the cluster. However, the slow Historical may still load the segment later and the Coordinator may issue drop requests if the segment is over-replicated.|false| -|`turboLoadingNodes`| List of Historical servers to place in turbo loading mode. These Historicals will load segments more quickly but at the cost of query performance. `turboLoadingNodes` requires dynamic configuration of batchSize to provide performance improvements. Hence, the runtime parameter `druid.coordinator.loadqueuepeon.http.batchSize` must not be configured. |none| +|`turboLoadingNodes`| List of Historical servers to place in turbo loading mode. These servers use a larger thread-pool to load segments faster but at the cost of query performance. For servers specified in `turboLoadingNodes`, `druid.coordinator.loadqueuepeon.http.batchSize` is ignored and the coordinator uses the value of the respective `numLoadingThreads` instead. |none| ##### Smart segment loading diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 6336036cef04..f2a95de79251 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -27,7 +27,6 @@ import com.google.common.util.concurrent.AbstractFuture; import com.google.common.util.concurrent.ListenableFuture; import com.google.inject.Inject; -import org.apache.druid.error.DruidException; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -69,8 +68,8 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final DataSegmentAnnouncer announcer; private final SegmentManager segmentManager; private final ScheduledExecutorService scheduledExecutorService; - private final ThreadPoolExecutor standardExec; - private final ThreadPoolExecutor turboExec; + private final ThreadPoolExecutor normalLoadExec; + private final ThreadPoolExecutor turboLoadExec; private final ConcurrentSkipListSet segmentsToDelete; @@ -96,16 +95,20 @@ public SegmentLoadDropHandler( announcer, segmentManager, new ThreadPoolExecutor( - config.getNumLoadingThreads(), config.getNumLoadingThreads(), - 60L, TimeUnit.SECONDS, + config.getNumLoadingThreads(), + config.getNumLoadingThreads(), + 60L, + TimeUnit.SECONDS, new SynchronousQueue<>(), - Execs.makeThreadFactory("SimpleDataSegmentChangeHandler-%s") + Execs.makeThreadFactory("SegmentLoadDropHandler-normal-%s") ), new ThreadPoolExecutor( - config.getNumBootstrapThreads(), config.getNumBootstrapThreads(), - 60L, TimeUnit.SECONDS, + config.getNumBootstrapThreads(), + config.getNumBootstrapThreads(), + 60L, + TimeUnit.SECONDS, new SynchronousQueue<>(), - Execs.makeThreadFactory("TurboDataSegmentChangeHandler-%s") + Execs.makeThreadFactory("SegmentLoadDropHandler-turbo-%s") ), Executors.newScheduledThreadPool( config.getNumLoadingThreads(), @@ -119,20 +122,20 @@ public SegmentLoadDropHandler( SegmentLoaderConfig config, DataSegmentAnnouncer announcer, SegmentManager segmentManager, - ThreadPoolExecutor standardExec, - ThreadPoolExecutor turboExec, + ThreadPoolExecutor normalLoadExec, + ThreadPoolExecutor turboLoadExec, ScheduledExecutorService scheduledExecutorService ) { this.config = config; this.announcer = announcer; this.segmentManager = segmentManager; - this.standardExec = standardExec; - this.turboExec = turboExec; + this.normalLoadExec = normalLoadExec; + this.turboLoadExec = turboLoadExec; this.scheduledExecutorService = scheduledExecutorService; - this.standardExec.allowCoreThreadTimeOut(true); - this.turboExec.allowCoreThreadTimeOut(true); + this.normalLoadExec.allowCoreThreadTimeOut(true); + this.turboLoadExec.allowCoreThreadTimeOut(true); this.segmentsToDelete = new ConcurrentSkipListSet<>(); requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); @@ -150,11 +153,6 @@ public Map getRowCountDistributionPerDataso @Override public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) - { - addSegment(segment, callback, SegmentLoadingMode.NORMAL); - } - - public void addSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback, SegmentLoadingMode loadingMode) { SegmentChangeStatus result = null; try { @@ -180,7 +178,7 @@ each time when addSegment() is called, it has to wait for the lock in order to m segmentManager.loadSegment(segment); } catch (Exception e) { - removeSegment(segment, DataSegmentChangeCallback.NOOP, false, loadingMode); + removeSegment(segment, DataSegmentChangeCallback.NOOP, false); throw new SegmentLoadingException(e, "Exception loading segment[%s]", segment.getId()); } try { @@ -210,21 +208,14 @@ each time when addSegment() is called, it has to wait for the lock in order to m @Override public void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback) { - removeSegment(segment, callback, true, SegmentLoadingMode.NORMAL); - } - - @VisibleForTesting - void removeSegment(DataSegment segment, @Nullable DataSegmentChangeCallback callback, boolean scheduleDrop) - { - removeSegment(segment, callback, scheduleDrop, SegmentLoadingMode.NORMAL); + removeSegment(segment, callback, true); } @VisibleForTesting void removeSegment( final DataSegment segment, @Nullable final DataSegmentChangeCallback callback, - final boolean scheduleDrop, - final SegmentLoadingMode segmentLoadingMode + final boolean scheduleDrop ) { SegmentChangeStatus result = null; @@ -346,8 +337,7 @@ public void addSegment( getExecutorService(segmentLoadingMode).submit( () -> SegmentLoadDropHandler.this.addSegment( ((SegmentChangeRequestLoad) changeRequest).getSegment(), - () -> resolveWaitingFutures(), - segmentLoadingMode + () -> resolveWaitingFutures() ) ); } @@ -362,8 +352,7 @@ public void removeSegment( SegmentLoadDropHandler.this.removeSegment( ((SegmentChangeRequestDrop) changeRequest).getSegment(), () -> resolveWaitingFutures(), - true, - segmentLoadingMode + true ); } }, @@ -452,14 +441,7 @@ public boolean cancel(boolean interruptIfRunning) private ExecutorService getExecutorService(SegmentLoadingMode loadingMode) { - switch (loadingMode) { - case TURBO: - return turboExec; - case NORMAL: - return standardExec; - default: - throw DruidException.defensive("Unknown execution mode [%s]", loadingMode); - } + return loadingMode == SegmentLoadingMode.TURBO ? turboLoadExec : normalLoadExec; } public SegmentLoaderConfig getSegmentLoaderConfig() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java index 44c471899cf2..806b6ebbee1e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; +import org.apache.druid.common.config.Configs; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.error.InvalidInput; import org.apache.druid.server.coordinator.duty.KillUnusedSegments; @@ -166,7 +167,7 @@ public CoordinatorDynamicConfig( ); this.debugDimensions = debugDimensions; this.validDebugDimensions = validateDebugDimensions(debugDimensions); - this.turboLoadingNodes = parseJsonStringOrArray(turboLoadingNodes); + this.turboLoadingNodes = Configs.valueOrDefault(turboLoadingNodes, Set.of()); } private Map validateDebugDimensions(Map debugDimensions) @@ -321,9 +322,9 @@ public boolean getReplicateAfterLoadTimeout() } /** - * List of historical servers to put into turboloading mode. These historicals will use a larger thread pool to load + * List of servers to put in turbo-loading mode. These servers will use a larger thread pool to load * segments. This causes decreases the average time taken to load segments. However, this also means less resources - * given to query threads which causes a drop in query performance. + * available to query threads which may cause a drop in query performance. * * @return Set of host:port entries */ diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 650e42fe5520..deb88a1f51dd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -47,7 +47,7 @@ import org.apache.druid.server.coordinator.stats.Dimension; import org.apache.druid.server.coordinator.stats.RowKey; import org.apache.druid.server.coordinator.stats.Stats; -import org.apache.druid.server.http.HistoricalLoadingCapabilities; +import org.apache.druid.server.http.SegmentLoadingCapabilities; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; import org.jboss.netty.handler.codec.http.HttpHeaders; @@ -57,7 +57,6 @@ import javax.servlet.http.HttpServletResponse; import javax.ws.rs.core.MediaType; import java.io.InputStream; -import java.net.MalformedURLException; import java.net.URL; import java.util.ArrayList; import java.util.Collections; @@ -126,7 +125,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon private final Supplier loadingModeSupplier; private final ObjectWriter requestBodyWriter; - private final HistoricalLoadingCapabilities serverCapabilities; + private final SegmentLoadingCapabilities serverCapabilities; public HttpLoadQueuePeon( String baseUrl, @@ -156,10 +155,10 @@ public HttpLoadQueuePeon( ObjectMapper jsonMapper, HttpClient httpClient, HttpLoadQueuePeonConfig config, + Supplier loadingModeSupplier, ScheduledExecutorService processingExecutor, ExecutorService callBackExecutor, - Supplier loadingModeSupplier, - HistoricalLoadingCapabilities serverCapabilities + SegmentLoadingCapabilities serverCapabilities ) { this.jsonMapper = jsonMapper; @@ -174,13 +173,12 @@ public HttpLoadQueuePeon( this.serverCapabilities = serverCapabilities; } - private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() + private SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() { try { - log.trace("Fetching historical capabilities from Server[%s].", new URL(serverId)); final URL segmentLoadingCapabilitiesURL = new URL( new URL(serverId), - "druid-internal/v1/segments/segmentLoadingCapabilities" + "druid-internal/v1/segments/loadCapabilities" ); BytesAccumulatingResponseHandler responseHandler = new BytesAccumulatingResponseHandler(); @@ -192,26 +190,23 @@ private HistoricalLoadingCapabilities fetchSegmentLoadingCapabilities() ).get(); if (HttpServletResponse.SC_NOT_FOUND == responseHandler.getStatus()) { - log.info( + log.warn( "Historical capabilities endpoint not found at server[%s]. Using default values.", new URL(serverId) ); - return new HistoricalLoadingCapabilities(1, 1); - } - - if (HttpServletResponse.SC_OK != responseHandler.getStatus()) { + return new SegmentLoadingCapabilities(1, 1); + } else if (HttpServletResponse.SC_OK != responseHandler.getStatus()) { + log.makeAlert("Error when fetching capabilities from server[%s]. Received [%s]", new URL(serverId), responseHandler.getStatus()); throw new RE("Error when fetching capabilities from server[%s]. Received [%s]", new URL(serverId), responseHandler.getStatus()); } return jsonMapper.readValue( stream, - HistoricalLoadingCapabilities.class + SegmentLoadingCapabilities.class ); } catch (Throwable th) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.RUNTIME_FAILURE) - .build(th, "Received error while fetching historical capabilities from Server[%s].", serverId); + throw new RE(th, "Received error while fetching historical capabilities from Server[%s].", serverId); } } @@ -223,7 +218,7 @@ private void doSegmentManagement() } final SegmentLoadingMode loadingMode = loadingModeSupplier.get(); - int batchSize = calculateBatchSize(loadingMode); + final int batchSize = calculateBatchSize(loadingMode); final List newRequests = new ArrayList<>(batchSize); @@ -264,7 +259,7 @@ private void doSegmentManagement() } try { - log.trace("Sending [%d] load/drop requests to Server[%s] in loadingMode [%s].", newRequests.size(), serverId, loadingMode); + log.trace("Sending [%d] load/drop requests to Server[%s] in loadingMode[%s].", newRequests.size(), serverId, loadingMode); final boolean hasLoadRequests = newRequests.stream().anyMatch(r -> r instanceof SegmentChangeRequestLoad); if (hasLoadRequests && !loadingRateTracker.isLoadingBatch()) { loadingRateTracker.markBatchLoadingStarted(); @@ -380,9 +375,6 @@ private void logRequestFailure(Throwable t) processingExecutor ); } - catch (MalformedURLException ex) { - throw new RuntimeException(ex); - } catch (Throwable th) { log.error(th, "Error sending load/drop request to [%s].", serverId); mainLoopInProgress.set(false); diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index caad5f86648e..48e037271995 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -221,10 +221,10 @@ public void onFailure(Throwable th) * This endpoint is used by HttpLoadQueuePeon to assign segment load/drop requests batch. This endpoint makes the * client wait till one of the following events occur. Note that this is implemented using async IO so no jetty * threads are held while in wait. - *
- * (1) Given timeout elapses. - * (2) Some load/drop request completed. - *
+ *
    + *
  1. Given timeout elapses.
  2. + *
  3. Some load/drop request completed.
  4. + *
* It returns a map of "load/drop request -> SUCCESS/FAILED/PENDING status" for each request in the batch. */ @POST @@ -331,20 +331,19 @@ public void onFailure(Throwable th) } @GET - @Path("/segmentLoadingCapabilities") + @Path("/loadCapabilities") @Produces({MediaType.APPLICATION_JSON, SmileMediaTypes.APPLICATION_JACKSON_SMILE}) public Response getSegmentLoadingCapabilities( @Context final HttpServletRequest req - ) throws IOException + ) { if (loadDropRequestHandler == null) { - sendErrorResponse(req, HttpServletResponse.SC_NOT_FOUND, "load/drop handler is not available."); return Response.status(Response.Status.INTERNAL_SERVER_ERROR).build(); } SegmentLoaderConfig config = loadDropRequestHandler.getSegmentLoaderConfig(); - HistoricalLoadingCapabilities capabilitiesResponse = - new HistoricalLoadingCapabilities(config.getNumLoadingThreads(), config.getNumBootstrapThreads()); + SegmentLoadingCapabilities capabilitiesResponse = + new SegmentLoadingCapabilities(config.getNumLoadingThreads(), config.getNumBootstrapThreads()); return Response.status(Response.Status.OK).entity(capabilitiesResponse).build(); } diff --git a/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java similarity index 95% rename from server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java rename to server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java index be16d995c8bb..8c04ed5e8ba8 100644 --- a/server/src/main/java/org/apache/druid/server/http/HistoricalLoadingCapabilities.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java @@ -23,13 +23,13 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.error.DruidException; -public class HistoricalLoadingCapabilities +public class SegmentLoadingCapabilities { private final int numLoadingThreads; private final int numTurboLoadingThreads; @JsonCreator - public HistoricalLoadingCapabilities( + public SegmentLoadingCapabilities( @JsonProperty("numLoadingThreads") int numLoadingThreads, @JsonProperty("numTurboLoadingThreads") int numTurboLoadingThreads ) diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 20073411c223..1803811e105e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.http.client.HttpClient; @@ -34,15 +33,13 @@ import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeStatus; -import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; import org.apache.druid.server.coordinator.CreateDataSegments; import org.apache.druid.server.coordinator.config.HttpLoadQueuePeonConfig; import org.apache.druid.server.coordinator.simulate.BlockingExecutorService; import org.apache.druid.server.coordinator.simulate.WrappingScheduledExecutorService; -import org.apache.druid.server.http.HistoricalLoadingCapabilities; +import org.apache.druid.server.http.SegmentLoadingCapabilities; import org.apache.druid.server.http.SegmentLoadingMode; import org.apache.druid.timeline.DataSegment; -import org.easymock.EasyMock; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -61,7 +58,6 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import java.util.stream.Collectors; @@ -82,28 +78,19 @@ public class HttpLoadQueuePeonTest public void setUp() { httpClient = new TestHttpClient(); - JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); - EasyMock.expect( - configManager.watch( - EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), - EasyMock.anyObject(Class.class), - EasyMock.anyObject() - ) - ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); - EasyMock.replay(configManager); httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", MAPPER, httpClient, new HttpLoadQueuePeonConfig(null, null, 10), + () -> SegmentLoadingMode.NORMAL, new WrappingScheduledExecutorService( "HttpLoadQueuePeonTest-%s", httpClient.processingExecutor, true ), httpClient.callbackExecutor, - () -> SegmentLoadingMode.NORMAL, - new HistoricalLoadingCapabilities(1, 3) + new SegmentLoadingCapabilities(1, 3) ); httpLoadQueuePeon.start(); } @@ -343,14 +330,14 @@ public void testBatchSize() MAPPER, httpClient, new HttpLoadQueuePeonConfig(null, null, null), + () -> SegmentLoadingMode.NORMAL, new WrappingScheduledExecutorService( "HttpLoadQueuePeonTest-%s", httpClient.processingExecutor, true ), httpClient.callbackExecutor, - () -> SegmentLoadingMode.NORMAL, - new HistoricalLoadingCapabilities(1, 3) + new SegmentLoadingCapabilities(1, 3) ); Assert.assertEquals(1, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.NORMAL)); 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 3bc22d3da02d..938b6b09b1a0 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 @@ -457,11 +457,6 @@ private Environment( JacksonConfigManager jacksonConfigManager = mockConfigManager(); setDynamicConfig(dynamicConfig); - CoordinatorConfigManager coordinatorConfigManager = new CoordinatorConfigManager( - jacksonConfigManager, - null, - null - ); this.loadQueueTaskMaster = new LoadQueueTaskMaster( OBJECT_MAPPER, @@ -469,7 +464,7 @@ private Environment( executorFactory.create(1, ExecutorFactory.LOAD_CALLBACK_EXECUTOR), coordinatorConfig.getHttpLoadQueuePeonConfig(), httpClient, - coordinatorConfigManager::getCurrentDynamicConfig + () -> dynamicConfig ); this.loadQueueManager = diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java index 28677d57fc08..4849b6440d55 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java @@ -33,7 +33,7 @@ import org.apache.druid.server.coordination.DataSegmentChangeRequest; import org.apache.druid.server.coordination.DataSegmentChangeResponse; import org.apache.druid.server.coordination.SegmentChangeStatus; -import org.apache.druid.server.http.HistoricalLoadingCapabilities; +import org.apache.druid.server.http.SegmentLoadingCapabilities; import org.jboss.netty.buffer.ChannelBuffers; import org.jboss.netty.handler.codec.http.DefaultHttpResponse; import org.jboss.netty.handler.codec.http.HttpResponse; @@ -160,7 +160,7 @@ private ListenableFuture getCapabilities(HttpRespon // Serialize SettableFuture future = SettableFuture.create(); try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - objectMapper.writeValue(baos, new HistoricalLoadingCapabilities(1, 1)); + objectMapper.writeValue(baos, new SegmentLoadingCapabilities(1, 1)); future.set(new ByteArrayInputStream(baos.toByteArray())); } return future; diff --git a/server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java b/server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java similarity index 82% rename from server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java rename to server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java index 5b82daed8683..55b784d0ae90 100644 --- a/server/src/test/java/org/apache/druid/server/http/HistoricalLoadingCapabilitiesTest.java +++ b/server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java @@ -29,16 +29,16 @@ import org.junit.Test; import org.junit.internal.matchers.ThrowableMessageMatcher; -public class HistoricalLoadingCapabilitiesTest +public class SegmentLoadingCapabilitiesTest { private final ObjectMapper jsonMapper = new DefaultObjectMapper(); @Test public void testSerde() throws Exception { - HistoricalLoadingCapabilities capabilities = new HistoricalLoadingCapabilities(1, 4); + SegmentLoadingCapabilities capabilities = new SegmentLoadingCapabilities(1, 4); - HistoricalLoadingCapabilities reread = jsonMapper.readValue(jsonMapper.writeValueAsString(capabilities), HistoricalLoadingCapabilities.class); + SegmentLoadingCapabilities reread = jsonMapper.readValue(jsonMapper.writeValueAsString(capabilities), SegmentLoadingCapabilities.class); Assert.assertEquals(capabilities.getNumLoadingThreads(), reread.getNumLoadingThreads()); Assert.assertEquals(capabilities.getNumTurboLoadingThreads(), reread.getNumTurboLoadingThreads()); @@ -48,7 +48,7 @@ public void testSerde() throws Exception public void testSerdeFromJson() throws JsonProcessingException { String json = "{\"numLoadingThreads\":3,\"numTurboLoadingThreads\":5}"; - HistoricalLoadingCapabilities reread = jsonMapper.readValue(json, HistoricalLoadingCapabilities.class); + SegmentLoadingCapabilities reread = jsonMapper.readValue(json, SegmentLoadingCapabilities.class); Assert.assertEquals(3, reread.getNumLoadingThreads()); Assert.assertEquals(5, reread.getNumTurboLoadingThreads()); @@ -59,7 +59,7 @@ public void testSerdeFromEmptyJson() { MatcherAssert.assertThat( Assert.assertThrows(ValueInstantiationException.class, () -> - jsonMapper.readValue("{}", HistoricalLoadingCapabilities.class) + jsonMapper.readValue("{}", SegmentLoadingCapabilities.class) ), CoreMatchers.allOf( CoreMatchers.instanceOf(ValueInstantiationException.class), @@ -71,7 +71,7 @@ public void testSerdeFromEmptyJson() MatcherAssert.assertThat( Assert.assertThrows(ValueInstantiationException.class, () -> - jsonMapper.readValue("{\"numLoadingThreads\":3}", HistoricalLoadingCapabilities.class) + jsonMapper.readValue("{\"numLoadingThreads\":3}", SegmentLoadingCapabilities.class) ), CoreMatchers.allOf( CoreMatchers.instanceOf(ValueInstantiationException.class), From dce2f12840ae75433590d524911e95f689ee9c7e Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 21 Mar 2025 14:08:11 +0530 Subject: [PATCH 25/30] Address review comments --- .../loading/HttpLoadQueuePeon.java | 31 +++---------------- .../loading/LoadQueueTaskMaster.java | 4 +-- .../http/SegmentLoadingCapabilities.java | 4 +++ .../loading/HttpLoadQueuePeonTest.java | 25 +++++++++++---- .../TestSegmentLoadingHttpClient.java | 2 +- 5 files changed, 30 insertions(+), 36 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index deb88a1f51dd..7af11fcc7106 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -132,9 +132,9 @@ public HttpLoadQueuePeon( ObjectMapper jsonMapper, HttpClient httpClient, HttpLoadQueuePeonConfig config, + Supplier loadingModeSupplier, ScheduledExecutorService processingExecutor, - ExecutorService callBackExecutor, - Supplier loadingModeSupplier + ExecutorService callBackExecutor ) { this.jsonMapper = jsonMapper; @@ -150,30 +150,7 @@ public HttpLoadQueuePeon( } @VisibleForTesting - HttpLoadQueuePeon( - String baseUrl, - ObjectMapper jsonMapper, - HttpClient httpClient, - HttpLoadQueuePeonConfig config, - Supplier loadingModeSupplier, - ScheduledExecutorService processingExecutor, - ExecutorService callBackExecutor, - SegmentLoadingCapabilities serverCapabilities - ) - { - this.jsonMapper = jsonMapper; - this.requestBodyWriter = jsonMapper.writerFor(REQUEST_ENTITY_TYPE_REF); - this.httpClient = httpClient; - this.config = config; - this.processingExecutor = processingExecutor; - this.callBackExecutor = callBackExecutor; - - this.serverId = baseUrl; - this.loadingModeSupplier = loadingModeSupplier; - this.serverCapabilities = serverCapabilities; - } - - private SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() + SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() { try { final URL segmentLoadingCapabilitiesURL = new URL( @@ -192,7 +169,7 @@ private SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() if (HttpServletResponse.SC_NOT_FOUND == responseHandler.getStatus()) { log.warn( "Historical capabilities endpoint not found at server[%s]. Using default values.", - new URL(serverId) + segmentLoadingCapabilitiesURL ); return new SegmentLoadingCapabilities(1, 1); } else if (HttpServletResponse.SC_OK != responseHandler.getStatus()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java index f97ee9bb7449..d9fdca36c67d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/LoadQueueTaskMaster.java @@ -80,9 +80,9 @@ private LoadQueuePeon createPeon(ImmutableDruidServer server) jsonMapper, httpClient, config, + () -> coordinatorDynamicConfigSupplier.get().getLoadingModeForServer(server.getName()), peonExec, - callbackExec, - () -> coordinatorDynamicConfigSupplier.get().getLoadingModeForServer(server.getName()) + callbackExec ); } diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java index 8c04ed5e8ba8..279b59df119f 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java @@ -23,6 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.error.DruidException; +/** + * Contains information related to the capability of a server to load segments, for example the number of threads + * available. + */ public class SegmentLoadingCapabilities { private final int numLoadingThreads; diff --git a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java index 1803811e105e..e511ce77ee2c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeonTest.java @@ -89,9 +89,15 @@ public void setUp() httpClient.processingExecutor, true ), - httpClient.callbackExecutor, - new SegmentLoadingCapabilities(1, 3) - ); + httpClient.callbackExecutor + ) + { + @Override + SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() + { + return new SegmentLoadingCapabilities(1, 3); + } + }; httpLoadQueuePeon.start(); } @@ -325,6 +331,7 @@ public void testBatchSize() { Assert.assertEquals(10, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.NORMAL)); + // Without a batch size runtime parameter httpLoadQueuePeon = new HttpLoadQueuePeon( "http://dummy:4000", MAPPER, @@ -336,9 +343,15 @@ public void testBatchSize() httpClient.processingExecutor, true ), - httpClient.callbackExecutor, - new SegmentLoadingCapabilities(1, 3) - ); + httpClient.callbackExecutor + ) + { + @Override + SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() + { + return new SegmentLoadingCapabilities(1, 3); + } + }; Assert.assertEquals(1, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.NORMAL)); Assert.assertEquals(3, httpLoadQueuePeon.calculateBatchSize(SegmentLoadingMode.TURBO)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java index 4849b6440d55..877265d85b4e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/TestSegmentLoadingHttpClient.java @@ -88,7 +88,7 @@ public ListenableFuture go( Duration readTimeout ) { - if (request.getUrl().toString().contains("/segmentLoadingCapabilities")) { + if (request.getUrl().toString().contains("/loadCapabilities")) { return getCapabilities(handler); } return executorService.submit(() -> processRequest(request, handler)); From 42e20d4709bdd55bb518b951a4dc2de5a39ed399 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 21 Mar 2025 16:07:55 +0530 Subject: [PATCH 26/30] Address review comments --- .../coordination/SegmentLoadDropHandler.java | 24 +++++-------------- .../loading/HttpLoadQueuePeon.java | 15 ++++++------ .../SegmentLoadDropHandlerTest.java | 5 ++-- 3 files changed, 16 insertions(+), 28 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index f2a95de79251..c40ce83034e1 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -46,8 +46,8 @@ import java.util.Map; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -67,8 +67,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final SegmentLoaderConfig config; private final DataSegmentAnnouncer announcer; private final SegmentManager segmentManager; - private final ScheduledExecutorService scheduledExecutorService; - private final ThreadPoolExecutor normalLoadExec; + private final ScheduledExecutorService normalLoadExec; private final ThreadPoolExecutor turboLoadExec; private final ConcurrentSkipListSet segmentsToDelete; @@ -94,12 +93,8 @@ public SegmentLoadDropHandler( config, announcer, segmentManager, - new ThreadPoolExecutor( - config.getNumLoadingThreads(), + new ScheduledThreadPoolExecutor( config.getNumLoadingThreads(), - 60L, - TimeUnit.SECONDS, - new SynchronousQueue<>(), Execs.makeThreadFactory("SegmentLoadDropHandler-normal-%s") ), new ThreadPoolExecutor( @@ -109,10 +104,6 @@ public SegmentLoadDropHandler( TimeUnit.SECONDS, new SynchronousQueue<>(), Execs.makeThreadFactory("SegmentLoadDropHandler-turbo-%s") - ), - Executors.newScheduledThreadPool( - config.getNumLoadingThreads(), - Execs.makeThreadFactory("ScheduledDataSegmentChangeHandler-%s") ) ); } @@ -122,9 +113,8 @@ public SegmentLoadDropHandler( SegmentLoaderConfig config, DataSegmentAnnouncer announcer, SegmentManager segmentManager, - ThreadPoolExecutor normalLoadExec, - ThreadPoolExecutor turboLoadExec, - ScheduledExecutorService scheduledExecutorService + ScheduledExecutorService normalLoadExec, + ThreadPoolExecutor turboLoadExec ) { this.config = config; @@ -132,9 +122,7 @@ public SegmentLoadDropHandler( this.segmentManager = segmentManager; this.normalLoadExec = normalLoadExec; this.turboLoadExec = turboLoadExec; - this.scheduledExecutorService = scheduledExecutorService; - this.normalLoadExec.allowCoreThreadTimeOut(true); this.turboLoadExec.allowCoreThreadTimeOut(true); this.segmentsToDelete = new ConcurrentSkipListSet<>(); @@ -243,7 +231,7 @@ void removeSegment( "Completely removing segment[%s] in [%,d]ms.", segment.getId(), config.getDropSegmentDelayMillis() ); - scheduledExecutorService.schedule( + normalLoadExec.schedule( runnable, config.getDropSegmentDelayMillis(), TimeUnit.MILLISECONDS diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index 7af11fcc7106..bb5c217f157e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -26,7 +26,7 @@ import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.error.DruidException; +import org.apache.druid.common.config.Configs; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -358,17 +358,18 @@ private void logRequestFailure(Throwable t) } } + /** + * Calculates the number of segments the server is capable of handling at a time. If loading segments in turbo loading + * mode, returns the number of turbo loading threads on the server. Otherwise, return the value set by the batch size + * runtime parameter, or number of normal threads on the server if the parameter is not set. + */ @VisibleForTesting int calculateBatchSize(SegmentLoadingMode loadingMode) { - if (config.getBatchSize() != null) { - return config.getBatchSize(); - } else if (SegmentLoadingMode.TURBO.equals(loadingMode)) { + if (SegmentLoadingMode.TURBO.equals(loadingMode)) { return serverCapabilities.getNumTurboLoadingThreads(); - } else if (SegmentLoadingMode.NORMAL.equals(loadingMode)) { - return serverCapabilities.getNumLoadingThreads(); } else { - throw DruidException.defensive().build("unsupported loading mode"); + return Configs.valueOrDefault(config.getBatchSize(), serverCapabilities.getNumLoadingThreads()); } } diff --git a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java index b7cb1831329b..7a8822a60d87 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/SegmentLoadDropHandlerTest.java @@ -422,9 +422,8 @@ private SegmentLoadDropHandler initSegmentLoadDropHandler( config, segmentAnnouncer, segmentManager, - (ThreadPoolExecutor) scheduledExecutorFactory.create(5, "SegmentLoadDropHandlerTest-[%d]"), - (ThreadPoolExecutor) scheduledExecutorFactory.create(5, "TurboSegmentLoadDropHandlerTest-[%d]"), - scheduledExecutorFactory.create(5, "ScheduledSegmentLoadDropHandlerTest-[%d]") + scheduledExecutorFactory.create(5, "LoadDropHandlerTest-[%d]"), + (ThreadPoolExecutor) scheduledExecutorFactory.create(5, "TurboSegmentLoadDropHandlerTest-[%d]") ); } } From fc92af4d55f259af5fcb31de7ef6ffcf063d8a84 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Fri, 21 Mar 2025 16:11:39 +0530 Subject: [PATCH 27/30] Address review comments --- .../druid/server/coordinator/loading/HttpLoadQueuePeon.java | 6 ++++++ .../druid/server/http/SegmentLoadingCapabilities.java | 6 ------ 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index bb5c217f157e..e853f6af7fc0 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -27,6 +27,7 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.common.config.Configs; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -197,6 +198,11 @@ private void doSegmentManagement() final SegmentLoadingMode loadingMode = loadingModeSupplier.get(); final int batchSize = calculateBatchSize(loadingMode); + if (batchSize < 1) { + log.error("Batch size must be greater than 0."); + throw new RE("Batch size must be greater than 0."); + } + final List newRequests = new ArrayList<>(batchSize); synchronized (lock) { diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java index 279b59df119f..9e84b7024ba9 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.error.DruidException; /** * Contains information related to the capability of a server to load segments, for example the number of threads @@ -38,11 +37,6 @@ public SegmentLoadingCapabilities( @JsonProperty("numTurboLoadingThreads") int numTurboLoadingThreads ) { - if (numLoadingThreads < 1 || numTurboLoadingThreads < 1) { - throw DruidException.forPersona(DruidException.Persona.OPERATOR) - .ofCategory(DruidException.Category.INVALID_INPUT) - .build("numLoadingThreads and numTurboLoadingThreads must be greater than 0."); - } this.numLoadingThreads = numLoadingThreads; this.numTurboLoadingThreads = numTurboLoadingThreads; } From d84e31c7d8ce43667ae0f8c3f8a59e514b15d549 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 24 Mar 2025 13:22:18 +0530 Subject: [PATCH 28/30] Address review comments --- .../java/util/common/concurrent/Execs.java | 18 ++++++++++ .../coordination/SegmentLoadDropHandler.java | 20 ++++------- .../loading/HttpLoadQueuePeon.java | 35 +++++++++++-------- .../http/CoordinatorDynamicConfigTest.java | 18 +++++----- .../http/SegmentLoadingCapabilitiesTest.java | 32 ----------------- 5 files changed, 55 insertions(+), 68 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java index 6b49ab0a0ee9..813befd7ebf3 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java @@ -71,6 +71,24 @@ public static ExecutorService multiThreaded(int threads, @NotNull String nameFor return Executors.newFixedThreadPool(threads, makeThreadFactory(nameFormat, priority)); } + public static ExecutorService multiThreaded( + int corePoolSize, + int maxPoolSize, + long keepAliveTime, + @NotNull TimeUnit timeUnit, + @NotNull String nameFormat + ) + { + return new ThreadPoolExecutor( + corePoolSize, + maxPoolSize, + keepAliveTime, + timeUnit, + new SynchronousQueue<>(), + Execs.makeThreadFactory(nameFormat) + ); + } + public static ScheduledExecutorService scheduledSingleThreaded(@NotNull String nameFormat) { return scheduledSingleThreaded(nameFormat, null); diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index c40ce83034e1..d0c86c2e4d7d 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -46,10 +46,8 @@ import java.util.Map; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledThreadPoolExecutor; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -68,7 +66,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final DataSegmentAnnouncer announcer; private final SegmentManager segmentManager; private final ScheduledExecutorService normalLoadExec; - private final ThreadPoolExecutor turboLoadExec; + private final ExecutorService turboLoadExec; private final ConcurrentSkipListSet segmentsToDelete; @@ -93,17 +91,16 @@ public SegmentLoadDropHandler( config, announcer, segmentManager, - new ScheduledThreadPoolExecutor( + Executors.newScheduledThreadPool( config.getNumLoadingThreads(), Execs.makeThreadFactory("SegmentLoadDropHandler-normal-%s") ), - new ThreadPoolExecutor( - config.getNumBootstrapThreads(), + Execs.multiThreaded( + 1, config.getNumBootstrapThreads(), 60L, TimeUnit.SECONDS, - new SynchronousQueue<>(), - Execs.makeThreadFactory("SegmentLoadDropHandler-turbo-%s") + "SegmentLoadDropHandler-turbo-%s" ) ); } @@ -114,7 +111,7 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, SegmentManager segmentManager, ScheduledExecutorService normalLoadExec, - ThreadPoolExecutor turboLoadExec + ExecutorService turboLoadExec ) { this.config = config; @@ -123,8 +120,6 @@ public SegmentLoadDropHandler( this.normalLoadExec = normalLoadExec; this.turboLoadExec = turboLoadExec; - this.turboLoadExec.allowCoreThreadTimeOut(true); - this.segmentsToDelete = new ConcurrentSkipListSet<>(); requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); } @@ -300,7 +295,6 @@ public ListenableFuture> processBatch( * Process a {@link DataSegmentChangeRequest}, invoking the request's * {@link DataSegmentChangeRequest#go(DataSegmentChangeHandler, DataSegmentChangeCallback)}. * The segmentLoadingMode parameter determines the thread pool to use. - * Returns an atomic reference to the segment status. */ private AtomicReference processRequest( DataSegmentChangeRequest changeRequest, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index e853f6af7fc0..a303913a79cc 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -27,7 +27,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import org.apache.druid.common.config.Configs; -import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.StringUtils; @@ -87,6 +86,7 @@ public class HttpLoadQueuePeon implements LoadQueuePeon new TypeReference<>() {}; private static final EmittingLogger log = new EmittingLogger(HttpLoadQueuePeon.class); + private static final long DEFAULT_TIMEOUT = 10000L; private final AtomicLong queuedSize = new AtomicLong(0); private final AtomicReference stats = new AtomicReference<>(new CoordinatorRunStats()); @@ -164,18 +164,21 @@ SegmentLoadingCapabilities fetchSegmentLoadingCapabilities() new Request(HttpMethod.GET, segmentLoadingCapabilitiesURL) .addHeader(HttpHeaders.Names.ACCEPT, MediaType.APPLICATION_JSON), responseHandler, - new Duration(10000) + new Duration(DEFAULT_TIMEOUT) ).get(); if (HttpServletResponse.SC_NOT_FOUND == responseHandler.getStatus()) { + int batchSize = config.getBatchSize() == null ? 1 : config.getBatchSize(); + SegmentLoadingCapabilities defaultCapabilities = new SegmentLoadingCapabilities(batchSize, batchSize); log.warn( - "Historical capabilities endpoint not found at server[%s]. Using default values.", - segmentLoadingCapabilitiesURL + "Historical capabilities endpoint not found at URL[%s]. Using default values[%s].", + segmentLoadingCapabilitiesURL, + defaultCapabilities ); - return new SegmentLoadingCapabilities(1, 1); + return defaultCapabilities; } else if (HttpServletResponse.SC_OK != responseHandler.getStatus()) { - log.makeAlert("Error when fetching capabilities from server[%s]. Received [%s]", new URL(serverId), responseHandler.getStatus()); - throw new RE("Error when fetching capabilities from server[%s]. Received [%s]", new URL(serverId), responseHandler.getStatus()); + log.makeAlert("Received status[%s] when fetching loading capabilities from server[%s]", responseHandler.getStatus(), serverId); + throw new RE("Received status[%s] when fetching loading capabilities from server[%s]", responseHandler.getStatus(), serverId); } return jsonMapper.readValue( @@ -198,11 +201,6 @@ private void doSegmentManagement() final SegmentLoadingMode loadingMode = loadingModeSupplier.get(); final int batchSize = calculateBatchSize(loadingMode); - if (batchSize < 1) { - log.error("Batch size must be greater than 0."); - throw new RE("Batch size must be greater than 0."); - } - final List newRequests = new ArrayList<>(batchSize); synchronized (lock) { @@ -368,15 +366,24 @@ private void logRequestFailure(Throwable t) * Calculates the number of segments the server is capable of handling at a time. If loading segments in turbo loading * mode, returns the number of turbo loading threads on the server. Otherwise, return the value set by the batch size * runtime parameter, or number of normal threads on the server if the parameter is not set. + * Always returns a positive integer. */ @VisibleForTesting int calculateBatchSize(SegmentLoadingMode loadingMode) { + int batchSize; if (SegmentLoadingMode.TURBO.equals(loadingMode)) { - return serverCapabilities.getNumTurboLoadingThreads(); + batchSize = serverCapabilities.getNumTurboLoadingThreads(); } else { - return Configs.valueOrDefault(config.getBatchSize(), serverCapabilities.getNumLoadingThreads()); + batchSize = Configs.valueOrDefault(config.getBatchSize(), serverCapabilities.getNumLoadingThreads()); } + + if (batchSize < 1) { + log.error("Batch size must be greater than 0."); + throw new RE("Batch size must be greater than 0."); + } + + return batchSize; } private void handleResponseStatus(DataSegmentChangeRequest changeRequest, SegmentChangeStatus status) diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java index d06d1b40f483..1be987ddc278 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorDynamicConfigTest.java @@ -66,7 +66,7 @@ public void testSerde() throws Exception ); ImmutableSet decommissioning = ImmutableSet.of("host1", "host2"); ImmutableSet whitelist = ImmutableSet.of("test1", "test2"); - ImmutableSet turboLoading = ImmutableSet.of("host1", "host3"); + ImmutableSet turboLoadingNodes = ImmutableSet.of("host1", "host3"); assertConfig( actual, 1, @@ -81,7 +81,7 @@ public void testSerde() throws Exception decommissioning, false, false, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().withDecommissioningNodes(ImmutableSet.of("host1")).build(actual); @@ -99,7 +99,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), false, false, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().build(actual); @@ -117,7 +117,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), false, false, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().withPauseCoordination(true).build(actual); @@ -135,7 +135,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), true, false, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().withReplicateAfterLoadTimeout(true).build(actual); @@ -153,7 +153,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), true, true, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().build(actual); @@ -171,7 +171,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), true, true, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().withKillTaskSlotRatio(0.1).build(actual); @@ -189,7 +189,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), true, true, - turboLoading + turboLoadingNodes ); actual = CoordinatorDynamicConfig.builder().withMaxKillTaskSlots(5).build(actual); @@ -207,7 +207,7 @@ public void testSerde() throws Exception ImmutableSet.of("host1"), true, true, - turboLoading + turboLoadingNodes ); } diff --git a/server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java b/server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java index 55b784d0ae90..0819439a0aa9 100644 --- a/server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java +++ b/server/src/test/java/org/apache/druid/server/http/SegmentLoadingCapabilitiesTest.java @@ -21,13 +21,9 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.exc.ValueInstantiationException; import org.apache.druid.jackson.DefaultObjectMapper; -import org.hamcrest.CoreMatchers; -import org.hamcrest.MatcherAssert; import org.junit.Assert; import org.junit.Test; -import org.junit.internal.matchers.ThrowableMessageMatcher; public class SegmentLoadingCapabilitiesTest { @@ -53,32 +49,4 @@ public void testSerdeFromJson() throws JsonProcessingException Assert.assertEquals(3, reread.getNumLoadingThreads()); Assert.assertEquals(5, reread.getNumTurboLoadingThreads()); } - - @Test - public void testSerdeFromEmptyJson() - { - MatcherAssert.assertThat( - Assert.assertThrows(ValueInstantiationException.class, () -> - jsonMapper.readValue("{}", SegmentLoadingCapabilities.class) - ), - CoreMatchers.allOf( - CoreMatchers.instanceOf(ValueInstantiationException.class), - ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("numLoadingThreads and numTurboLoadingThreads must be greater than 0.") - ) - ) - ); - - MatcherAssert.assertThat( - Assert.assertThrows(ValueInstantiationException.class, () -> - jsonMapper.readValue("{\"numLoadingThreads\":3}", SegmentLoadingCapabilities.class) - ), - CoreMatchers.allOf( - CoreMatchers.instanceOf(ValueInstantiationException.class), - ThrowableMessageMatcher.hasMessage( - CoreMatchers.containsString("numLoadingThreads and numTurboLoadingThreads must be greater than 0.") - ) - ) - ); - } } From 752c1b703f41bf5b737c7426e9d247ecbce688a9 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 24 Mar 2025 17:15:32 +0530 Subject: [PATCH 29/30] Add correct threadpool --- .../java/util/common/concurrent/Execs.java | 18 ------ .../coordination/SegmentLoadDropHandler.java | 20 +++--- .../config/HttpLoadQueuePeonConfig.java | 6 ++ .../loading/HttpLoadQueuePeon.java | 7 +-- .../http/SegmentLoadingCapabilities.java | 9 +++ .../config/HttpLoadQueuePeonConfigTest.java | 62 +++++++++++++++++++ 6 files changed, 91 insertions(+), 31 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfigTest.java diff --git a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java index 813befd7ebf3..6b49ab0a0ee9 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/concurrent/Execs.java @@ -71,24 +71,6 @@ public static ExecutorService multiThreaded(int threads, @NotNull String nameFor return Executors.newFixedThreadPool(threads, makeThreadFactory(nameFormat, priority)); } - public static ExecutorService multiThreaded( - int corePoolSize, - int maxPoolSize, - long keepAliveTime, - @NotNull TimeUnit timeUnit, - @NotNull String nameFormat - ) - { - return new ThreadPoolExecutor( - corePoolSize, - maxPoolSize, - keepAliveTime, - timeUnit, - new SynchronousQueue<>(), - Execs.makeThreadFactory(nameFormat) - ); - } - public static ScheduledExecutorService scheduledSingleThreaded(@NotNull String nameFormat) { return scheduledSingleThreaded(nameFormat, null); diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index d0c86c2e4d7d..0566886d0b90 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -46,8 +46,10 @@ import java.util.Map; import java.util.concurrent.ConcurrentSkipListSet; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; +import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; @@ -66,7 +68,7 @@ public class SegmentLoadDropHandler implements DataSegmentChangeHandler private final DataSegmentAnnouncer announcer; private final SegmentManager segmentManager; private final ScheduledExecutorService normalLoadExec; - private final ExecutorService turboLoadExec; + private final ThreadPoolExecutor turboLoadExec; private final ConcurrentSkipListSet segmentsToDelete; @@ -91,16 +93,18 @@ public SegmentLoadDropHandler( config, announcer, segmentManager, - Executors.newScheduledThreadPool( + new ScheduledThreadPoolExecutor( config.getNumLoadingThreads(), Execs.makeThreadFactory("SegmentLoadDropHandler-normal-%s") ), - Execs.multiThreaded( - 1, + // Create a fixed size threadpool which has a timeout of 1 minute + new ThreadPoolExecutor( + config.getNumBootstrapThreads(), config.getNumBootstrapThreads(), 60L, TimeUnit.SECONDS, - "SegmentLoadDropHandler-turbo-%s" + new LinkedBlockingQueue<>(), + Execs.makeThreadFactory("SegmentLoadDropHandler-turbo-%s") ) ); } @@ -111,7 +115,7 @@ public SegmentLoadDropHandler( DataSegmentAnnouncer announcer, SegmentManager segmentManager, ScheduledExecutorService normalLoadExec, - ExecutorService turboLoadExec + ThreadPoolExecutor turboLoadExec ) { this.config = config; @@ -120,6 +124,8 @@ public SegmentLoadDropHandler( this.normalLoadExec = normalLoadExec; this.turboLoadExec = turboLoadExec; + this.turboLoadExec.allowCoreThreadTimeOut(true); + this.segmentsToDelete = new ConcurrentSkipListSet<>(); requestStatuses = CacheBuilder.newBuilder().maximumSize(config.getStatusQueueMaxSize()).initialCapacity(8).build(); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java index d77943d61017..42af2e948cf8 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfig.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.common.config.Configs; +import org.apache.druid.java.util.common.RE; import org.joda.time.Duration; import javax.annotation.Nullable; @@ -49,6 +50,11 @@ public HttpLoadQueuePeonConfig( { this.hostTimeout = Configs.valueOrDefault(hostTimeout, Duration.standardMinutes(5)); this.repeatDelay = Configs.valueOrDefault(repeatDelay, Duration.standardMinutes(1)); + + if (batchSize != null && batchSize < 1) { + throw new RE("Batch size must be greater than 0."); + } + this.batchSize = batchSize; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java index a303913a79cc..4f5ff04911ed 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/loading/HttpLoadQueuePeon.java @@ -378,12 +378,7 @@ int calculateBatchSize(SegmentLoadingMode loadingMode) batchSize = Configs.valueOrDefault(config.getBatchSize(), serverCapabilities.getNumLoadingThreads()); } - if (batchSize < 1) { - log.error("Batch size must be greater than 0."); - throw new RE("Batch size must be greater than 0."); - } - - return batchSize; + return Math.max(batchSize, 1); } private void handleResponseStatus(DataSegmentChangeRequest changeRequest, SegmentChangeStatus status) diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java index 9e84b7024ba9..9dba8af5e6bb 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentLoadingCapabilities.java @@ -52,4 +52,13 @@ public int getNumTurboLoadingThreads() { return numTurboLoadingThreads; } + + @Override + public String toString() + { + return "SegmentLoadingCapabilities{" + + "numLoadingThreads=" + numLoadingThreads + + ", numTurboLoadingThreads=" + numTurboLoadingThreads + + '}'; + } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfigTest.java new file mode 100644 index 000000000000..4d9a2400215c --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/config/HttpLoadQueuePeonConfigTest.java @@ -0,0 +1,62 @@ +/* + * 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.coordinator.config; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.exc.ValueInstantiationException; +import org.hamcrest.CoreMatchers; +import org.hamcrest.MatcherAssert; +import org.junit.Assert; +import org.junit.Test; +import org.junit.internal.matchers.ThrowableMessageMatcher; + +public class HttpLoadQueuePeonConfigTest +{ + @Test + public void testValidateBatchSize() throws JsonProcessingException + { + ObjectMapper jsonMapper = new ObjectMapper(); + + MatcherAssert.assertThat( + Assert.assertThrows(ValueInstantiationException.class, () -> + jsonMapper.readValue("{\"batchSize\":0}", HttpLoadQueuePeonConfig.class) + ), + CoreMatchers.allOf( + CoreMatchers.instanceOf(ValueInstantiationException.class), + ThrowableMessageMatcher.hasMessage( + CoreMatchers.containsString("Batch size must be greater than 0.") + ) + ) + ); + + HttpLoadQueuePeonConfig emptyConfig = jsonMapper.readValue( + "{}", + HttpLoadQueuePeonConfig.class + ); + Assert.assertNull(emptyConfig.getBatchSize()); + + HttpLoadQueuePeonConfig config = jsonMapper.readValue( + "{\"batchSize\":2}", + HttpLoadQueuePeonConfig.class + ); + Assert.assertEquals(2, config.getBatchSize().intValue()); + } +} From 52ac772f47ac12f2e099b5b25e3018bbe5878cca Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Mon, 24 Mar 2025 17:20:05 +0530 Subject: [PATCH 30/30] Add comment --- .../druid/server/coordination/SegmentLoadDropHandler.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java index 0566886d0b90..4d3e22c1a131 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java +++ b/server/src/main/java/org/apache/druid/server/coordination/SegmentLoadDropHandler.java @@ -97,7 +97,8 @@ public SegmentLoadDropHandler( config.getNumLoadingThreads(), Execs.makeThreadFactory("SegmentLoadDropHandler-normal-%s") ), - // Create a fixed size threadpool which has a timeout of 1 minute + // Create a fixed size threadpool which has a timeout of 1 minute. Since they are all core threads, new threads + // will be created without enqueing the tasks till the capacity is reached. new ThreadPoolExecutor( config.getNumBootstrapThreads(), config.getNumBootstrapThreads(),