From 87556bc75109c4555d9eecd40634124d0ae6bec2 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 19 Mar 2019 18:26:13 -0300 Subject: [PATCH 01/40] Reconcile terminology and method naming to 'used/unused segments'; Don't use terms 'enable/disable data source'; Rename MetadataSegmentManager to MetadataSegments; Make REST API methods which mark segments as used/unused to return server error instead of an empty response in case of error --- .../timeline/VersionedIntervalTimeline.java | 93 ++++++++++---- .../partition/ImmutablePartitionHolder.java | 4 +- .../timeline/partition/PartitionHolder.java | 52 +++----- .../VersionedIntervalTimelineTest.java | 34 ++--- docs/content/configuration/index.md | 15 ++- docs/content/dependencies/metadata-storage.md | 21 ++- docs/content/design/coordinator.md | 34 ++++- docs/content/design/index.md | 6 +- docs/content/ingestion/delete-data.md | 3 +- docs/content/operations/api-reference.md | 18 +-- docs/content/operations/metrics.md | 4 +- docs/content/operations/rule-configuration.md | 11 +- .../MaterializedViewSupervisor.java | 24 ++-- .../MaterializedViewSupervisorSpec.java | 14 +- .../MaterializedViewSupervisorSpecTest.java | 4 +- .../MaterializedViewSupervisorTest.java | 10 +- .../ActionBasedUsedSegmentChecker.java | 27 ++-- .../SinglePhaseParallelIndexTaskRunner.java | 3 +- .../IngestSegmentFirehoseFactory.java | 6 +- .../actions/SegmentListActionsTest.java | 2 +- .../common/actions/TaskActionTestKit.java | 18 +-- .../common/task/CompactionTaskRunTest.java | 2 +- .../common/task/CompactionTaskTest.java | 2 +- .../IngestSegmentFirehoseFactoryTest.java | 2 +- ...estSegmentFirehoseFactoryTimelineTest.java | 2 +- .../client/ImmutableDruidDataSource.java | 2 +- .../druid/client/ImmutableDruidServer.java | 21 +-- .../client/coordinator/CoordinatorClient.java | 4 +- .../druid/guice/MetadataConfigModule.java | 4 +- .../guice/SQLMetadataStorageDruidModule.java | 20 +-- .../IndexerMetadataStorageCoordinator.java | 3 +- .../metadata/MetadataSegmentManager.java | 80 ------------ .../druid/metadata/MetadataSegments.java | 121 ++++++++++++++++++ ...onfig.java => MetadataSegmentsConfig.java} | 2 +- ...der.java => MetadataSegmentsProvider.java} | 4 +- .../metadata/SQLMetadataRuleManager.java | 2 +- ...tManager.java => SqlMetadataSegments.java} | 55 ++++---- ....java => SqlMetadataSegmentsProvider.java} | 12 +- .../org/apache/druid/server/JettyUtils.java | 40 ++++++ .../coordinator/CoordinatorDynamicConfig.java | 34 ++--- .../coordinator/CostBalancerStrategy.java | 24 ++-- .../DiskNormalizedCostBalancerStrategy.java | 4 +- .../server/coordinator/DruidCoordinator.java | 100 ++++++++------- ...ruidCoordinatorCleanupPendingSegments.java | 2 +- .../DruidCoordinatorRuntimeParams.java | 100 ++++++++------- .../coordinator/ReservoirSegmentSampler.java | 2 +- .../coordinator/SegmentReplicantLookup.java | 2 +- .../helper/DruidCoordinatorBalancer.java | 19 ++- .../DruidCoordinatorCleanupOvershadowed.java | 4 +- ...ruidCoordinatorCleanupUnusedSegments.java} | 23 ++-- .../helper/DruidCoordinatorLogger.java | 47 +++---- .../helper/DruidCoordinatorRuleRunner.java | 13 +- .../DruidCoordinatorSegmentCompactor.java | 2 +- .../helper/DruidCoordinatorSegmentKiller.java | 27 ++-- ...> DruidCoordinatorUsedSegmentsLoader.java} | 32 ++--- .../server/coordinator/rules/DropRule.java | 2 +- .../druid/server/coordinator/rules/Rule.java | 8 +- .../server/http/CoordinatorResource.java | 2 +- .../server/http/DataSourcesResource.java | 89 +++++++------ .../druid/server/http/MetadataResource.java | 113 ++++++++-------- ...Test.java => SqlMetadataSegmentsTest.java} | 24 ++-- .../coordinator/CostBalancerStrategyTest.java | 2 +- .../CuratorDruidCoordinatorTest.java | 14 +- ...iskNormalizedCostBalancerStrategyTest.java | 2 +- .../DruidCoordinatorBalancerProfiler.java | 12 +- .../DruidCoordinatorBalancerTest.java | 4 +- .../DruidCoordinatorRuleRunnerTest.java | 90 ++++++------- .../coordinator/DruidCoordinatorTest.java | 40 +++--- .../ReservoirSegmentSamplerTest.java | 8 +- .../cost/CachingCostBalancerStrategyTest.java | 2 +- ...uidCoordinatorCleanupOvershadowedTest.java | 10 +- .../DruidCoordinatorSegmentCompactorTest.java | 2 +- .../DruidCoordinatorSegmentKillerTest.java | 16 +-- .../rules/BroadcastDistributionRuleTest.java | 8 +- .../coordinator/rules/LoadRuleTest.java | 24 ++-- .../http/CoordinatorDynamicConfigTest.java | 4 +- .../server/http/DataSourcesResourceTest.java | 50 ++++---- .../org/apache/druid/cli/CliCoordinator.java | 12 +- .../sql/calcite/schema/SystemSchema.java | 2 +- .../sql/calcite/schema/DruidSchemaTest.java | 3 +- web-console/README.md | 2 +- .../css/{enable.css => use_data_source.css} | 2 +- web-console/old-console/index.html | 2 +- web-console/old-console/js/kill-0.0.1.js | 2 +- ...able-0.0.1.js => use-data-source-0.0.1.js} | 42 +++--- web-console/old-console/kill.html | 2 +- .../{enable.html => use-data-source.html} | 34 ++--- web-console/src/views/datasource-view.tsx | 64 ++++----- 88 files changed, 1035 insertions(+), 874 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java create mode 100644 server/src/main/java/org/apache/druid/metadata/MetadataSegments.java rename server/src/main/java/org/apache/druid/metadata/{MetadataSegmentManagerConfig.java => MetadataSegmentsConfig.java} (96%) rename server/src/main/java/org/apache/druid/metadata/{MetadataSegmentManagerProvider.java => MetadataSegmentsProvider.java} (88%) rename server/src/main/java/org/apache/druid/metadata/{SQLMetadataSegmentManager.java => SqlMetadataSegments.java} (91%) rename server/src/main/java/org/apache/druid/metadata/{SQLMetadataSegmentManagerProvider.java => SqlMetadataSegmentsProvider.java} (85%) rename server/src/main/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorCleanupUnneeded.java => DruidCoordinatorCleanupUnusedSegments.java} (72%) rename server/src/main/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorSegmentInfoLoader.java => DruidCoordinatorUsedSegmentsLoader.java} (68%) rename server/src/test/java/org/apache/druid/metadata/{SQLMetadataSegmentManagerTest.java => SqlMetadataSegmentsTest.java} (88%) rename web-console/old-console/css/{enable.css => use_data_source.css} (97%) rename web-console/old-console/js/{enable-0.0.1.js => use-data-source-0.0.1.js} (70%) rename web-console/old-console/{enable.html => use-data-source.html} (60%) diff --git a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java index 36c177dfb0f0..bf20fa5a4e17 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -30,9 +30,12 @@ import org.apache.druid.timeline.partition.ImmutablePartitionHolder; import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; +import org.apache.druid.utils.CollectionUtils; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -44,28 +47,43 @@ import java.util.Objects; import java.util.Set; import java.util.TreeMap; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.StreamSupport; /** * VersionedIntervalTimeline is a data structure that manages objects on a specific timeline. * - * It associates a jodatime Interval and a generically-typed version with the object that is being stored. + * It associates an {@link Interval} and a generically-typed version with the object that is being stored. * * In the event of overlapping timeline entries, timeline intervals may be chunked. The underlying data associated * with a timeline entry remains unchanged when chunking occurs. * - * After loading objects via the add() method, the lookup(Interval) method can be used to get the list of the most - * recent objects (according to the version) that match the given interval. The intent is that objects represent - * a certain time period and when you do a lookup(), you are asking for all of the objects that you need to look - * at in order to get a correct answer about that time period. + * After loading objects via the {@link #add} method, the {@link #lookup(Interval)} method can be used to get the list + * of the most recent objects (according to the version) that match the given interval. The intent is that objects + * represent a certain time period and when you do a {@link #lookup(Interval)}, you are asking for all of the objects + * that you need to look at in order to get a correct answer about that time period. * - * The findOvershadowed() method returns a list of objects that will never be returned by a call to lookup() because - * they are overshadowed by some other object. This can be used in conjunction with the add() and remove() methods - * to achieve "atomic" updates. First add new items, then check if those items caused anything to be overshadowed, if - * so, remove the overshadowed elements and you have effectively updated your data set without any user impact. + * The {@link #findOvershadowed} method returns a list of objects that will never be returned by a call to {@link + * #lookup} because they are overshadowed by some other object. This can be used in conjunction with the {@link #add} + * and {@link #remove} methods to achieve "atomic" updates. First add new items, then check if those items caused + * anything to be overshadowed, if so, remove the overshadowed elements and you have effectively updated your data set + * without any user impact. */ public class VersionedIntervalTimeline implements TimelineLookup { + public static VersionedIntervalTimeline forSegments(Iterable segments) + { + return forSegments(segments.iterator()); + } + + public static VersionedIntervalTimeline forSegments(Iterator segments) + { + final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + addSegments(timeline, segments); + return timeline; + } + private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); final NavigableMap completePartitionsTimeline = new TreeMap( @@ -75,6 +93,7 @@ public class VersionedIntervalTimeline implements Timel Comparators.intervalsByStartThenEnd() ); private final Map> allTimelineEntries = new HashMap<>(); + private final AtomicInteger numObjects = new AtomicInteger(); private final Comparator versionComparator; @@ -85,18 +104,6 @@ public VersionedIntervalTimeline( this.versionComparator = versionComparator; } - public static VersionedIntervalTimeline forSegments(Iterable segments) - { - return forSegments(segments.iterator()); - } - - public static VersionedIntervalTimeline forSegments(Iterator segments) - { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - addSegments(timeline, segments); - return timeline; - } - public static void addSegments( VersionedIntervalTimeline timeline, Iterator segments @@ -115,6 +122,31 @@ public Map> getAllTimelineEntries( return allTimelineEntries; } + /** + * Returns a lazy collection with all objects in this VersionedIntervalTimeline to be used for iteration or {@link + * Collection#stream()} transformation. The order of objects in this collection is unspecified. + * + * Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an + * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only + * once rather than several times. + */ + public Collection iterateAllObjects() + { + return CollectionUtils.createLazyCollectionFromStream( + () -> allTimelineEntries + .values() + .stream() + .flatMap((TreeMap entryMap) -> entryMap.values().stream()) + .flatMap((TimelineEntry entry) -> StreamSupport.stream(entry.getPartitionHolder().spliterator(), false)) + .map(PartitionChunk::getObject), + numObjects.get() + ); + } + + public int getNumObjects() { + return numObjects.get(); + } + public void add(final Interval interval, VersionType version, PartitionChunk object) { addAll(Iterators.singletonIterator(object), o -> interval, o -> version); @@ -143,15 +175,19 @@ private void addAll( TreeMap versionEntry = new TreeMap<>(versionComparator); versionEntry.put(version, entry); allTimelineEntries.put(interval, versionEntry); + numObjects.incrementAndGet(); } else { entry = exists.get(version); if (entry == null) { entry = new TimelineEntry(interval, version, new PartitionHolder<>(object)); exists.put(version, entry); + numObjects.incrementAndGet(); } else { PartitionHolder partitionHolder = entry.getPartitionHolder(); - partitionHolder.add(object); + if (partitionHolder.add(object)) { + numObjects.incrementAndGet(); + } } } @@ -174,6 +210,7 @@ private void addAll( } } + @Nullable public PartitionChunk remove(Interval interval, VersionType version, PartitionChunk chunk) { try { @@ -189,7 +226,11 @@ public PartitionChunk remove(Interval interval, VersionType version, return null; } - PartitionChunk retVal = entry.getPartitionHolder().remove(chunk); + PartitionChunk removedChunk = entry.getPartitionHolder().remove(chunk); + if (removedChunk == null) { + return null; + } + numObjects.decrementAndGet(); if (entry.getPartitionHolder().isEmpty()) { versionEntries.remove(version); if (versionEntries.isEmpty()) { @@ -201,7 +242,7 @@ public PartitionChunk remove(Interval interval, VersionType version, remove(completePartitionsTimeline, interval, entry, false); - return retVal; + return removedChunk; } finally { lock.writeLock().unlock(); @@ -217,9 +258,7 @@ public PartitionHolder findEntry(Interval interval, VersionType vers if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { TimelineEntry foundEntry = entry.getValue().get(version); if (foundEntry != null) { - return new ImmutablePartitionHolder( - foundEntry.getPartitionHolder() - ); + return new ImmutablePartitionHolder<>(foundEntry.getPartitionHolder()); } } } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java index ec8f7d891a7e..5003f651c121 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java @@ -23,7 +23,7 @@ */ public class ImmutablePartitionHolder extends PartitionHolder { - public ImmutablePartitionHolder(PartitionHolder partitionHolder) + public ImmutablePartitionHolder(PartitionHolder partitionHolder) { super(partitionHolder); } @@ -35,7 +35,7 @@ public PartitionChunk remove(PartitionChunk tPartitionChunk) } @Override - public void add(PartitionChunk tPartitionChunk) + public boolean add(PartitionChunk tPartitionChunk) { throw new UnsupportedOperationException(); } diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java index 396b4f3bdd63..dcf29aedc488 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java @@ -22,72 +22,62 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; +import javax.annotation.Nullable; import java.util.Iterator; import java.util.List; -import java.util.SortedSet; import java.util.Spliterator; -import java.util.TreeSet; +import java.util.TreeMap; /** * An object that clumps together multiple other objects which each represent a shard of some space. */ public class PartitionHolder implements Iterable> { - private final TreeSet> holderSet; + private final TreeMap, PartitionChunk> holderMap; public PartitionHolder(PartitionChunk initialChunk) { - this.holderSet = new TreeSet<>(); + this.holderMap = new TreeMap<>(); add(initialChunk); } public PartitionHolder(List> initialChunks) { - this.holderSet = new TreeSet<>(); + this.holderMap = new TreeMap<>(); for (PartitionChunk chunk : initialChunks) { add(chunk); } } - public PartitionHolder(PartitionHolder partitionHolder) + public PartitionHolder(PartitionHolder partitionHolder) { - this.holderSet = new TreeSet<>(); - this.holderSet.addAll(partitionHolder.holderSet); + this.holderMap = new TreeMap<>(); + this.holderMap.putAll(partitionHolder.holderMap); } - public void add(PartitionChunk chunk) + public boolean add(PartitionChunk chunk) { - holderSet.add(chunk); + return holderMap.putIfAbsent(chunk, chunk) == null; } + @Nullable public PartitionChunk remove(PartitionChunk chunk) { - if (!holderSet.isEmpty()) { - // Somewhat funky implementation in order to return the removed object as it exists in the set - SortedSet> tailSet = holderSet.tailSet(chunk, true); - if (!tailSet.isEmpty()) { - PartitionChunk element = tailSet.first(); - if (chunk.equals(element)) { - holderSet.remove(element); - return element; - } - } - } - return null; + return holderMap.remove(chunk); } public boolean isEmpty() { - return holderSet.isEmpty(); + return holderMap.isEmpty(); } public boolean isComplete() { - if (holderSet.isEmpty()) { + if (holderMap.isEmpty()) { return false; } - Iterator> iter = holderSet.iterator(); + Iterator> iter = holderMap.keySet().iterator(); PartitionChunk curr = iter.next(); @@ -117,7 +107,7 @@ public boolean isComplete() public PartitionChunk getChunk(final int partitionNum) { final Iterator> retVal = Iterators.filter( - holderSet.iterator(), + holderMap.keySet().iterator(), input -> input.getChunkNumber() == partitionNum ); @@ -127,13 +117,13 @@ public PartitionChunk getChunk(final int partitionNum) @Override public Iterator> iterator() { - return holderSet.iterator(); + return holderMap.keySet().iterator(); } @Override public Spliterator> spliterator() { - return holderSet.spliterator(); + return holderMap.keySet().spliterator(); } public Iterable payloads() @@ -153,7 +143,7 @@ public boolean equals(Object o) PartitionHolder that = (PartitionHolder) o; - if (!holderSet.equals(that.holderSet)) { + if (!holderMap.equals(that.holderMap)) { return false; } @@ -163,14 +153,14 @@ public boolean equals(Object o) @Override public int hashCode() { - return holderSet.hashCode(); + return holderMap.hashCode(); } @Override public String toString() { return "PartitionHolder{" + - "holderSet=" + holderSet + + "holderMap=" + holderMap + '}'; } } diff --git a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java index 681f6a605c5c..5b2241ec0ef7 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -240,22 +240,22 @@ public void testRemove() public void testFindEntry() { Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1") ); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1") ); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1") ); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1") ); @@ -279,7 +279,7 @@ public void testFindEntryWithOverlap() add("2011-01-02/2011-01-05", "2", 1); Assert.assertEquals( - new ImmutablePartitionHolder(new PartitionHolder(makeSingle(1))), + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))), timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1") ); } @@ -1063,7 +1063,7 @@ public void testOverlapLargeUnderlyingWithSmallDayAlignedOverlays() createExpected("2011-01-04/2011-01-05", "3", 3), createExpected("2011-01-05/2011-01-06", "4", 4) ), - timeline.lookup(Intervals.of("0000-01-01/3000-01-01")) + timeline.lookup(Intervals.ETERNITY) ); } @@ -1564,11 +1564,11 @@ public void testIsOvershadowedWithNonOverlappingSegmentsInTimeline() { timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk(1)); - add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk(1)); + add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk<>(1)); + add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk<>(1)); - add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk(1)); - add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk(1)); + add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<>(1)); + add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk<>(1)); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0")); @@ -1629,11 +1629,11 @@ public void testIsOvershadowedWithOverlappingSegmentsInTimeline() { timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk(1)); - add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk(1)); + add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk<>(1)); + add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk<>(1)); - add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk(1)); - add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk(1)); + add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk<>(1)); + add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk<>(1)); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0")); @@ -1730,13 +1730,13 @@ private Pair>> createExpected( { return Pair.of( Intervals.of(intervalString), - Pair.of(version, new PartitionHolder(values)) + Pair.of(version, new PartitionHolder<>(values)) ); } private SingleElementPartitionChunk makeSingle(Integer value) { - return new SingleElementPartitionChunk(value); + return new SingleElementPartitionChunk<>(value); } private void add(String interval, String version, Integer value) @@ -1808,7 +1808,7 @@ public Pair>> apply( private VersionedIntervalTimeline makeStringIntegerTimeline() { - return new VersionedIntervalTimeline(Ordering.natural()); + return new VersionedIntervalTimeline<>(Ordering.natural()); } } diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index fa71318fff47..0166099cb723 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -716,7 +716,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti |Property|Description|Default| |--------|-----------|-------| -|`druid.coordinator.period`|The run period for the Coordinator. The Coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S| +|`druid.coordinator.period`|The run period for the Coordinator. The Coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of "used" segments and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S| |`druid.coordinator.period.indexingPeriod`|How often to send compact/merge/conversion tasks to the indexing service. It's recommended to be longer than `druid.manager.segments.pollDuration`|PT1800S (30 mins)| |`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S| |`druid.coordinator.load.timeout`|The timeout duration for when the Coordinator assigns a segment to a Historical process.|PT15M| @@ -799,9 +799,9 @@ Issuing a GET request at the same URL will return the spec that is currently in |`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10| |`balancerComputeThreads`|Thread pool size for computing moving cost of segments in segment balancing. Consider increasing this if you have a lot of segments and moving segments starts to get stuck.|1| |`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false| -|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| +|`killDataSourceWhitelist`|List of specific data sources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated data source names or a JSON array.|none| |`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false| -|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none| +|`killPendingSegmentsSkipList`|List of data sources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated data sources or a JSON array.|none| |`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0| |`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 `decommissioningMaxPercentOfMaxSegmentsToMove`.|none| |`decommissioningMaxPercentOfMaxSegmentsToMove`| The maximum number of segments that may be moved away from 'decommissioning' servers to non-decommissioning (that is, active) servers during one Coordinator run. This value is relative to the total maximum segment movements allowed during one run which is determined by `maxSegmentsToMove`. If `decommissioningMaxPercentOfMaxSegmentsToMove` is 0, segments will neither be moved from _or to_ 'decommissioning' servers, effectively putting them in a sort of "maintenance" mode that will not participate in balancing or assignment by load rules. Decommissioning can also become stalled if there are no available active servers to place the segments. By leveraging the maximum percent of decommissioning segment movements, an operator can prevent active servers from overload by prioritizing balancing, or decrease decommissioning time instead. The value should be between 0 and 100.|70| @@ -1023,7 +1023,7 @@ Worker select strategies control how Druid assigns tasks to middleManagers. ###### Equal Distribution -Tasks are assigned to the middleManager with the most available capacity at the time the task begins running. This is +Tasks are assigned to the middleManager with the most spare capacity at the time the task begins running. This is useful if you want work evenly distributed across your middleManagers. |Property|Description|Default| @@ -1129,7 +1129,7 @@ Middle managers pass their configurations down to their child peons. The MiddleM |`druid.indexer.runner.ports`|A json array of integers to specify ports that used for peon processes. If provided and non-empty, ports for peon processes will be chosen from these ports. And `druid.indexer.runner.startPort/druid.indexer.runner.endPort` will be completely ignored.|`[]`| |`druid.worker.ip`|The IP of the worker.|localhost| |`druid.worker.version`|Version identifier for the MiddleManager.|0| -|`druid.worker.capacity`|Maximum number of tasks the MiddleManager can accept.|Number of available processors - 1| +|`druid.worker.capacity`|Maximum number of tasks the MiddleManager can accept.|Number of CPUs on the machine - 1| #### Peon Processing @@ -1456,8 +1456,9 @@ See [cache configuration](#cache-configuration) for how to configure cache setti This section describes caching configuration that is common to Broker, Historical, and MiddleManager/Peon processes. -Caching can optionally be enabled on the Broker, Historical, and MiddleManager/Peon processses. See [Broker](#broker-caching), -[Historical](#Historical-caching), and [Peon](#peon-caching) configuration options for how to enable it for different processes. +Caching can optionally be enabled on the Broker, Historical, and MiddleManager/Peon processes. See [Broker]( +#broker-caching), [Historical](#Historical-caching), and [Peon](#peon-caching) configuration options for how to enable +it for different processes. Druid uses a local in-memory cache by default, unless a diffrent type of cache is specified. Use the `druid.cache.type` configuration to set a different kind of cache. diff --git a/docs/content/dependencies/metadata-storage.md b/docs/content/dependencies/metadata-storage.md index f5cc7673a630..d55ff378119a 100644 --- a/docs/content/dependencies/metadata-storage.md +++ b/docs/content/dependencies/metadata-storage.md @@ -74,18 +74,15 @@ See [BasicDataSource Configuration](https://commons.apache.org/proper/commons-db This is dictated by the `druid.metadata.storage.tables.segments` property. -This table stores metadata about the segments that are available in the system. -The table is polled by the [Coordinator](../design/coordinator.html) to -determine the set of segments that should be available for querying in the -system. The table has two main functional columns, the other columns are for -indexing purposes. - -The `used` column is a boolean "tombstone". A 1 means that the segment should -be "used" by the cluster (i.e. it should be loaded and available for requests). -A 0 means that the segment should not be actively loaded into the cluster. We -do this as a means of removing segments from the cluster without actually -removing their metadata (which allows for simpler rolling back if that is ever -an issue). +This table stores metadata about the segments that should available in the system. (This set of segments is called +"used segments" elsewhere in the documentation and throughout the project.) The table is polled by the [Coordinator]( +../design/coordinator.html) to determine the set of segments that should be available for querying in the system. The +table has two main functional columns, the other columns are for indexing purposes. + +Value 1 in the `used` column means that the segment should be "used" by the cluster (i.e. it should be loaded and +available for requests). Value 0 means that the segment should not be loaded into the cluster. We do this as a means of +unloading segments from the cluster without actually removing their metadata (which allows for simpler rolling back if +that is ever an issue). The `payload` column stores a JSON blob that has all of the metadata for the segment (some of the data stored in this payload is redundant with some of the columns in the table, that is intentional). This looks something like diff --git a/docs/content/design/coordinator.md b/docs/content/design/coordinator.md index 810f212e604a..5cb242faed08 100644 --- a/docs/content/design/coordinator.md +++ b/docs/content/design/coordinator.md @@ -34,11 +34,24 @@ For a list of API endpoints supported by the Coordinator, see [Coordinator API]( ### Overview -The Druid Coordinator process is primarily responsible for segment management and distribution. More specifically, the Druid Coordinator process communicates to Historical processes to load or drop segments based on configurations. The Druid Coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load. - -The Druid Coordinator runs periodically and the time between each run is a configurable parameter. Each time the Druid Coordinator runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the Broker and Historical processses, the Druid Coordinator maintains a connection to a Zookeeper cluster for current cluster information. The Coordinator also maintains a connection to a database containing information about available segments and rules. Available segments are stored in a segment table and list all segments that should be loaded in the cluster. Rules are stored in a rule table and indicate how segments should be handled. - -Before any unassigned segments are serviced by Historical processes, the available Historical processes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the processes with least capacity to maintain a level of balance between processes. The Coordinator does not directly communicate with a historical process when assigning it a new segment; instead the Coordinator creates some temporary information about the new segment under load queue path of the historical process. Once this request is seen, the historical process will load the segment and begin servicing it. +The Druid Coordinator process is primarily responsible for segment management and distribution. More specifically, the +Druid Coordinator process communicates to Historical processes to load or drop segments based on configurations. The +Druid Coordinator is responsible for loading new segments, dropping outdated segments, ensuring that segments are +"replicated" (that is, loaded on multiple different Historical nodes) proper (configurated) number of times, and moving +("balancing") segments between Historical nodes to keep the latter evenly loaded. + +The Druid Coordinator runs its duties periodically and the time between each run is a configurable parameter. On each +run, the Coordinator assesses the current state of the cluster before deciding on the appropriate actions to take. +Similar to the Broker and Historical processes, the Druid Coordinator maintains a connection to a Zookeeper cluster for +current cluster information. The Coordinator also maintains a connection to a database containing information about +"used" segments (that is, the segments that *should* be loaded in the cluster) and the loading rules. + +Before any unassigned segments are serviced by Historical processes, the Historical processes for each tier are first +sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always +assigned to the processes with least capacity to maintain a level of balance between processes. The Coordinator does not +directly communicate with a historical process when assigning it a new segment; instead the Coordinator creates some +temporary information about the new segment under load queue path of the historical process. Once this request is seen, +the historical process will load the segment and begin servicing it. ### Running @@ -52,8 +65,15 @@ Segments can be automatically loaded and dropped from the cluster based on a set ### Cleaning Up Segments -Each run, the Druid Coordinator compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped. -Note that if all segments in database are deleted(or marked unused), then Coordinator will not drop anything from the Historicals. This is done to prevent a race condition in which the Coordinator would drop all segments if it started running cleanup before it finished polling the database for available segments for the first time and believed that there were no segments. +Each run, the Druid Coordinator compares the set of used segments in the database with the segments served by some nodes +in the cluster. Coordinator sends requests to Historical nodes to unload unused segments or segments the are removed +from the database. Segments that are overshadowed (their versions are too old and their data has been replaced by newer +segments) are also unloaded. + +Note that if all segments in database are deleted (or marked unused), then Coordinator will not send any unloading +requests to Historicals. This is done to prevent a race condition in which the Coordinator would drop all segments if it +started running cleanup before it finished polling the database for used segments for the first time and believed that +there were no segments. ### Segment Availability diff --git a/docs/content/design/index.md b/docs/content/design/index.md index 60a5a2bc5ed9..c08276cf055d 100644 --- a/docs/content/design/index.md +++ b/docs/content/design/index.md @@ -110,7 +110,7 @@ Druid processes can be deployed any way you like, but for ease of deployment we * **Query**: Runs Broker and optional Router processes, handles queries from external clients. * **Data**: Runs Historical and MiddleManager processes, executes ingestion workloads and stores all queryable data. -For more details on process and server organization, please see [Druid Processses and Servers](../design/processes.html). +For more details on process and server organization, please see [Druid Processes and Servers](../design/processes.html). ### External dependencies @@ -131,8 +131,8 @@ both in deep storage and across your Historical processes for the data you plan For more details, please see [Deep storage dependency](../dependencies/deep-storage.html). #### Metadata storage -The metadata storage holds various shared system metadata such as segment availability information and task information. This is typically going to be a traditional RDBMS -like PostgreSQL or MySQL. +The metadata storage holds various shared system metadata such as segment usage information and task information. This +is typically going to be a traditional RDBMS like PostgreSQL or MySQL. For more details, please see [Metadata storage dependency](../dependencies/metadata-storage.html) diff --git a/docs/content/ingestion/delete-data.md b/docs/content/ingestion/delete-data.md index 7e28ffca56bd..461e310308b0 100644 --- a/docs/content/ingestion/delete-data.md +++ b/docs/content/ingestion/delete-data.md @@ -37,7 +37,8 @@ A data deletion tutorial is available at [Tutorial: Deleting data](../tutorials/ ## Kill Task -Kill tasks delete all information about a segment and removes it from deep storage. Killable segments must be disabled (used==0) in the Druid segment table. The available grammar is: +Kill tasks delete all information about a segment and removes it from deep storage. Segments to kill must be unused +(used==0) in the Druid segment table. The available grammar is: ```json { diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 5ad0e6c00285..3adf1856f2a2 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -113,15 +113,17 @@ Returns the serialized JSON of segments to load and drop for each Historical pro * `/druid/coordinator/v1/metadata/datasources` -Returns a list of the names of enabled datasources in the cluster. +Returns a list of the names of data sources with at least one used segment in the cluster. -* `/druid/coordinator/v1/metadata/datasources?includeDisabled` +* `/druid/coordinator/v1/metadata/datasources?includeUnused` -Returns a list of the names of enabled and disabled datasources in the cluster. +Returns a list of the names of data sources, regardless of whether there are used segments belonging to those data +sources in the cluster or not. * `/druid/coordinator/v1/metadata/datasources?full` -Returns a list of all enabled datasources with all metadata about those datasources as stored in the metadata store. +Returns a list of all data sources with at least one used segment in the cluster. Returns all metadata about those data +sources as stored in the metadata store. * `/druid/coordinator/v1/metadata/datasources/{dataSourceName}` @@ -224,17 +226,17 @@ Return the tiers that a datasource exists in. * `/druid/coordinator/v1/datasources/{dataSourceName}` -Enables all segments of datasource which are not overshadowed by others. +Marks as used all segments belonging to a data source. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` -Enables a segment of a datasource. +Marks as used a segment of a data source. ##### DELETE * `/druid/coordinator/v1/datasources/{dataSourceName}` -Disables a datasource. +Marks as unused all segments belonging to a data source. * `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` * `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}` @@ -243,7 +245,7 @@ Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` -Disables a segment. +Marks as unused a segment of a data source. #### Retention Rules diff --git a/docs/content/operations/metrics.md b/docs/content/operations/metrics.md index 99934cb65107..64cbdd8bf63a 100644 --- a/docs/content/operations/metrics.md +++ b/docs/content/operations/metrics.md @@ -205,8 +205,8 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina |`segment/loadQueue/failed`|Number of segments that failed to load.|server.|0| |`segment/loadQueue/count`|Number of segments to load.|server.|Varies.| |`segment/dropQueue/count`|Number of segments to drop.|server.|Varies.| -|`segment/size`|Size in bytes of available segments.|dataSource.|Varies.| -|`segment/count`|Number of available segments.|dataSource.|< max| +|`segment/size`|Sum of sizes of all used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|Varies.| +|`segment/count`|Number of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|< max| |`segment/overShadowed/count`|Number of overShadowed segments.||Varies.| |`segment/unavailable/count`|Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries.|datasource.|0| |`segment/underReplicated/count`|Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries.|tier, datasource.|0| diff --git a/docs/content/operations/rule-configuration.md b/docs/content/operations/rule-configuration.md index 4bc37a8578d6..9f49571abf3f 100644 --- a/docs/content/operations/rule-configuration.md +++ b/docs/content/operations/rule-configuration.md @@ -29,7 +29,10 @@ Coordinator processes use rules to determine what data should be loaded to or dr There are three types of rules, i.e., load rules, drop rules, and broadcast rules. Load rules indicate how segments should be assigned to different historical process tiers and how many replicas of a segment should exist in each tier. Drop rules indicate when segments should be dropped entirely from the cluster. Finally, broadcast rules indicate how segments of different data sources should be co-located in Historical processes. -The Coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The Coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule. +The Coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain data source and/or a +default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The +Coordinator will cycle through all used segments and match each segment with the first rule that applies. Each segment +may only match a single rule. Note: It is recommended that the Coordinator console is used to configure rules. However, the Coordinator process does have HTTP endpoints to programmatically configure rules. @@ -236,5 +239,7 @@ marked as unused (segments dropped from the cluster via rules are always marked # Reloading Dropped Data -Data that has been dropped from a Druid cluster cannot be reloaded using only rules. To reload dropped data in Druid, you must first set your retention period (i.e. changing the retention period from 1 month to 2 months), and -then enable the datasource in the Druid Coordinator console, or through the Druid Coordinator endpoints. +Data that has been dropped from a Druid cluster cannot be reloaded using only rules. To reload dropped data in Druid, +you must first set your retention period (i.e. changing the retention period from 1 month to 2 months), and then +mark as used all segments belonging to the data source in the Druid Coordinator console, or through the Druid +Coordinator endpoints. diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index 105afdf8f23f..c6bb1c39435c 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -46,7 +46,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SQLMetadataSegmentManager; +import org.apache.druid.metadata.SqlMetadataSegments; import org.apache.druid.timeline.DataSegment; import org.joda.time.Duration; import org.joda.time.Interval; @@ -66,13 +66,12 @@ public class MaterializedViewSupervisor implements Supervisor { private static final EmittingLogger log = new EmittingLogger(MaterializedViewSupervisor.class); - private static final Interval ALL_INTERVAL = Intervals.of("0000-01-01/3000-01-01"); private static final int DEFAULT_MAX_TASK_COUNT = 1; // there is a lag between derivatives and base dataSource, to prevent repeatedly building for some delay data. private static final long DEFAULT_MIN_DATA_LAG_MS = 24 * 3600 * 1000L; private final MetadataSupervisorManager metadataSupervisorManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final SQLMetadataSegmentManager segmentManager; + private final SqlMetadataSegments metadataSegments; private final MaterializedViewSupervisorSpec spec; private final TaskMaster taskMaster; private final TaskStorage taskStorage; @@ -98,7 +97,7 @@ public MaterializedViewSupervisor( TaskMaster taskMaster, TaskStorage taskStorage, MetadataSupervisorManager metadataSupervisorManager, - SQLMetadataSegmentManager segmentManager, + SqlMetadataSegments metadataSegments, IndexerMetadataStorageCoordinator metadataStorageCoordinator, MaterializedViewTaskConfig config, MaterializedViewSupervisorSpec spec @@ -107,7 +106,7 @@ public MaterializedViewSupervisor( this.taskMaster = taskMaster; this.taskStorage = taskStorage; this.metadataStorageCoordinator = metadataStorageCoordinator; - this.segmentManager = segmentManager; + this.metadataSegments = metadataSegments; this.metadataSupervisorManager = metadataSupervisorManager; this.config = config; this.spec = spec; @@ -319,17 +318,14 @@ Pair, Map>> checkSegment // Pair version, interval -> list> Pair, Map>> derivativeSegmentsSnapshot = getVersionAndBaseSegments( - metadataStorageCoordinator.getUsedSegmentsForInterval( - dataSource, - ALL_INTERVAL - ) + metadataStorageCoordinator.getUsedSegmentsForInterval(dataSource, Intervals.ETERNITY) ); // Pair max(created_date), interval -> list> Pair, Map>> baseSegmentsSnapshot = getMaxCreateDateAndBaseSegments( metadataStorageCoordinator.getUsedSegmentAndCreatedDateForInterval( spec.getBaseDataSource(), - ALL_INTERVAL + Intervals.ETERNITY ) ); // baseSegments are used to create HadoopIndexTask @@ -365,7 +361,7 @@ Pair, Map>> checkSegment // drop derivative segments which interval equals the interval in toDeleteBaseSegments for (Interval interval : toDropInterval.keySet()) { for (DataSegment segment : derivativeSegments.get(interval)) { - segmentManager.removeSegment(segment.getId()); + metadataSegments.tryMarkSegmentAsUnused(segment.getId()); } } // data of the latest interval will be built firstly. @@ -472,8 +468,10 @@ private void clearTasks() private void clearSegments() { log.info("Clear all metadata of dataSource %s", dataSource); - metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL); - segmentManager.removeDataSource(dataSource); + metadataStorageCoordinator.deletePendingSegments(dataSource, Intervals.ETERNITY); + if (!metadataSegments.tryMarkAsUnusedAllSegmentsInDataSource(dataSource)) { + log.error("Failed to mark all segments in " + dataSource + " as unused."); + } metadataStorageCoordinator.deleteDataSourceMetadata(dataSource); } diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index 3a9a1137b319..36065f662706 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -42,7 +42,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SQLMetadataSegmentManager; +import org.apache.druid.metadata.SqlMetadataSegments; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; @@ -75,7 +75,7 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec private final ObjectMapper objectMapper; private final MetadataSupervisorManager metadataSupervisorManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final SQLMetadataSegmentManager segmentManager; + private final SqlMetadataSegments metadataSegments; private final TaskMaster taskMaster; private final TaskStorage taskStorage; private final MaterializedViewTaskConfig config; @@ -98,7 +98,7 @@ public MaterializedViewSupervisorSpec( @JacksonInject TaskMaster taskMaster, @JacksonInject TaskStorage taskStorage, @JacksonInject MetadataSupervisorManager metadataSupervisorManager, - @JacksonInject SQLMetadataSegmentManager segmentManager, + @JacksonInject SqlMetadataSegments metadataSegments, @JacksonInject IndexerMetadataStorageCoordinator metadataStorageCoordinator, @JacksonInject MaterializedViewTaskConfig config, @JacksonInject AuthorizerMapper authorizerMapper, @@ -136,7 +136,7 @@ public MaterializedViewSupervisorSpec( this.taskMaster = taskMaster; this.taskStorage = taskStorage; this.metadataSupervisorManager = metadataSupervisorManager; - this.segmentManager = segmentManager; + this.metadataSegments = metadataSegments; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; this.chatHandlerProvider = chatHandlerProvider; @@ -329,7 +329,7 @@ public Supervisor createSupervisor() taskMaster, taskStorage, metadataSupervisorManager, - segmentManager, + metadataSegments, metadataStorageCoordinator, config, this @@ -360,7 +360,7 @@ public SupervisorSpec createSuspendedSpec() taskMaster, taskStorage, metadataSupervisorManager, - segmentManager, + metadataSegments, metadataStorageCoordinator, config, authorizerMapper, @@ -386,7 +386,7 @@ public SupervisorSpec createRunningSpec() taskMaster, taskStorage, metadataSupervisorManager, - segmentManager, + metadataSegments, metadataStorageCoordinator, config, authorizerMapper, diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java index 97d5dba62c02..9168b1744c3d 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java @@ -31,7 +31,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SQLMetadataSegmentManager; +import org.apache.druid.metadata.SqlMetadataSegments; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -69,7 +69,7 @@ public void setup() .addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class, objectMapper) .addValue(MetadataSupervisorManager.class, null) - .addValue(SQLMetadataSegmentManager.class, null) + .addValue(SqlMetadataSegments.class, null) .addValue(IndexerMetadataStorageCoordinator.class, null) .addValue(MaterializedViewTaskConfig.class, new MaterializedViewTaskConfig()) .addValue(AuthorizerMapper.class, createMock(AuthorizerMapper.class)) diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 1bf1c39709d5..251d3ff4f34f 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -40,7 +40,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SQLMetadataSegmentManager; +import org.apache.druid.metadata.SqlMetadataSegments; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -82,7 +82,7 @@ public class MaterializedViewSupervisorTest private TaskMaster taskMaster; private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private MetadataSupervisorManager metadataSupervisorManager; - private SQLMetadataSegmentManager sqlMetadataSegmentManager; + private SqlMetadataSegments sqlMetadataSegments; private TaskQueue taskQueue; private MaterializedViewSupervisor supervisor; private MaterializedViewSupervisorSpec spec; @@ -102,7 +102,7 @@ public void setUp() derbyConnector ); metadataSupervisorManager = createMock(MetadataSupervisorManager.class); - sqlMetadataSegmentManager = createMock(SQLMetadataSegmentManager.class); + sqlMetadataSegments = createMock(SqlMetadataSegments.class); taskQueue = createMock(TaskQueue.class); taskQueue.start(); objectMapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); @@ -121,7 +121,7 @@ public void setUp() taskMaster, taskStorage, metadataSupervisorManager, - sqlMetadataSegmentManager, + sqlMetadataSegments, indexerMetadataStorageCoordinator, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), @@ -278,7 +278,7 @@ public void testSuspendedDoesntRun() taskMaster, taskStorage, metadataSupervisorManager, - sqlMetadataSegmentManager, + sqlMetadataSegments, indexerMetadataStorageCoordinator, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java index a1eb90f88e76..92daf2130eed 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java @@ -45,35 +45,34 @@ public ActionBasedUsedSegmentChecker(TaskActionClient taskActionClient) } @Override - public Set findUsedSegments(Set identifiers) throws IOException + public Set findUsedSegments(Set segmentIds) throws IOException { // Group by dataSource - final Map> identifiersByDataSource = new TreeMap<>(); - for (SegmentIdWithShardSpec identifier : identifiers) { - if (!identifiersByDataSource.containsKey(identifier.getDataSource())) { - identifiersByDataSource.put(identifier.getDataSource(), new HashSet<>()); - } - identifiersByDataSource.get(identifier.getDataSource()).add(identifier); + final Map> idsByDataSource = new TreeMap<>(); + for (SegmentIdWithShardSpec segmentId : segmentIds) { + idsByDataSource.computeIfAbsent(segmentId.getDataSource(), i -> new HashSet<>()).add(segmentId); } - final Set retVal = new HashSet<>(); + final Set usedSegments = new HashSet<>(); - for (Map.Entry> entry : identifiersByDataSource.entrySet()) { + for (Map.Entry> entry : idsByDataSource.entrySet()) { + String dataSource = entry.getKey(); + Set segmentIdsInDataSource = entry.getValue(); final List intervals = JodaUtils.condenseIntervals( - Iterables.transform(entry.getValue(), input -> input.getInterval()) + Iterables.transform(segmentIdsInDataSource, SegmentIdWithShardSpec::getInterval) ); final List usedSegmentsForIntervals = taskActionClient.submit( - new SegmentListUsedAction(entry.getKey(), null, intervals) + new SegmentListUsedAction(dataSource, null, intervals) ); for (DataSegment segment : usedSegmentsForIntervals) { - if (identifiers.contains(SegmentIdWithShardSpec.fromDataSegment(segment))) { - retVal.add(segment); + if (segmentIds.contains(SegmentIdWithShardSpec.fromDataSegment(segment))) { + usedSegments.add(segment); } } } - return retVal; + return usedSegments; } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java index ecbd8ca78800..0fa6d5630655 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexTaskRunner.java @@ -423,8 +423,7 @@ private void publish(TaskToolbox toolbox) throws IOException .flatMap(report -> report.getSegments().stream()) .map(SegmentIdWithShardSpec::fromDataSegment) .collect(Collectors.toSet()); - if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers) - .equals(segmentsToPublish)) { + if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers).equals(segmentsToPublish)) { log.info("Our segments really do exist, awaiting handoff."); } else { throw new ISE("Failed to publish segments[%s]", segmentsToPublish); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index 5ccd9d379935..88414ead8353 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -146,12 +146,12 @@ public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) List usedSegments; while (true) { try { - usedSegments = - coordinatorClient.getDatabaseSegmentDataSourceSegments(dataSource, Collections.singletonList(interval)); + List intervals = Collections.singletonList(interval); + usedSegments = coordinatorClient.getUsedSegmentsInDataSourceForIntervals(dataSource, intervals); break; } catch (Throwable e) { - log.warn(e, "Exception getting database segments"); + log.warn(e, "Exception getting used segments for interval"); final Duration delay = retryPolicy.getAndIncrementRetryDelay(); if (delay == null) { throw e; diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java index 7d5c64c2c61c..4c2edd0d6940 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java @@ -73,7 +73,7 @@ public void setup() throws IOException expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); - expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().removeSegment(s.getId())); + expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegments().tryMarkSegmentAsUnused(s.getId())); } private DataSegment createSegment(Interval interval, String version) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index ceb65d1f31e7..980c67a699de 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -29,11 +29,11 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.MetadataSegmentManager; -import org.apache.druid.metadata.MetadataSegmentManagerConfig; +import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.MetadataSegmentsConfig; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.metadata.SQLMetadataSegmentManager; +import org.apache.druid.metadata.SqlMetadataSegments; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; @@ -48,7 +48,7 @@ public class TaskActionTestKit extends ExternalResource private TaskLockbox taskLockbox; private TestDerbyConnector testDerbyConnector; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private MetadataSegmentManager metadataSegmentManager; + private MetadataSegments metadataSegments; private TaskActionToolbox taskActionToolbox; public TaskLockbox getTaskLockbox() @@ -61,9 +61,9 @@ public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator() return metadataStorageCoordinator; } - public MetadataSegmentManager getMetadataSegmentManager() + public MetadataSegments getMetadataSegments() { - return metadataSegmentManager; + return metadataSegments; } public TaskActionToolbox getTaskActionToolbox() @@ -86,9 +86,9 @@ public void before() metadataStorageTablesConfig, testDerbyConnector ); - metadataSegmentManager = new SQLMetadataSegmentManager( + metadataSegments = new SqlMetadataSegments( objectMapper, - Suppliers.ofInstance(new MetadataSegmentManagerConfig()), + Suppliers.ofInstance(new MetadataSegmentsConfig()), Suppliers.ofInstance(metadataStorageTablesConfig), testDerbyConnector ); @@ -116,7 +116,7 @@ public void after() taskLockbox = null; testDerbyConnector = null; metadataStorageCoordinator = null; - metadataSegmentManager = null; + metadataSegments = null; taskActionToolbox = null; } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java index ea109c8b6cdc..8d71cf3f8448 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java @@ -113,7 +113,7 @@ public CompactionTaskRunTest() coordinatorClient = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public List getUsedSegmentsInDataSourceForIntervals(String dataSource, List intervals) { return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index cbd52f58d57d..940bf4ff4383 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -1325,7 +1325,7 @@ private static class TestCoordinatorClient extends CoordinatorClient } @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public List getUsedSegmentsInDataSourceForIntervals(String dataSource, List intervals) { return new ArrayList<>(segmentMap.keySet()); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 7f44ad6a528e..a456387675fc 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -160,7 +160,7 @@ public static Collection constructorFeeder() throws IOException final CoordinatorClient cc = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public List getUsedSegmentsInDataSourceForIntervals(String dataSource, List intervals) { return ImmutableList.copyOf(segmentSet); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index e224b953efbd..ef8c7e750a7d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -276,7 +276,7 @@ public static Collection constructorFeeder() final CoordinatorClient cc = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public List getUsedSegmentsInDataSourceForIntervals(String dataSource, List intervals) { // Expect the interval we asked for if (intervals.equals(ImmutableList.of(testCase.interval))) { diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java index 59539443b618..6be3a6452879 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java @@ -146,6 +146,6 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(name, properties, idToSegments); + return Objects.hash(name, properties); } } diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java index d01ad961a70c..605b2de877ff 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java @@ -42,19 +42,19 @@ public class ImmutableDruidServer private final DruidServerMetadata metadata; private final long currSize; private final ImmutableMap dataSources; - private final int totalSegments; + private final int numSegments; public ImmutableDruidServer( DruidServerMetadata metadata, long currSize, ImmutableMap dataSources, - int totalSegments + int numSegments ) { this.metadata = Preconditions.checkNotNull(metadata); this.currSize = currSize; this.dataSources = dataSources; - this.totalSegments = totalSegments; + this.numSegments = numSegments; } public String getName() @@ -128,23 +128,26 @@ public ImmutableDruidDataSource getDataSource(String name) } /** - * Returns a lazy collection with all segments in all data sources, stored on this ImmutableDruidServer. The order - * of segments in this collection is unspecified. - * - * Calling {@link Collection#size()} on the returned collection is cheap, O(1). + * Returns a lazy collection with all segments in all data sources stored on this ImmutableDruidServer to be used for + * iteration or {@link Collection#stream()} transformation. The order of segments in this collection is unspecified. * * Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only * once rather than several times. */ - public Collection getLazyAllSegments() + public Collection iterateAllSegments() { return CollectionUtils.createLazyCollectionFromStream( () -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()), - totalSegments + numSegments ); } + public int getNumSegments() + { + return numSegments; + } + public String getURL() { if (metadata.getHostAndTlsPort() != null) { diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index 418854059389..a92d1f1bec13 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -126,7 +126,7 @@ public List fetchServerView(String dataSource, Interva } } - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public List getUsedSegmentsInDataSourceForIntervals(String dataSource, List intervals) { try { FullResponseHolder response = druidLeaderClient.go( @@ -141,7 +141,7 @@ public List getDatabaseSegmentDataSourceSegments(String dataSource, if (!response.getStatus().equals(HttpResponseStatus.OK)) { throw new ISE( - "Error while fetching database segment data source segments status[%s] content[%s]", + "Error while fetching used segments in a data source for intervals: status[%s] content[%s]", response.getStatus(), response.getContent() ); diff --git a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java index 99545650dc76..396922d255ee 100644 --- a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java +++ b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java @@ -22,7 +22,7 @@ import com.google.inject.Binder; import com.google.inject.Module; import org.apache.druid.metadata.MetadataRuleManagerConfig; -import org.apache.druid.metadata.MetadataSegmentManagerConfig; +import org.apache.druid.metadata.MetadataSegmentsConfig; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -34,7 +34,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.metadata.storage.tables", MetadataStorageTablesConfig.class); JsonConfigProvider.bind(binder, "druid.metadata.storage.connector", MetadataStorageConnectorConfig.class); - JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentsConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); } } diff --git a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java index 955ab6bb4be4..10fe2ee60ed3 100644 --- a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java @@ -29,8 +29,8 @@ import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManagerProvider; -import org.apache.druid.metadata.MetadataSegmentManager; -import org.apache.druid.metadata.MetadataSegmentManagerProvider; +import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.MetadataSegmentsProvider; import org.apache.druid.metadata.MetadataSegmentPublisher; import org.apache.druid.metadata.MetadataSegmentPublisherProvider; import org.apache.druid.metadata.MetadataStorageActionHandlerFactory; @@ -40,8 +40,8 @@ import org.apache.druid.metadata.SQLMetadataConnector; import org.apache.druid.metadata.SQLMetadataRuleManager; import org.apache.druid.metadata.SQLMetadataRuleManagerProvider; -import org.apache.druid.metadata.SQLMetadataSegmentManager; -import org.apache.druid.metadata.SQLMetadataSegmentManagerProvider; +import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlMetadataSegmentsProvider; import org.apache.druid.metadata.SQLMetadataSegmentPublisher; import org.apache.druid.metadata.SQLMetadataSegmentPublisherProvider; import org.apache.druid.metadata.SQLMetadataSupervisorManager; @@ -73,8 +73,8 @@ public void createBindingChoices(Binder binder, String defaultValue) PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(SQLMetadataConnector.class), defaultValue); - PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentManager.class), defaultValue); - PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentManagerProvider.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegments.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentsProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManager.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManagerProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentPublisher.class), defaultValue); @@ -90,14 +90,14 @@ public void createBindingChoices(Binder binder, String defaultValue) @Override public void configure(Binder binder) { - PolyBind.optionBinder(binder, Key.get(MetadataSegmentManager.class)) + PolyBind.optionBinder(binder, Key.get(MetadataSegments.class)) .addBinding(type) - .to(SQLMetadataSegmentManager.class) + .to(SqlMetadataSegments.class) .in(LazySingleton.class); - PolyBind.optionBinder(binder, Key.get(MetadataSegmentManagerProvider.class)) + PolyBind.optionBinder(binder, Key.get(MetadataSegmentsProvider.class)) .addBinding(type) - .to(SQLMetadataSegmentManagerProvider.class) + .to(SqlMetadataSegmentsProvider.class) .in(LazySingleton.class); PolyBind.optionBinder(binder, Key.get(MetadataRuleManager.class)) diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 95c1fce15c6e..f628b4c1c760 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -64,7 +64,8 @@ default List getUsedSegmentsForInterval(String dataSource, Interval * @param dataSource The datasource to query * @param intervals The intervals for which all applicable and used datasources are requested. * - * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the requested interval. + * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the + * requested interval. * * @throws IOException */ diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java deleted file mode 100644 index 436ad125bf7d..000000000000 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ /dev/null @@ -1,80 +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.metadata; - -import com.google.common.annotations.VisibleForTesting; -import org.apache.druid.client.ImmutableDruidDataSource; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.SegmentId; -import org.joda.time.Interval; - -import javax.annotation.Nullable; -import java.util.Collection; -import java.util.List; - -/** - */ -public interface MetadataSegmentManager -{ - void start(); - - void stop(); - - boolean enableDataSource(String dataSource); - - boolean enableSegment(String segmentId); - - boolean removeDataSource(String dataSource); - - /** - * Prefer {@link #removeSegment(SegmentId)} to this method when possible. - * - * This method is not removed because {@link org.apache.druid.server.http.DataSourcesResource#deleteDatasourceSegment} - * uses it and if it migrates to {@link #removeSegment(SegmentId)} the performance will be worse. - */ - boolean removeSegment(String dataSource, String segmentId); - - boolean removeSegment(SegmentId segmentId); - - boolean isStarted(); - - @Nullable - ImmutableDruidDataSource getDataSource(String dataSourceName); - - Collection getDataSources(); - - /** - * Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is - * unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try - * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than - * several times. - */ - Iterable iterateAllSegments(); - - Collection getAllDataSourceNames(); - - /** - * Returns top N unused segment intervals in given interval when ordered by segment start time, end time. - */ - List getUnusedSegmentIntervals(String dataSource, Interval interval, int limit); - - @VisibleForTesting - void poll(); -} diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegments.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegments.java new file mode 100644 index 000000000000..a66d203f7b1e --- /dev/null +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegments.java @@ -0,0 +1,121 @@ +/* + * 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.metadata; + +import com.google.common.annotations.VisibleForTesting; +import org.apache.druid.client.DruidDataSource; +import org.apache.druid.client.ImmutableDruidDataSource; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.SegmentId; +import org.joda.time.Interval; + +import javax.annotation.Nullable; +import java.util.Collection; +import java.util.List; + +/** + */ +public interface MetadataSegments +{ + void start(); + + void stop(); + + boolean tryMarkAsUsedAllSegmentsInDataSource(String dataSource); + + boolean tryMarkSegmentAsUsed(String segmentId); + + boolean tryMarkAsUnusedAllSegmentsInDataSource(String dataSource); + + /** + * Prefer {@link #tryMarkSegmentAsUnused(SegmentId)} to this method when possible. + * + * This method is not removed because {@link org.apache.druid.server.http.DataSourcesResource#removeSegment} + * uses it and if it migrates to {@link #tryMarkSegmentAsUnused(SegmentId)} the performance will be worse. + */ + boolean tryMarkSegmentAsUnused(String dataSource, String segmentId); + + boolean tryMarkSegmentAsUnused(SegmentId segmentId); + + boolean isStarted(); + + /** + * If there are used segments belonging to the given data source, this method converts this set of segments to an + * {@link ImmutableDruidDataSource} object and returns. If there are no used segments belonging to the given data + * source, this method returns null. + * + * This method's name starts with "prepare" to indicate that it's not cheap (it creates an {@link + * ImmutableDruidDataSource} object). Not used "create" prefix to avoid giving a false impression that this method + * might put something into the database to create a data source with the given name, if absent. + */ + @Nullable + ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSource); + + /** + * If there are used segments belonging to the given data source, this method returns a {@link DruidDataSource} object + * with a view on those segments. If there are no used segments belonging to the given data source, this method + * returns null. + * + * Note that the returned {@link DruidDataSource} object may be updated concurrently and already be empty by the time + * it is returned. + */ + @Nullable + DruidDataSource getDataSourceWithUsedSegments(String dataSource); + + /** + * Prepares a set of {@link ImmutableDruidDataSource} objects containing information about all used segments. {@link + * ImmutableDruidDataSource} objects in the returned collection are unique. If there are no used segments, this method + * returns an empty collection. + * + * This method's name starts with "prepare" for the same reason as {@link + * #prepareImmutableDataSourceWithUsedSegments}. + */ + Collection prepareImmutableDataSourcesWithAllUsedSegments(); + + /** + * Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is + * unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try + * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than + * several times. + */ + Iterable iterateAllUsedSegments(); + + /** + * Retrieves all data source names for which there are segment in the database, regardless of whether those segments + * are used or not. Data source names in the returned collection are unique. If there are no segments in the database, + * returns an empty collection. + * + * Performance warning: this method makes a query into the database. + * + * This method might return a different set of data source names than may be observed via {@link + * #prepareImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there + * are no used segments belonging to it, while {@link #prepareImmutableDataSourcesWithAllUsedSegments} won't return + * such a data source. + */ + Collection retrieveAllDataSourceNames(); + + /** + * Returns top N unused segment intervals in given interval when ordered by segment start time, end time. + */ + List getUnusedSegmentIntervals(String dataSource, Interval interval, int limit); + + @VisibleForTesting + void poll(); +} diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsConfig.java similarity index 96% rename from server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java rename to server/src/main/java/org/apache/druid/metadata/MetadataSegmentsConfig.java index 3f58cfa70982..0e6b686181dc 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsConfig.java @@ -24,7 +24,7 @@ /** */ -public class MetadataSegmentManagerConfig +public class MetadataSegmentsConfig { @JsonProperty private Period pollDuration = new Period("PT1M"); diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerProvider.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsProvider.java similarity index 88% rename from server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerProvider.java rename to server/src/main/java/org/apache/druid/metadata/MetadataSegmentsProvider.java index c57f5bf66d9c..c1e0900a6135 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsProvider.java @@ -23,8 +23,8 @@ /** */ -public interface MetadataSegmentManagerProvider extends Provider +public interface MetadataSegmentsProvider extends Provider { @Override - MetadataSegmentManager get(); + MetadataSegments get(); } diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java index be95669feb25..2aff2f2fce82 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java @@ -148,7 +148,7 @@ public Void withHandle(Handle handle) throws Exception * the theoretical situation of two tasks scheduled in {@link #start()} calling {@link #poll()} concurrently, if * the sequence of {@link #start()} - {@link #stop()} - {@link #start()} actions occurs quickly. * - * {@link SQLMetadataSegmentManager} also have a similar issue. + * {@link SqlMetadataSegments} also have a similar issue. */ private long currentStartOrder = -1; private ScheduledExecutorService exec = null; diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java similarity index 91% rename from server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java rename to server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java index dff9c9df61ca..752791938a70 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource; @@ -75,9 +76,9 @@ /** */ @ManageLifecycle -public class SQLMetadataSegmentManager implements MetadataSegmentManager +public class SqlMetadataSegments implements MetadataSegments { - private static final EmittingLogger log = new EmittingLogger(SQLMetadataSegmentManager.class); + private static final EmittingLogger log = new EmittingLogger(SqlMetadataSegments.class); /** * Use to synchronize {@link #start()}, {@link #stop()}, {@link #poll()}, and {@link #isStarted()}. These methods @@ -98,16 +99,16 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager private final Object pollLock = new Object(); private final ObjectMapper jsonMapper; - private final Supplier config; + private final Supplier config; private final Supplier dbTables; private final SQLMetadataConnector connector; private ConcurrentHashMap dataSources = new ConcurrentHashMap<>(); - /** The number of times this SQLMetadataSegmentManager was started. */ + /** The number of times this SqlMetadataSegments was started. */ private long startCount = 0; /** - * Equal to the current {@link #startCount} value, if the SQLMetadataSegmentManager is currently started; -1 if + * Equal to the current {@link #startCount} value, if the SqlMetadataSegments is currently started; -1 if * currently stopped. * * This field is used to implement a simple stamp mechanism instead of just a boolean "started" flag to prevent @@ -121,9 +122,9 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager private ScheduledExecutorService exec = null; @Inject - public SQLMetadataSegmentManager( + public SqlMetadataSegments( ObjectMapper jsonMapper, - Supplier config, + Supplier config, Supplier dbTables, SQLMetadataConnector connector ) @@ -149,7 +150,7 @@ public void start() currentStartOrder = startCount; final long localStartOrder = currentStartOrder; - exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); + exec = Execs.scheduledSingleThreaded(getClass().getName() + "-Exec--%d"); final Duration delay = config.get().getPollDuration().toStandardDuration(); exec.scheduleWithFixedDelay( @@ -170,7 +171,7 @@ private Runnable createPollTaskForStartOrder(long startOrder) // poll() is synchronized together with start(), stop() and isStarted() to ensure that when stop() exits, poll() // won't actually run anymore after that (it could only enter the syncrhonized section and exit immediately // because the localStartedOrder doesn't match the new currentStartOrder). It's needed to avoid flakiness in - // SQLMetadataSegmentManagerTest. See https://github.com/apache/incubator-druid/issues/6028 + // SqlMetadataSegmentsTest. See https://github.com/apache/incubator-druid/issues/6028 ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock(); lock.lock(); try { @@ -181,7 +182,7 @@ private Runnable createPollTaskForStartOrder(long startOrder) } } catch (Exception e) { - log.makeAlert(e, "uncaught exception in segment manager polling thread").emit(); + log.makeAlert(e, "uncaught exception in " + getClass().getName() + "'s polling thread").emit(); } finally { lock.unlock(); @@ -211,7 +212,7 @@ public void stop() } @Override - public boolean enableDataSource(final String dataSource) + public boolean tryMarkAsUsedAllSegmentsInDataSource(final String dataSource) { try { final IDBI dbi = connector.getDBI(); @@ -244,7 +245,7 @@ public boolean enableDataSource(final String dataSource) final List segments = new ArrayList<>(); List> timelineObjectHolders = segmentTimeline.lookup( - Intervals.of("0000-01-01/3000-01-01") + Intervals.ETERNITY ); for (TimelineObjectHolder objectHolder : timelineObjectHolders) { for (PartitionChunk partitionChunk : objectHolder.getObject()) { @@ -254,7 +255,7 @@ public boolean enableDataSource(final String dataSource) if (segments.isEmpty()) { log.warn("No segments found in the database!"); - return false; + return true; } dbi.withHandle( @@ -282,7 +283,7 @@ public Void withHandle(Handle handle) ); } catch (Exception e) { - log.error(e, "Exception enabling datasource %s", dataSource); + log.error(e, "Exception marking all segments as used in the data source %s", dataSource); return false; } @@ -290,7 +291,7 @@ public Void withHandle(Handle handle) } @Override - public boolean enableSegment(final String segmentId) + public boolean tryMarkSegmentAsUsed(final String segmentId) { try { connector.getDBI().withHandle( @@ -308,7 +309,7 @@ public Void withHandle(Handle handle) ); } catch (Exception e) { - log.error(e, "Exception enabling segment %s", segmentId); + log.error(e, "Exception marking segment %s as used", segmentId); return false; } @@ -316,7 +317,7 @@ public Void withHandle(Handle handle) } @Override - public boolean removeDataSource(final String dataSource) + public boolean tryMarkAsUnusedAllSegmentsInDataSource(final String dataSource) { try { final int removed = connector.getDBI().withHandle( @@ -340,7 +341,7 @@ public boolean removeDataSource(final String dataSource) } @Override - public boolean removeSegment(String dataSourceName, final String segmentId) + public boolean tryMarkSegmentAsUnused(String dataSourceName, final String segmentId) { try { final boolean removed = removeSegmentFromTable(segmentId); @@ -371,7 +372,7 @@ public boolean removeSegment(String dataSourceName, final String segmentId) } @Override - public boolean removeSegment(SegmentId segmentId) + public boolean tryMarkSegmentAsUnused(SegmentId segmentId) { try { final boolean removed = removeSegmentFromTable(segmentId.toString()); @@ -420,14 +421,20 @@ public boolean isStarted() @Override @Nullable - public ImmutableDruidDataSource getDataSource(String dataSourceName) + public ImmutableDruidDataSource prepareImmutableDataSourceWithUsedSegments(String dataSourceName) { final DruidDataSource dataSource = dataSources.get(dataSourceName); return dataSource == null ? null : dataSource.toImmutableDruidDataSource(); } @Override - public Collection getDataSources() + public @Nullable DruidDataSource getDataSourceWithUsedSegments(String dataSource) + { + return dataSources.get(dataSource); + } + + @Override + public Collection prepareImmutableDataSourcesWithAllUsedSegments() { return dataSources.values() .stream() @@ -436,13 +443,13 @@ public Collection getDataSources() } @Override - public Iterable iterateAllSegments() + public Iterable iterateAllUsedSegments() { return () -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()).iterator(); } @Override - public Collection getAllDataSourceNames() + public Collection retrieveAllDataSourceNames() { return connector.getDBI().withHandle( handle -> handle.createQuery( @@ -484,6 +491,8 @@ public void poll() } } + /** This method is extracted from {@link #poll()} solely to reduce code nesting. */ + @GuardedBy("pollLock") private void doPoll() { log.debug("Starting polling of segment table"); diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManagerProvider.java b/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegmentsProvider.java similarity index 85% rename from server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManagerProvider.java rename to server/src/main/java/org/apache/druid/metadata/SqlMetadataSegmentsProvider.java index 0644e7feddf0..3050e9889873 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManagerProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegmentsProvider.java @@ -26,18 +26,18 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; -public class SQLMetadataSegmentManagerProvider implements MetadataSegmentManagerProvider +public class SqlMetadataSegmentsProvider implements MetadataSegmentsProvider { private final ObjectMapper jsonMapper; - private final Supplier config; + private final Supplier config; private final Supplier storageConfig; private final SQLMetadataConnector connector; private final Lifecycle lifecycle; @Inject - public SQLMetadataSegmentManagerProvider( + public SqlMetadataSegmentsProvider( ObjectMapper jsonMapper, - Supplier config, + Supplier config, Supplier storageConfig, SQLMetadataConnector connector, Lifecycle lifecycle @@ -51,7 +51,7 @@ public SQLMetadataSegmentManagerProvider( } @Override - public MetadataSegmentManager get() + public MetadataSegments get() { lifecycle.addHandler( new Lifecycle.Handler() @@ -70,7 +70,7 @@ public void stop() } ); - return new SQLMetadataSegmentManager( + return new SqlMetadataSegments( jsonMapper, config, storageConfig, diff --git a/server/src/main/java/org/apache/druid/server/JettyUtils.java b/server/src/main/java/org/apache/druid/server/JettyUtils.java index 9374fb07d931..717eb136b43a 100644 --- a/server/src/main/java/org/apache/druid/server/JettyUtils.java +++ b/server/src/main/java/org/apache/druid/server/JettyUtils.java @@ -19,10 +19,16 @@ package org.apache.druid.server; +import org.apache.druid.java.util.common.logger.Logger; + import javax.annotation.Nullable; +import javax.ws.rs.core.MultivaluedMap; +import javax.ws.rs.core.UriInfo; public class JettyUtils { + private static final Logger log = new Logger(JettyUtils.class); + /** * Concatenate URI parts, in a way that is useful for proxy servlets. * @@ -46,4 +52,38 @@ public static String concatenateForRewrite( return url.toString(); } + + /** + * Returns the value of the query parameter of the given name. If not found, but there is a value corresponding to + * the parameter of the given compatiblityName it is returned instead and a warning is logged suggestion to make + * queries using the new parameter name. + * + * This method is useful for renaming query parameters (from name to compatiblityName) while preserving backward + * compatibility of the REST API. + */ + @Nullable + public static String getQueryParam(UriInfo uriInfo, String name, String compatiblityName) + { + MultivaluedMap queryParameters = uriInfo.getQueryParameters(); + // Returning the first value, according to the @QueryParam spec: + // https://docs.oracle.com/javaee/7/api/javax/ws/rs/QueryParam.html: + // "If the type is not one of the collection types listed in 5 above and the query parameter is represented by + // multiple values then the first value (lexically) of the parameter is used." + String paramValue = queryParameters.getFirst(name); + if (paramValue != null) { + return paramValue; + } + String compatibilityParamValue = queryParameters.getFirst(compatiblityName); + if (compatibilityParamValue != null) { + log.warn( + "Parameter %s in %s query has been renamed to %s. Use the new parameter name.", + compatiblityName, + uriInfo.getPath(), + name + ); + return compatibilityParamValue; + } + // Not found neither name nor compatiblityName + return null; + } } 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 14bf3395add3..9ba84de16606 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 @@ -57,7 +57,7 @@ public class CoordinatorDynamicConfig private final int balancerComputeThreads; private final boolean emitBalancingStats; private final boolean killAllDataSources; - private final Set killableDataSources; + private final Set specificDataSourcesToKill; private final Set decommissioningNodes; private final int decommissioningMaxPercentOfMaxSegmentsToMove; @@ -86,7 +86,7 @@ public CoordinatorDynamicConfig( // Type is Object here so that we can support both string and list as // coordinator console can not send array of strings in the update request. // See https://github.com/apache/incubator-druid/issues/3055 - @JsonProperty("killDataSourceWhitelist") Object killableDataSources, + @JsonProperty("killDataSourceWhitelist") Object specificDataSourcesToKill, @JsonProperty("killAllDataSources") boolean killAllDataSources, @JsonProperty("killPendingSegmentsSkipList") Object protectedPendingSegmentDatasources, @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue, @@ -103,7 +103,7 @@ public CoordinatorDynamicConfig( this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; this.killAllDataSources = killAllDataSources; - this.killableDataSources = parseJsonStringOrArray(killableDataSources); + this.specificDataSourcesToKill = parseJsonStringOrArray(specificDataSourcesToKill); this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes); @@ -113,8 +113,8 @@ public CoordinatorDynamicConfig( ); this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; - if (this.killAllDataSources && !this.killableDataSources.isEmpty()) { - throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist"); + if (this.killAllDataSources && !this.specificDataSourcesToKill.isEmpty()) { + throw new IAE("can't have killAllDataSources and non-empty specificDataSourcesToKill"); } } @@ -201,13 +201,13 @@ public int getBalancerComputeThreads() } /** - * List of dataSources for which kill tasks are sent in + * List of specific data sources for which kill tasks are sent in * {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. */ @JsonProperty("killDataSourceWhitelist") - public Set getKillableDataSources() + public Set getSpecificDataSourcesToKill() { - return killableDataSources; + return specificDataSourcesToKill; } @JsonProperty @@ -279,7 +279,7 @@ public String toString() ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + ", killAllDataSources=" + killAllDataSources + - ", killDataSourceWhitelist=" + killableDataSources + + ", killDataSourceWhitelist=" + specificDataSourcesToKill + ", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + ", decommissioningNodes=" + decommissioningNodes + @@ -329,7 +329,7 @@ public boolean equals(Object o) if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { return false; } - if (!Objects.equals(killableDataSources, that.killableDataSources)) { + if (!Objects.equals(specificDataSourcesToKill, that.specificDataSourcesToKill)) { return false; } if (!Objects.equals(protectedPendingSegmentDatasources, that.protectedPendingSegmentDatasources)) { @@ -355,7 +355,7 @@ public int hashCode() emitBalancingStats, killAllDataSources, maxSegmentsInNodeLoadingQueue, - killableDataSources, + specificDataSourcesToKill, protectedPendingSegmentDatasources, decommissioningNodes, decommissioningMaxPercentOfMaxSegmentsToMove @@ -389,7 +389,7 @@ public static class Builder private Integer replicationThrottleLimit; private Boolean emitBalancingStats; private Integer balancerComputeThreads; - private Object killableDataSources; + private Object specificDataSourcesToKill; private Boolean killAllDataSources; private Object killPendingSegmentsSkipList; private Integer maxSegmentsInNodeLoadingQueue; @@ -410,7 +410,7 @@ public Builder( @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, - @JsonProperty("killDataSourceWhitelist") @Nullable Object killableDataSources, + @JsonProperty("killDataSourceWhitelist") @Nullable Object specificDataSourcesToKill, @JsonProperty("killAllDataSources") @Nullable Boolean killAllDataSources, @JsonProperty("killPendingSegmentsSkipList") @Nullable Object killPendingSegmentsSkipList, @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue, @@ -427,7 +427,7 @@ public Builder( this.balancerComputeThreads = balancerComputeThreads; this.emitBalancingStats = emitBalancingStats; this.killAllDataSources = killAllDataSources; - this.killableDataSources = killableDataSources; + this.specificDataSourcesToKill = specificDataSourcesToKill; this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = decommissioningNodes; @@ -484,7 +484,7 @@ public Builder withEmitBalancingStats(boolean emitBalancingStats) public Builder withKillDataSourceWhitelist(Set killDataSourceWhitelist) { - this.killableDataSources = killDataSourceWhitelist; + this.specificDataSourcesToKill = killDataSourceWhitelist; return this; } @@ -523,7 +523,7 @@ public CoordinatorDynamicConfig build() replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit, balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, - killableDataSources, + specificDataSourcesToKill, killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, killPendingSegmentsSkipList, maxSegmentsInNodeLoadingQueue == null @@ -547,7 +547,7 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit, balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, - killableDataSources == null ? defaults.getKillableDataSources() : killableDataSources, + specificDataSourcesToKill == null ? defaults.getSpecificDataSourcesToKill() : specificDataSourcesToKill, killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, killPendingSegmentsSkipList == null ? defaults.getProtectedPendingSegmentDatasources() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java index d2d3029b17b3..6f2373b3f8e9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java @@ -117,7 +117,7 @@ public static double intervalCost(double x1, double y0, double y1) // since x_0 <= y_0, Y must overlap X if y_0 < x_1 if (y0 < x1) { - /** + /* * We have two possible cases of overlap: * * X = [ A )[ B )[ C ) or [ A )[ B ) @@ -151,7 +151,7 @@ public static double intervalCost(double x1, double y0, double y1) intervalCost(beta, beta, gamma) + // cost(B, C) 2 * (beta + FastMath.exp(-beta) - 1); // cost(B, B) } else { - /** + /* * In the case where there is no overlap: * * Given that x_0 <= y_0, @@ -258,9 +258,14 @@ public double calculateInitialTotalCost(final List serverHolders) { double cost = 0; for (ServerHolder server : serverHolders) { - Iterable segments = server.getServer().getLazyAllSegments(); - for (DataSegment s : segments) { - cost += computeJointSegmentsCost(s, segments); + // segments are dumped into an array because it's probably better than iterating the iterateAllSegments() result + // quadratically in a loop, which can generate garbage in the form of Stream, Spliterator, Iterator, etc. objects + // whose total memory volume exceeds the size of the DataSegment array. + DataSegment[] segments = server.getServer().iterateAllSegments().toArray(new DataSegment[0]); + for (DataSegment s1 : segments) { + for (DataSegment s2 : segments) { + cost += computeJointSegmentsCost(s1, s2); + } } } return cost; @@ -280,7 +285,7 @@ public double calculateNormalization(final List serverHolders) { double cost = 0; for (ServerHolder server : serverHolders) { - for (DataSegment segment : server.getServer().getLazyAllSegments()) { + for (DataSegment segment : server.getServer().iterateAllSegments()) { cost += computeJointSegmentsCost(segment, segment); } } @@ -288,10 +293,7 @@ public double calculateNormalization(final List serverHolders) } @Override - public void emitStats( - String tier, - CoordinatorStats stats, List serverHolderList - ) + public void emitStats(String tier, CoordinatorStats stats, List serverHolderList) { final double initialTotalCost = calculateInitialTotalCost(serverHolderList); final double normalization = calculateNormalization(serverHolderList); @@ -334,7 +336,7 @@ protected double computeCost( // the sum of the costs of other (exclusive of the proposalSegment) segments on the server cost += computeJointSegmentsCost( proposalSegment, - Iterables.filter(server.getServer().getLazyAllSegments(), segment -> !proposalSegment.equals(segment)) + Iterables.filter(server.getServer().iterateAllSegments(), segment -> !proposalSegment.equals(segment)) ); // plus the costs of segments that will be loaded diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java index dff28710a006..8e559b4a2741 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java @@ -47,8 +47,8 @@ protected double computeCost( } int nSegments = 1; - if (server.getServer().getLazyAllSegments().size() > 0) { - nSegments = server.getServer().getLazyAllSegments().size(); + if (server.getServer().getNumSegments() > 0) { + nSegments = server.getServer().getNumSegments(); } double normalizedCost = cost / nSegments; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 3b5faef7a04b..97d2af0376ff 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -25,6 +25,9 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Inject; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import it.unimi.dsi.fastutil.objects.Object2IntMaps; +import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.curator.framework.CuratorFramework; @@ -55,16 +58,16 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupOvershadowed; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnneeded; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnusedSegments; import org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper; import org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger; import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentCompactor; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentInfoLoader; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorUsedSegmentsLoader; import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -98,7 +101,7 @@ public class DruidCoordinator * It is used in historical nodes' {@link LoadQueuePeon}s to make historicals load more recent segment first. * * It is also used in {@link DruidCoordinatorRuntimeParams} for {@link - * DruidCoordinatorRuntimeParams#getAvailableSegments()} - a collection of segments to be considered during some + * DruidCoordinatorRuntimeParams#getUsedSegments()} - a collection of segments to be considered during some * coordinator run for different {@link DruidCoordinatorHelper}s. The order matters only for {@link * DruidCoordinatorRuleRunner}, which tries to apply the rules while iterating the segments in the order imposed by * this comparator. In {@link LoadRule} the throttling limit may be hit (via {@link ReplicationThrottler}; see @@ -122,7 +125,7 @@ public class DruidCoordinator private final DruidCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; - private final MetadataSegmentManager metadataSegmentManager; + private final MetadataSegments metadataSegments; private final ServerInventoryView serverInventoryView; private final MetadataRuleManager metadataRuleManager; private final CuratorFramework curator; @@ -148,7 +151,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegmentManager metadataSegmentManager, + MetadataSegments metadataSegments, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -168,7 +171,7 @@ public DruidCoordinator( config, zkPaths, configManager, - metadataSegmentManager, + metadataSegments, serverInventoryView, metadataRuleManager, curator, @@ -190,7 +193,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegmentManager metadataSegmentManager, + MetadataSegments metadataSegments, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -211,7 +214,7 @@ public DruidCoordinator( this.zkPaths = zkPaths; this.configManager = configManager; - this.metadataSegmentManager = metadataSegmentManager; + this.metadataSegments = metadataSegments; this.serverInventoryView = serverInventoryView; this.metadataRuleManager = metadataRuleManager; this.curator = curator; @@ -253,7 +256,7 @@ public Map> computeUnderReplicationCountsPerDataS final DateTime now = DateTimes.nowUtc(); - for (final DataSegment segment : iterateAvailableDataSegments()) { + for (final DataSegment segment : iterateAllUsedSegments()) { final List rules = metadataRuleManager.getRulesWithDefault(segment.getDataSource()); for (final Rule rule : rules) { @@ -277,31 +280,31 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - public Object2LongMap getSegmentAvailability() + public Object2IntMap computeNumsUnavailableUsedSegmentsPerDataSource() { - final Object2LongOpenHashMap retVal = new Object2LongOpenHashMap<>(); - if (segmentReplicantLookup == null) { - return retVal; + return Object2IntMaps.emptyMap(); } - for (DataSegment segment : iterateAvailableDataSegments()) { + final Object2IntOpenHashMap numsUnavailableUsedSegmentsPerDataSource = new Object2IntOpenHashMap<>(); + + for (DataSegment segment : iterateAllUsedSegments()) { if (segmentReplicantLookup.getLoadedReplicants(segment.getId()) == 0) { - retVal.addTo(segment.getDataSource(), 1); + numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 1); } else { - retVal.addTo(segment.getDataSource(), 0); + numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 0); } } - return retVal; + return numsUnavailableUsedSegmentsPerDataSource; } public Map getLoadStatus() { Map loadStatus = new HashMap<>(); - for (ImmutableDruidDataSource dataSource : metadataSegmentManager.getDataSources()) { + for (ImmutableDruidDataSource dataSource : metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) { final Set segments = Sets.newHashSet(dataSource.getSegments()); - final int availableSegmentSize = segments.size(); + final int numUsedSegments = segments.size(); // remove loaded segments for (DruidServer druidServer : serverInventoryView.getInventory()) { @@ -314,10 +317,10 @@ public Map getLoadStatus() } } } - final int unloadedSegmentSize = segments.size(); + final int numUnloadedSegments = segments.size(); loadStatus.put( dataSource.getName(), - 100 * ((double) (availableSegmentSize - unloadedSegmentSize) / (double) availableSegmentSize) + 100 * ((double) (numUsedSegments - numUnloadedSegments) / (double) numUsedSegments) ); } @@ -339,10 +342,10 @@ public CoordinatorCompactionConfig getCompactionConfig() return CoordinatorCompactionConfig.current(configManager); } - public void removeSegment(DataSegment segment) + public void tryMarkSegmentAsUnused(DataSegment segment) { log.info("Removing Segment[%s]", segment.getId()); - metadataSegmentManager.removeSegment(segment.getId()); + metadataSegments.tryMarkSegmentAsUnused(segment.getId()); } public String getCurrentLeader() @@ -370,13 +373,13 @@ public void moveSegment( throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); } - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(segment.getDataSource()); + ImmutableDruidDataSource dataSource = metadataSegments.prepareImmutableDataSourceWithUsedSegments(segment.getDataSource()); if (dataSource == null) { throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); } - // get segment information from MetadataSegmentManager instead of getting it from fromServer's. - // This is useful when MetadataSegmentManager and fromServer DataSegment's are different for same + // get segment information from MetadataSegments instead of getting it from fromServer's. + // This is useful when MetadataSegments and fromServer DataSegment's are different for same // identifier (say loadSpec differs because of deep storage migration). final DataSegment segmentToLoad = dataSource.getSegment(segment.getId()); if (segmentToLoad == null) { @@ -449,14 +452,16 @@ public void moveSegment( } /** - * Returns an iterable to go over all available segments in all data sources. The order in which segments are iterated - * is unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try - * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than - * several times. + * Returns an iterable to go over all used segments in all data sources. The order in which segments are iterated + * is unspecified. + * + * Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try (to some + * reasonable extent) to organize the code so that it iterates the returned iterable only once rather than several + * times. */ - public Iterable iterateAvailableDataSegments() + public Iterable iterateAllUsedSegments() { - return metadataSegmentManager.iterateAllSegments(); + return metadataSegments.iterateAllUsedSegments(); } @LifecycleStart @@ -513,7 +518,7 @@ private void becomeLeader() log.info("I am the leader of the coordinators, all must bow!"); log.info("Starting coordination in [%s]", config.getCoordinatorStartDelay()); - metadataSegmentManager.start(); + metadataSegments.start(); metadataRuleManager.start(); lookupCoordinatorManager.start(); serviceAnnouncer.announce(self); @@ -581,14 +586,14 @@ private void stopBeingLeader() serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); metadataRuleManager.stop(); - metadataSegmentManager.stop(); + metadataSegments.stop(); } } private List makeIndexingServiceHelpers() { List helpers = new ArrayList<>(); - helpers.add(new DruidCoordinatorSegmentInfoLoader(DruidCoordinator.this)); + helpers.add(new DruidCoordinatorUsedSegmentsLoader(DruidCoordinator.this)); helpers.add(segmentCompactor); helpers.addAll(indexingServiceHelpers); @@ -625,7 +630,7 @@ public void run() } List allStarted = Arrays.asList( - metadataSegmentManager.isStarted(), + metadataSegments.isStarted(), serverInventoryView.isStarted() ); for (Boolean aBoolean : allStarted) { @@ -644,14 +649,15 @@ public void run() // Do coordinator stuff. DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withStartTime(startTime) - .withDataSources(metadataSegmentManager.getDataSources()) - .withDynamicConfigs(getDynamicConfigs()) - .withCompactionConfig(getCompactionConfig()) - .withEmitter(emitter) - .withBalancerStrategy(balancerStrategy) - .build(); + DruidCoordinatorRuntimeParams + .newBuilder() + .withStartTime(startTime) + .withDataSourcesWithUsedSegments(metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) + .withDynamicConfigs(getDynamicConfigs()) + .withCompactionConfig(getCompactionConfig()) + .withEmitter(emitter) + .withBalancerStrategy(balancerStrategy) + .build(); for (DruidCoordinatorHelper helper : helpers) { // Don't read state and run state in the same helper otherwise racy conditions may exist if (coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { @@ -676,7 +682,7 @@ public CoordinatorHistoricalManagerRunnable(final int startingLeaderCounter) { super( ImmutableList.of( - new DruidCoordinatorSegmentInfoLoader(DruidCoordinator.this), + new DruidCoordinatorUsedSegmentsLoader(DruidCoordinator.this), params -> { List servers = serverInventoryView .getInventory() @@ -740,7 +746,7 @@ public CoordinatorHistoricalManagerRunnable(final int startingLeaderCounter) .build(); }, new DruidCoordinatorRuleRunner(DruidCoordinator.this), - new DruidCoordinatorCleanupUnneeded(), + new DruidCoordinatorCleanupUnusedSegments(), new DruidCoordinatorCleanupOvershadowed(DruidCoordinator.this), new DruidCoordinatorBalancer(DruidCoordinator.this), new DruidCoordinatorLogger(DruidCoordinator.this) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java index a25ea07c523f..34ed7485b133 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java @@ -88,7 +88,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // If there is no running/pending/waiting/complete tasks, pendingSegmentsCleanupEndTime is // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); - for (String dataSource : params.getDataSources().keySet()) { + for (String dataSource : params.getDataSourcesWithUsedSegments().keySet()) { if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) { log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index 655d6bdf22ff..e9f91ed212dd 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -41,14 +41,17 @@ public class DruidCoordinatorRuntimeParams { /** + * Creates a set to be provided to {@link Builder#setUsedSegments(TreeSet)} method from the given {@link + * Iterable} of segments. + * * Creates a TreeSet sorted in {@link DruidCoordinator#SEGMENT_COMPARATOR_RECENT_FIRST} order and populates it with * the segments from the given iterable. The given iterable is iterated exactly once. No special action is taken if * duplicate segments are encountered in the iterable. */ - public static TreeSet createAvailableSegmentsSet(Iterable availableSegments) + public static TreeSet createUsedSegmentsSet(Iterable usedSegments) { TreeSet segmentsSet = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); - availableSegments.forEach(segmentsSet::add); + usedSegments.forEach(segmentsSet::add); return segmentsSet; } @@ -56,8 +59,9 @@ public static TreeSet createAvailableSegmentsSet(Iterable> dataSources; - private final @Nullable TreeSet availableSegments; + /** dataSource -> VersionedIntervalTimeline[version String, DataSegment] */ + private final Map> dataSourcesWithUsedSegments; + private final @Nullable TreeSet usedSegments; private final Map loadManagementPeons; private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; @@ -72,8 +76,8 @@ private DruidCoordinatorRuntimeParams( DruidCluster druidCluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Map> dataSources, - @Nullable TreeSet availableSegments, + Map> dataSourcesWithUsedSegments, + @Nullable TreeSet usedSegments, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -88,8 +92,8 @@ private DruidCoordinatorRuntimeParams( this.druidCluster = druidCluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSources = dataSources; - this.availableSegments = availableSegments; + this.dataSourcesWithUsedSegments = dataSourcesWithUsedSegments; + this.usedSegments = usedSegments; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -120,15 +124,18 @@ public SegmentReplicantLookup getSegmentReplicantLookup() return segmentReplicantLookup; } - public Map> getDataSources() + /** + * Returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used" segments. + */ + public Map> getDataSourcesWithUsedSegments() { - return dataSources; + return dataSourcesWithUsedSegments; } - public TreeSet getAvailableSegments() + public TreeSet getUsedSegments() { - Preconditions.checkState(availableSegments != null, "availableSegments must be set"); - return availableSegments; + Preconditions.checkState(usedSegments != null, "usedSegments must be set"); + return usedSegments; } public Map getLoadManagementPeons() @@ -188,8 +195,8 @@ public Builder buildFromExisting() druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - availableSegments, + dataSourcesWithUsedSegments, + usedSegments, loadManagementPeons, replicationManager, emitter, @@ -201,15 +208,15 @@ public Builder buildFromExisting() ); } - public Builder buildFromExistingWithoutAvailableSegments() + public Builder buildFromExistingWithoutUsedSegments() { return new Builder( startTime, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - null, // availableSegments + dataSourcesWithUsedSegments, + null, // usedSegments loadManagementPeons, replicationManager, emitter, @@ -227,8 +234,8 @@ public static class Builder private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; private SegmentReplicantLookup segmentReplicantLookup; - private Map> dataSources; - private @Nullable TreeSet availableSegments; + private Map> dataSourcesWithUsedSegments; + private @Nullable TreeSet usedSegments; private final Map loadManagementPeons; private ReplicationThrottler replicationManager; private ServiceEmitter emitter; @@ -244,8 +251,8 @@ public static class Builder this.druidCluster = null; this.databaseRuleManager = null; this.segmentReplicantLookup = null; - this.dataSources = new HashMap<>(); - this.availableSegments = null; + this.dataSourcesWithUsedSegments = new HashMap<>(); + this.usedSegments = null; this.loadManagementPeons = new HashMap<>(); this.replicationManager = null; this.emitter = null; @@ -260,8 +267,8 @@ public static class Builder DruidCluster cluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Map> dataSources, - @Nullable TreeSet availableSegments, + Map> dataSourcesWithUsedSegments, + @Nullable TreeSet usedSegments, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -276,8 +283,8 @@ public static class Builder this.druidCluster = cluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSources = dataSources; - this.availableSegments = availableSegments; + this.dataSourcesWithUsedSegments = dataSourcesWithUsedSegments; + this.usedSegments = usedSegments; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -295,8 +302,8 @@ public DruidCoordinatorRuntimeParams build() druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - availableSegments, + dataSourcesWithUsedSegments, + usedSegments, loadManagementPeons, replicationManager, emitter, @@ -332,26 +339,29 @@ public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup) return this; } - public Builder withDataSources(Map> dataSources) + @VisibleForTesting + public Builder setDataSourcesWithUsedSegments( + Map> dataSources + ) { - this.dataSources = dataSources; + this.dataSourcesWithUsedSegments = dataSources; return this; } - public Builder withDataSources(Collection dataSourcesCollection) + Builder withDataSourcesWithUsedSegments(Collection dataSourcesWithUsedSegments) { - dataSourcesCollection.forEach( + dataSourcesWithUsedSegments.forEach( dataSource -> { - VersionedIntervalTimeline timeline = dataSources.computeIfAbsent( + VersionedIntervalTimeline timeline = this.dataSourcesWithUsedSegments.computeIfAbsent( dataSource.getName(), k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER) ); dataSource.getSegments().forEach( - segment -> timeline.add( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().createChunk(segment) + usedSegment -> timeline.add( + usedSegment.getInterval(), + usedSegment.getVersion(), + usedSegment.getShardSpec().createChunk(usedSegment) ) ); } @@ -361,29 +371,29 @@ public Builder withDataSources(Collection dataSourcesC /** This method must be used in test code only. */ @VisibleForTesting - public Builder withAvailableSegmentsInTest(DataSegment... availableSegments) + public Builder withUsedSegmentsInTest(DataSegment... usedSegments) { - return withAvailableSegmentsInTest(Arrays.asList(availableSegments)); + return withUsedSegmentsInTest(Arrays.asList(usedSegments)); } /** This method must be used in test code only. */ @VisibleForTesting - public Builder withAvailableSegmentsInTest(Collection availableSegments) + public Builder withUsedSegmentsInTest(Collection usedSegments) { - return setAvailableSegments(createAvailableSegmentsSet(availableSegments)); + return setUsedSegments(createUsedSegmentsSet(usedSegments)); } /** - * Note: unlike {@link #withAvailableSegmentsInTest(Collection)}, this method doesn't make a defensive copy of the + * Note: unlike {@link #withUsedSegmentsInTest(Collection)}, this method doesn't make a defensive copy of the * provided set. The set passed into this method must not be modified afterwards. */ - public Builder setAvailableSegments(TreeSet availableSegments) + public Builder setUsedSegments(TreeSet usedSegments) { //noinspection ObjectEquality - if (availableSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) { + if (usedSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) { throw new IllegalArgumentException("Expected DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST"); } - this.availableSegments = availableSegments; + this.usedSegments = usedSegments; return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java index e770ef78210c..c2c4a7ad10c4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java @@ -34,7 +34,7 @@ static BalancerSegmentHolder getRandomBalancerSegmentHolder(final List balanceServers( log.info("All servers to move segments from are empty, ending run."); break; } - // DruidCoordinatorRuntimeParams.getAvailableSegments originate from MetadataSegmentManager, i. e. that's a - // "desired" or "theoretical" set of segments. segmentToMoveHolder.getSegment originates from ServerInventoryView, - // i. e. that may be any segment that happens to be loaded on some server, even if it "shouldn't" from the - // "theoretical" point of view (Coordinator closes such discrepancies eventually via - // DruidCoordinatorCleanupUnneeded). Therefore the picked segmentToMoveHolder's segment may not need to be - // balanced. - boolean needToBalancePickedSegment = params.getAvailableSegments().contains(segmentToMoveHolder.getSegment()); + // DruidCoordinatorRuntimeParams.getUsedSegments originate from MetadataSegments, i. e. that's a set of segments + // that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be + // any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such + // discrepancies eventually via DruidCoordinatorCleanupUnusedSegments). Therefore the picked segmentToMoveHolder's + // segment may not need to be balanced. + boolean needToBalancePickedSegment = params.getUsedSegments().contains(segmentToMoveHolder.getSegment()); if (needToBalancePickedSegment) { final DataSegment segmentToMove = segmentToMoveHolder.getSegment(); final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java index 8131e4f62be9..72aad9ea9dc6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java @@ -71,10 +71,10 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } //Remove all segments in db that are overshadowed by served segments - for (DataSegment dataSegment : params.getAvailableSegments()) { + for (DataSegment dataSegment : params.getUsedSegments()) { VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { - coordinator.removeSegment(dataSegment); + coordinator.tryMarkSegmentAsUnused(dataSegment); stats.addToGlobalStat("overShadowedCount", 1); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnusedSegments.java similarity index 72% rename from server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java rename to server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnusedSegments.java index a7a1bcce2205..2086ec17f84a 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnusedSegments.java @@ -34,39 +34,36 @@ /** */ -public class DruidCoordinatorCleanupUnneeded implements DruidCoordinatorHelper +public class DruidCoordinatorCleanupUnusedSegments implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidCoordinatorCleanupUnneeded.class); + private static final Logger log = new Logger(DruidCoordinatorCleanupUnusedSegments.class); @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { CoordinatorStats stats = new CoordinatorStats(); - Set availableSegments = params.getAvailableSegments(); + Set usedSegments = params.getUsedSegments(); DruidCluster cluster = params.getDruidCluster(); - if (availableSegments.isEmpty()) { + if (usedSegments.isEmpty()) { log.info( - "Found 0 availableSegments, skipping the cleanup of segments from historicals. This is done to prevent " + + "Found 0 used segments, skipping the cleanup of segments from historicals. This is done to prevent " + "a race condition in which the coordinator would drop all segments if it started running cleanup before " + - "it finished polling the metadata storage for available segments for the first time." + "it finished polling the metadata storage for used segments for the first time." ); return params.buildFromExisting().withCoordinatorStats(stats).build(); } - // Drop segments that no longer exist in the available segments configuration, *if* it has been populated. (It might - // not have been loaded yet since it's filled asynchronously. But it's also filled atomically, so if there are any - // segments at all, we should have all of them.) - // Note that if metadata store has no segments, then availableSegments will stay empty and nothing will be dropped. - // This is done to prevent a race condition in which the coordinator would drop all segments if it started running - // cleanup before it finished polling the metadata storage for available segments for the first time. + // Drop segments that are no longer marked as used, *if* the usedSegments collection has been populated. Used + // segments might not have been loaded yet since it's done asynchronously (in SqlMetadataSegments). But it's also + // done atomically (see SqlMetadataSegments code), so if there are any segments at all, we should have all of them. for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { ImmutableDruidServer server = serverHolder.getServer(); for (ImmutableDruidDataSource dataSource : server.getDataSources()) { for (DataSegment segment : dataSource.getSegments()) { - if (!availableSegments.contains(segment)) { + if (!usedSegments.contains(segment)) { LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName()); if (!queuePeon.getSegmentsToDrop().contains(segment)) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java index a271109f6a7c..1a167534a93f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -19,6 +19,7 @@ package org.apache.druid.server.coordinator.helper; +import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.logger.Logger; @@ -32,12 +33,7 @@ import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.PartitionChunk; - -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; +import org.apache.druid.timeline.VersionedIntervalTimeline; /** */ @@ -214,14 +210,14 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) ); }); - coordinator.getSegmentAvailability().object2LongEntrySet().forEach( - (final Object2LongMap.Entry entry) -> { + coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().object2IntEntrySet().forEach( + (final Object2IntMap.Entry entry) -> { final String dataSource = entry.getKey(); - final long count = entry.getLongValue(); + final int numUnavailableUsedSegmentsInDataSource = entry.getIntValue(); emitter.emit( new ServiceMetricEvent.Builder() .setDimension(DruidMetrics.DATASOURCE, dataSource).build( - "segment/unavailable/count", count + "segment/unavailable/count", numUnavailableUsedSegmentsInDataSource ) ); } @@ -263,28 +259,25 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) ); // Emit segment metrics - final Stream allSegments = params - .getDataSources() - .values() - .stream() - .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) - .flatMap(entryMap -> entryMap.values().stream()) - .flatMap(entry -> StreamSupport.stream(entry.getPartitionHolder().spliterator(), false)) - .map(PartitionChunk::getObject); - - allSegments - .collect(Collectors.groupingBy(DataSegment::getDataSource)) - .forEach((final String name, final List segments) -> { - final long size = segments.stream().mapToLong(DataSegment::getSize).sum(); + params.getDataSourcesWithUsedSegments().forEach( + (String dataSource, VersionedIntervalTimeline dataSourceWithUsedSegments) -> { + long totalSizeOfUsedSegments = dataSourceWithUsedSegments + .iterateAllObjects() + .stream() + .mapToLong(DataSegment::getSize) + .sum(); emitter.emit( - new ServiceMetricEvent.Builder().setDimension(DruidMetrics.DATASOURCE, name).build("segment/size", size) + new ServiceMetricEvent.Builder() + .setDimension(DruidMetrics.DATASOURCE, dataSource) + .build("segment/size", totalSizeOfUsedSegments) ); emitter.emit( new ServiceMetricEvent.Builder() - .setDimension(DruidMetrics.DATASOURCE, name) - .build("segment/count", segments.size()) + .setDimension(DruidMetrics.DATASOURCE, dataSource) + .build("segment/count", dataSourceWithUsedSegments.getNumObjects()) ); - }); + } + ); return params; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index 7d7830170c99..b0212dddc04f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -86,9 +86,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params; } - // find available segments which are not overshadowed by other segments in DB - // only those would need to be loaded/dropped - // anything overshadowed by served segments is dropped automatically by DruidCoordinatorCleanupOvershadowed + // Find used segments which are overshadowed by other used segments. Those would not need to be loaded and dropped. + // Segments overshadowed by *served* used segments is dropped automatically by DruidCoordinatorCleanupOvershadowed. Set overshadowed = determineOvershadowedSegments(params); for (String tier : cluster.getTierNames()) { @@ -96,17 +95,17 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } DruidCoordinatorRuntimeParams paramsWithReplicationManager = params - .buildFromExistingWithoutAvailableSegments() + .buildFromExistingWithoutUsedSegments() .withReplicationManager(replicatorThrottler) .build(); - // Run through all matched rules for available segments + // Run through all matched rules for used segments DateTime now = DateTimes.nowUtc(); MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager(); final List segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); int missingRules = 0; - for (DataSegment segment : params.getAvailableSegments()) { + for (DataSegment segment : params.getUsedSegments()) { if (overshadowed.contains(segment)) { // Skipping overshadowed segments continue; @@ -142,7 +141,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) private Set determineOvershadowedSegments(DruidCoordinatorRuntimeParams params) { Map> timelines = new HashMap<>(); - for (DataSegment segment : params.getAvailableSegments()) { + for (DataSegment segment : params.getUsedSegments()) { timelines .computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural())) .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java index 7cbffa540d39..b1ac299d2162 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -77,7 +77,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final CoordinatorStats stats = new CoordinatorStats(); if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { - Map> dataSources = params.getDataSources(); + Map> dataSources = params.getDataSourcesWithUsedSegments(); List compactionConfigList = dynamicConfig.getCompactionConfigs(); if (compactionConfigList != null && !compactionConfigList.isEmpty()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java index 3f719d3f3d40..33d2f0030fab 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java @@ -26,7 +26,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.joda.time.Interval; @@ -46,12 +46,12 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper private long lastKillTime = 0; - private final MetadataSegmentManager segmentManager; + private final MetadataSegments metadataSegments; private final IndexingServiceClient indexingServiceClient; @Inject public DruidCoordinatorSegmentKiller( - MetadataSegmentManager segmentManager, + MetadataSegments metadataSegments, IndexingServiceClient indexingServiceClient, DruidCoordinatorConfig config ) @@ -75,7 +75,7 @@ public DruidCoordinatorSegmentKiller( this.maxSegmentsToKill ); - this.segmentManager = segmentManager; + this.metadataSegments = metadataSegments; this.indexingServiceClient = indexingServiceClient; } @@ -83,21 +83,26 @@ public DruidCoordinatorSegmentKiller( public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources(); - Collection whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources(); + Collection specificDataSourcesToKill = params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKill(); - if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) { - log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled."); + if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) { + log.error( + "killAllDataSources can't be true when specificDataSourcesToKill is non-empty. No kill tasks are scheduled." + ); return params; } + Collection dataSourcesToKill = specificDataSourcesToKill; if (killAllDataSources) { - whitelist = segmentManager.getAllDataSourceNames(); + dataSourcesToKill = metadataSegments.retrieveAllDataSourceNames(); } - if (whitelist != null && whitelist.size() > 0 && (lastKillTime + period) < System.currentTimeMillis()) { + if (dataSourcesToKill != null && + dataSourcesToKill.size() > 0 && + (lastKillTime + period) < System.currentTimeMillis()) { lastKillTime = System.currentTimeMillis(); - for (String dataSource : whitelist) { + for (String dataSource : dataSourcesToKill) { final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill); if (intervalToKill != null) { try { @@ -119,7 +124,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) @VisibleForTesting Interval findIntervalForKillTask(String dataSource, int limit) { - List unusedSegmentIntervals = segmentManager.getUnusedSegmentIntervals( + List unusedSegmentIntervals = metadataSegments.getUnusedSegmentIntervals( dataSource, new Interval(DateTimes.EPOCH, DateTimes.nowUtc().minus(retainDuration)), limit diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUsedSegmentsLoader.java similarity index 68% rename from server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java rename to server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUsedSegmentsLoader.java index 801bd3ba5021..960eba7867d9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUsedSegmentsLoader.java @@ -27,13 +27,13 @@ import java.util.TreeSet; -public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper +public class DruidCoordinatorUsedSegmentsLoader implements DruidCoordinatorHelper { - private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class); + private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorUsedSegmentsLoader.class); private final DruidCoordinator coordinator; - public DruidCoordinatorSegmentInfoLoader(DruidCoordinator coordinator) + public DruidCoordinatorUsedSegmentsLoader(DruidCoordinator coordinator) { this.coordinator = coordinator; } @@ -41,20 +41,20 @@ public DruidCoordinatorSegmentInfoLoader(DruidCoordinator coordinator) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - log.info("Starting coordination. Getting available segments."); + log.info("Starting coordination. Getting used segments."); // The following transform() call doesn't actually transform the iterable. It only checks the sizes of the segments // and emits alerts if segments with negative sizes are encountered. In other words, semantically it's similar to - // Stream.peek(). It works as long as DruidCoordinatorRuntimeParams.createAvailableSegmentsSet() (which is called + // Stream.peek(). It works as long as DruidCoordinatorRuntimeParams.createUsedSegmentsSet() (which is called // below) guarantees to go over the passed iterable exactly once. // - // An iterable returned from iterateAvailableDataSegments() is not simply iterated (with size checks) before passing - // into DruidCoordinatorRuntimeParams.createAvailableSegmentsSet() because iterateAvailableDataSegments()'s + // An iterable returned from iterateAllUsedSegments() is not simply iterated (with size checks) before passing + // into DruidCoordinatorRuntimeParams.createUsedSegmentsSet() because iterateAllUsedSegments()'s // documentation says to strive to avoid iterating the result more than once. // //noinspection StaticPseudoFunctionalStyleMethod: https://youtrack.jetbrains.com/issue/IDEA-153047 - Iterable availableSegmentsWithSizeChecking = Iterables.transform( - coordinator.iterateAvailableDataSegments(), + Iterable usedSegmentsWithSizeChecking = Iterables.transform( + coordinator.iterateAllUsedSegments(), segment -> { if (segment.getSize() < 0) { log.makeAlert("No size on a segment") @@ -64,21 +64,21 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return segment; } ); - final TreeSet availableSegments = - DruidCoordinatorRuntimeParams.createAvailableSegmentsSet(availableSegmentsWithSizeChecking); + final TreeSet usedSegments = + DruidCoordinatorRuntimeParams.createUsedSegmentsSet(usedSegmentsWithSizeChecking); - // Log info about all available segments + // Log info about all used segments if (log.isDebugEnabled()) { - log.debug("Available DataSegments"); - for (DataSegment dataSegment : availableSegments) { + log.debug("Used Segments"); + for (DataSegment dataSegment : usedSegments) { log.debug(" %s", dataSegment); } } - log.info("Found [%,d] available segments.", availableSegments.size()); + log.info("Found [%,d] used segments.", usedSegments.size()); return params.buildFromExisting() - .setAvailableSegments(availableSegments) + .setUsedSegments(usedSegments) .build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java index fdd017802f52..f3620a2e5444 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java @@ -33,7 +33,7 @@ public abstract class DropRule implements Rule public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) { CoordinatorStats stats = new CoordinatorStats(); - coordinator.removeSegment(segment); + coordinator.tryMarkSegmentAsUnused(segment); stats.addToGlobalStat("deletedCount", 1); return stats; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java index 48aff5129229..14f791e6adb4 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java @@ -52,13 +52,13 @@ public interface Rule boolean appliesTo(Interval interval, DateTime referenceTimestamp); /** - * {@link DruidCoordinatorRuntimeParams#getAvailableSegments()} must not be called in Rule's code, because the - * available segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is - * because {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions: + * {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used + * segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because + * {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions: * "DruidCoordinatorHelperParams" and "RuleParams" which contain params that only {@link * org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper}s and Rules need, respectively. * For example, {@link org.apache.druid.server.coordinator.ReplicationThrottler} needs to belong only to "RuleParams", - * but not "DruidCoordinatorHelperParams". The opposite for "AvailableSegments". + * but not "DruidCoordinatorHelperParams". The opposite for the collection of used segments. * * See https://github.com/apache/incubator-druid/issues/7228 */ diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java index 779e909c4f60..9522ca87a697 100644 --- a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java +++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java @@ -91,7 +91,7 @@ public Response getLoadStatus( ) { if (simple != null) { - return Response.ok(coordinator.getSegmentAvailability()).build(); + return Response.ok(coordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).build(); } if (full != null) { diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index 05535053f83a..fc4096108e68 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -39,14 +39,13 @@ import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.http.security.DatasourceResourceFilter; -import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -93,27 +92,24 @@ public class DataSourcesResource private static final Logger log = new Logger(DataSourcesResource.class); private final CoordinatorServerView serverInventoryView; - private final MetadataSegmentManager databaseSegmentManager; - private final MetadataRuleManager databaseRuleManager; + private final MetadataSegments metadataSegments; + private final MetadataRuleManager metadataRuleManager; private final IndexingServiceClient indexingServiceClient; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; @Inject public DataSourcesResource( CoordinatorServerView serverInventoryView, - MetadataSegmentManager databaseSegmentManager, - MetadataRuleManager databaseRuleManager, + MetadataSegments metadataSegments, + MetadataRuleManager metadataRuleManager, @Nullable IndexingServiceClient indexingServiceClient, - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; - this.databaseSegmentManager = databaseSegmentManager; - this.databaseRuleManager = databaseRuleManager; + this.metadataSegments = metadataSegments; + this.metadataRuleManager = metadataRuleManager; this.indexingServiceClient = indexingServiceClient; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; } @@ -153,7 +149,7 @@ public Response getQueryableDataSources( @Path("/{dataSourceName}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getTheDataSource( + public Response getDataSource( @PathParam("dataSourceName") final String dataSourceName, @QueryParam("full") final String full ) @@ -175,29 +171,28 @@ public Response getTheDataSource( @Path("/{dataSourceName}") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDataSource( - @PathParam("dataSourceName") final String dataSourceName - ) + public Response markAsUsedAllSegments(@PathParam("dataSourceName") final String dataSourceName) { - if (!databaseSegmentManager.enableDataSource(dataSourceName)) { - return Response.noContent().build(); + if (!metadataSegments.tryMarkAsUsedAllSegmentsInDataSource(dataSourceName)) { + return Response.serverError().build(); } return Response.ok().build(); } - /* When this method is removed, a new method needs to be introduced corresponding to - the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters). - Ultimately we want to have no method with kill parameter - - DELETE `{dataSourceName}` will be used to disable datasource and - DELETE `{dataSourceName}/intervals/{interval}` will be used to nuke segments - */ + /** + * When this method is removed, a new method needs to be introduced corresponding to + * the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters). + * Ultimately we want to have no method with kill parameter - + * DELETE `{dataSourceName}` will be used to mark all segments belonging to a data source as unused, and + * DELETE `{dataSourceName}/intervals/{interval}` will be used to kill segments within an interval + */ @DELETE @Deprecated @Path("/{dataSourceName}") @ResourceFilters(DatasourceResourceFilter.class) @Produces(MediaType.APPLICATION_JSON) - public Response deleteDataSource( + public Response tryMarkAsUnusedAllSegmentsOrKillSegmentsInInterval( @PathParam("dataSourceName") final String dataSourceName, @QueryParam("kill") final String kill, @QueryParam("interval") final String interval @@ -207,7 +202,8 @@ public Response deleteDataSource( return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } - if (kill != null && Boolean.valueOf(kill)) { + boolean killSegments = kill != null && Boolean.valueOf(kill); + if (killSegments) { try { indexingServiceClient.killSegments(dataSourceName, Intervals.of(interval)); } @@ -235,8 +231,8 @@ public Response deleteDataSource( .build(); } } else { - if (!databaseSegmentManager.removeDataSource(dataSourceName)) { - return Response.noContent().build(); + if (!metadataSegments.tryMarkAsUnusedAllSegmentsInDataSource(dataSourceName)) { + return Response.serverError().build(); } } @@ -247,7 +243,7 @@ public Response deleteDataSource( @Path("/{dataSourceName}/intervals/{interval}") @ResourceFilters(DatasourceResourceFilter.class) @Produces(MediaType.APPLICATION_JSON) - public Response deleteDataSourceSpecificInterval( + public Response killSegmentsInInterval( @PathParam("dataSourceName") final String dataSourceName, @PathParam("interval") final String interval ) @@ -276,7 +272,7 @@ public Response deleteDataSourceSpecificInterval( @Path("/{dataSourceName}/intervals") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceIntervals( + public Response getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals( @PathParam("dataSourceName") String dataSourceName, @QueryParam("simple") String simple, @QueryParam("full") String full @@ -292,14 +288,14 @@ public Response getSegmentDataSourceIntervals( dataSource.getSegments().forEach(segment -> intervals.add(segment.getInterval())); return Response.ok(intervals).build(); } - return getSegmentDataSourceIntervals(dataSourceName, full != null, interval -> true); + return getServedSegmentsInInterval(dataSourceName, full != null, interval -> true); } @GET @Path("/{dataSourceName}/intervals/{interval}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSpecificInterval( + public Response getServedSegmentsInInterval( @PathParam("dataSourceName") String dataSourceName, @PathParam("interval") String interval, @QueryParam("simple") String simple, @@ -320,7 +316,7 @@ public Response getSegmentDataSourceSpecificInterval( } return Response.ok(segmentIds).build(); } - return getSegmentDataSourceIntervals(dataSourceName, full != null, theInterval::contains); + return getServedSegmentsInInterval(dataSourceName, full != null, theInterval::contains); } /** @@ -333,7 +329,7 @@ enum SimpleProperties count } - private Response getSegmentDataSourceIntervals( + private Response getServedSegmentsInInterval( String dataSourceName, boolean full, Predicate intervalFilter @@ -381,7 +377,7 @@ private Response getSegmentDataSourceIntervals( @Path("/{dataSourceName}/segments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSegments( + public Response getAllServedSegments( @PathParam("dataSourceName") String dataSourceName, @QueryParam("full") String full ) @@ -403,7 +399,7 @@ public Response getSegmentDataSourceSegments( @Path("/{dataSourceName}/segments/{segmentId}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSegment( + public Response getServedSegment( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) @@ -425,12 +421,12 @@ public Response getSegmentDataSourceSegment( @DELETE @Path("/{dataSourceName}/segments/{segmentId}") @ResourceFilters(DatasourceResourceFilter.class) - public Response deleteDatasourceSegment( + public Response removeSegment( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - if (databaseSegmentManager.removeSegment(dataSourceName, segmentId)) { + if (metadataSegments.tryMarkSegmentAsUnused(dataSourceName, segmentId)) { return Response.ok().build(); } return Response.noContent().build(); @@ -440,13 +436,13 @@ public Response deleteDatasourceSegment( @Path("/{dataSourceName}/segments/{segmentId}") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDatasourceSegment( + public Response markSegmentAsUsed( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - if (!databaseSegmentManager.enableSegment(segmentId)) { - return Response.noContent().build(); + if (!metadataSegments.tryMarkSegmentAsUsed(segmentId)) { + return Response.serverError().build(); } return Response.ok().build(); @@ -456,7 +452,7 @@ public Response enableDatasourceSegment( @Path("/{dataSourceName}/tiers") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceTiers(@PathParam("dataSourceName") String dataSourceName) + public Response getTiersWhereSegmentsAreServed(@PathParam("dataSourceName") String dataSourceName) { Set retVal = new HashSet<>(); for (DruidServer druidServer : serverInventoryView.getInventory()) { @@ -585,14 +581,14 @@ private Map> getSimpleDatasource(String dataSourceNa } /** - * Provides serverView for a datasource and Interval which gives details about servers hosting segments for an interval - * Used by the realtime tasks to fetch a view of the interval they are interested in. + * Provides serverView for a datasource and Interval which gives details about servers hosting segments for an + * interval. Used by the realtime tasks to fetch a view of the interval they are interested in. */ @GET @Path("/{dataSourceName}/intervals/{interval}/serverview") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getSegmentDataSourceSpecificInterval( + public Response getServedSegmentsInInterval( @PathParam("dataSourceName") String dataSourceName, @PathParam("interval") String interval, @QueryParam("partial") final boolean partial @@ -607,7 +603,8 @@ public Response getSegmentDataSourceSpecificInterval( return Response.ok(new ArrayList()).build(); } - Iterable> lookup = timeline.lookupWithIncompletePartitions(theInterval); + Iterable> lookup = + timeline.lookupWithIncompletePartitions(theInterval); FunctionalIterable retval = FunctionalIterable .create(lookup).transformCat( (TimelineObjectHolder input) -> @@ -636,7 +633,7 @@ public Response isHandOffComplete( ) { try { - final List rules = databaseRuleManager.getRulesWithDefault(dataSourceName); + final List rules = metadataRuleManager.getRulesWithDefault(dataSourceName); final Interval theInterval = Intervals.of(interval); final SegmentDescriptor descriptor = new SegmentDescriptor(theInterval, version, partitionNumber); final DateTime now = DateTimes.nowUtc(); diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index c7e270214ff4..a6460bfe419c 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -25,15 +25,15 @@ import com.google.common.base.Function; import com.google.common.collect.Collections2; import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.server.JettyUtils; import org.apache.druid.server.http.security.DatasourceResourceFilter; -import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.ResourceAction; @@ -41,6 +41,7 @@ import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.GET; import javax.ws.rs.POST; @@ -52,6 +53,7 @@ import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; import javax.ws.rs.core.StreamingOutput; +import javax.ws.rs.core.UriInfo; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -65,24 +67,21 @@ @Path("/druid/coordinator/v1/metadata") public class MetadataResource { - private final MetadataSegmentManager metadataSegmentManager; + private final MetadataSegments metadataSegments; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final AuthConfig authConfig; private final AuthorizerMapper authorizerMapper; private final ObjectMapper jsonMapper; @Inject public MetadataResource( - MetadataSegmentManager metadataSegmentManager, + MetadataSegments metadataSegments, IndexerMetadataStorageCoordinator metadataStorageCoordinator, - AuthConfig authConfig, AuthorizerMapper authorizerMapper, @Json ObjectMapper jsonMapper ) { - this.metadataSegmentManager = metadataSegmentManager; + this.metadataSegments = metadataSegments; this.metadataStorageCoordinator = metadataStorageCoordinator; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; this.jsonMapper = jsonMapper; } @@ -90,26 +89,28 @@ public MetadataResource( @GET @Path("/datasources") @Produces(MediaType.APPLICATION_JSON) - public Response getDatabaseDataSources( + public Response getDataSources( @QueryParam("full") final String full, - @QueryParam("includeDisabled") final String includeDisabled, + @Context final UriInfo uriInfo, @Context final HttpServletRequest req ) { - final Collection druidDataSources = metadataSegmentManager.getDataSources(); + final @Nullable String includeUnused = JettyUtils.getQueryParam(uriInfo, "includeUnused", "includeDisabled"); + Collection druidDataSources = null; final Set dataSourceNamesPreAuth; - if (includeDisabled != null) { - dataSourceNamesPreAuth = new TreeSet<>(metadataSegmentManager.getAllDataSourceNames()); + if (includeUnused != null) { + dataSourceNamesPreAuth = new TreeSet<>(metadataSegments.retrieveAllDataSourceNames()); } else { - dataSourceNamesPreAuth = Sets.newTreeSet( - Iterables.transform(druidDataSources, ImmutableDruidDataSource::getName) - ); + druidDataSources = metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments(); + dataSourceNamesPreAuth = druidDataSources + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toCollection(TreeSet::new)); } final Set dataSourceNamesPostAuth = new TreeSet<>(); - Function> raGenerator = datasourceName -> { - return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); - }; + Function> raGenerator = datasourceName -> + Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); Iterables.addAll( dataSourceNamesPostAuth, @@ -121,9 +122,9 @@ public Response getDatabaseDataSources( ) ); - // Cannot do both includeDisabled and full, let includeDisabled take priority + // Cannot do both includeUnused and full, let includeUnused take priority // Always use dataSourceNamesPostAuth to determine the set of returned dataSources - if (full != null && includeDisabled == null) { + if (full != null && includeUnused == null) { return Response.ok().entity( Collections2.filter(druidDataSources, dataSource -> dataSourceNamesPostAuth.contains(dataSource.getName())) ).build(); @@ -132,35 +133,23 @@ public Response getDatabaseDataSources( } } - @GET - @Path("/datasources/{dataSourceName}") - @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSource(@PathParam("dataSourceName") final String dataSourceName) - { - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); - if (dataSource == null) { - return Response.status(Response.Status.NOT_FOUND).build(); - } - - return Response.status(Response.Status.OK).entity(dataSource).build(); - } - @GET @Path("/segments") @Produces(MediaType.APPLICATION_JSON) - public Response getDatabaseSegments( + public Response getAllUsedSegments( @Context final HttpServletRequest req, - @QueryParam("datasources") final Set datasources + @QueryParam("datasources") final Set dataSources ) { - Collection druidDataSources = metadataSegmentManager.getDataSources(); - if (datasources != null && !datasources.isEmpty()) { - druidDataSources = druidDataSources.stream() - .filter(src -> datasources.contains(src.getName())) - .collect(Collectors.toSet()); + Collection dataSourcesWithUsedSegments = + metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments(); + if (dataSources != null && !dataSources.isEmpty()) { + dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + .stream() + .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) + .collect(Collectors.toList()); } - final Stream metadataSegments = druidDataSources + final Stream usedSegments = dataSourcesWithUsedSegments .stream() .flatMap(t -> t.getSegments().stream()); @@ -168,7 +157,7 @@ public Response getDatabaseSegments( AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); final Iterable authorizedSegments = - AuthorizationUtils.filterAuthorizedResources(req, metadataSegments::iterator, raGenerator, authorizerMapper); + AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); final StreamingOutput stream = outputStream -> { final JsonFactory jsonFactory = jsonMapper.getFactory(); @@ -186,16 +175,36 @@ public Response getDatabaseSegments( return builder.entity(stream).build(); } + /** + * The difference of this method from {@link #getUsedSegmentsInDataSource} is that the latter returns only a list of + * segments, while this method also includes the properties of data source, such as the time when it was created. + */ + @GET + @Path("/datasources/{dataSourceName}") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response getDataSourceWithUsedSegments(@PathParam("dataSourceName") final String dataSourceName) + { + ImmutableDruidDataSource dataSource = + metadataSegments.prepareImmutableDataSourceWithUsedSegments(dataSourceName); + if (dataSource == null) { + return Response.status(Response.Status.NOT_FOUND).build(); + } + + return Response.status(Response.Status.OK).entity(dataSource).build(); + } + @GET @Path("/datasources/{dataSourceName}/segments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSourceSegments( + public Response getUsedSegmentsInDataSource( @PathParam("dataSourceName") String dataSourceName, @QueryParam("full") String full ) { - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); + ImmutableDruidDataSource dataSource = + metadataSegments.prepareImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -208,11 +217,15 @@ public Response getDatabaseSegmentDataSourceSegments( return builder.entity(Collections2.transform(dataSource.getSegments(), DataSegment::getId)).build(); } + /** + * This is a {@link POST} method to pass the list of intervals in the body, + * see https://github.com/apache/incubator-druid/pull/2109#issuecomment-182191258 + */ @POST @Path("/datasources/{dataSourceName}/segments") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSourceSegments( + public Response getUsedSegmentsInDataSourceForIntervals( @PathParam("dataSourceName") String dataSourceName, @QueryParam("full") String full, List intervals @@ -232,12 +245,12 @@ public Response getDatabaseSegmentDataSourceSegments( @Path("/datasources/{dataSourceName}/segments/{segmentId}") @Produces(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response getDatabaseSegmentDataSourceSegment( + public Response isSegmentUsed( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); + DruidDataSource dataSource = metadataSegments.getDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SqlMetadataSegmentsTest.java similarity index 88% rename from server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java rename to server/src/test/java/org/apache/druid/metadata/SqlMetadataSegmentsTest.java index a9f8f3c5df00..716338e77aa6 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlMetadataSegmentsTest.java @@ -41,12 +41,12 @@ import java.io.IOException; -public class SQLMetadataSegmentManagerTest +public class SqlMetadataSegmentsTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private SQLMetadataSegmentManager manager; + private SqlMetadataSegments manager; private SQLMetadataSegmentPublisher publisher; private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @@ -86,9 +86,9 @@ public class SQLMetadataSegmentManagerTest public void setUp() throws Exception { TestDerbyConnector connector = derbyConnectorRule.getConnector(); - manager = new SQLMetadataSegmentManager( + manager = new SqlMetadataSegments( jsonMapper, - Suppliers.ofInstance(new MetadataSegmentManagerConfig()), + Suppliers.ofInstance(new MetadataSegmentsConfig()), derbyConnectorRule.metadataTablesConfigSupplier(), connector ); @@ -121,11 +121,11 @@ public void testPoll() Assert.assertTrue(manager.isStarted()); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + manager.retrieveAllDataSourceNames() ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.getDataSource("wikipedia").getSegments()) + ImmutableSet.copyOf(manager.prepareImmutableDataSourceWithUsedSegments("wikipedia").getSegments()) ); } @@ -153,7 +153,7 @@ public void testPollWithCurroptedSegment() Assert.assertTrue(manager.isStarted()); Assert.assertEquals( - "wikipedia", Iterables.getOnlyElement(manager.getDataSources()).getName() + "wikipedia", Iterables.getOnlyElement(manager.prepareImmutableDataSourcesWithAllUsedSegments()).getName() ); } @@ -163,7 +163,7 @@ public void testGetUnusedSegmentsForInterval() manager.start(); manager.poll(); Assert.assertTrue(manager.isStarted()); - Assert.assertTrue(manager.removeDataSource("wikipedia")); + Assert.assertTrue(manager.tryMarkAsUnusedAllSegmentsInDataSource("wikipedia")); Assert.assertEquals( ImmutableList.of(segment2.getInterval()), @@ -202,8 +202,8 @@ public void testRemoveDataSource() throws IOException publisher.publishSegment(newSegment); - Assert.assertNull(manager.getDataSource(newDataSource)); - Assert.assertTrue(manager.removeDataSource(newDataSource)); + Assert.assertNull(manager.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + Assert.assertTrue(manager.tryMarkAsUnusedAllSegmentsInDataSource(newDataSource)); } @Test @@ -232,8 +232,8 @@ public void testRemoveDataSegment() throws IOException publisher.publishSegment(newSegment); - Assert.assertNull(manager.getDataSource(newDataSource)); - Assert.assertTrue(manager.removeSegment(newSegment.getId())); + Assert.assertNull(manager.prepareImmutableDataSourceWithUsedSegments(newDataSource)); + Assert.assertTrue(manager.tryMarkSegmentAsUnused(newSegment.getId())); } @Test diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java index e6aa17ebd1ca..4259689d4d3c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java @@ -94,7 +94,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme segments.put(segment.getId(), segment); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); } - EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments.values()).anyTimes(); + EasyMock.expect(druidServer.iterateAllSegments()).andReturn(segments.values()).anyTimes(); EasyMock.replay(druidServer); serverHolderList.add(new ServerHolder(druidServer, fromPeon)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index 25dce8cf577f..d96a0f00cc58 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -45,7 +45,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -82,7 +82,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase { private DruidCoordinator coordinator; - private MetadataSegmentManager databaseSegmentManager; + private MetadataSegments metadataSegments; private ScheduledExecutorFactory scheduledExecutorFactory; private ConcurrentMap loadManagementPeons; private LoadQueuePeon sourceLoadQueuePeon; @@ -124,7 +124,7 @@ public CuratorDruidCoordinatorTest() @Before public void setUp() throws Exception { - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); + metadataSegments = EasyMock.createNiceMock(MetadataSegments.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -211,7 +211,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + metadataSegments, baseView, metadataRuleManager, curator, @@ -364,8 +364,8 @@ public void testMoveSegment() throws Exception ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(sourceSegments.get(2)); EasyMock.replay(druidDataSource); - EasyMock.expect(databaseSegmentManager.getDataSource(EasyMock.anyString())).andReturn(druidDataSource); - EasyMock.replay(databaseSegmentManager); + EasyMock.expect(metadataSegments.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock.replay(metadataSegments); coordinator.moveSegment( source.toImmutableDruidServer(), @@ -494,7 +494,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + metadataSegments, baseView, metadataRuleManager, curator, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java index 0bbe46d9b85b..6d61a4f3a85a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java @@ -88,7 +88,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme segments.add(segment); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); } - EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(druidServer.iterateAllSegments()).andReturn(segments).anyTimes(); EasyMock.replay(druidServer); serverHolderList.add(new ServerHolder(druidServer, fromPeon)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index ed4f600c2cd6..de33a5256dd4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -116,9 +116,9 @@ public void bigProfiler() EasyMock.expect(server.getName()).andReturn(Integer.toString(i)).atLeastOnce(); EasyMock.expect(server.getHost()).andReturn(Integer.toString(i)).anyTimes(); if (i == 0) { - EasyMock.expect(server.getLazyAllSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(server.iterateAllSegments()).andReturn(segments).anyTimes(); } else { - EasyMock.expect(server.getLazyAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + EasyMock.expect(server.iterateAllSegments()).andReturn(Collections.emptyList()).anyTimes(); } EasyMock.expect(server.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(server); @@ -148,7 +148,7 @@ public void bigProfiler() .withLoadManagementPeons( peonMap ) - .withAvailableSegmentsInTest(segments) + .withUsedSegmentsInTest(segments) .withDynamicConfigs( CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE @@ -197,7 +197,7 @@ public void profileRun() EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.getLazyAllSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(druidServer1.iterateAllSegments()).andReturn(segments).anyTimes(); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -205,7 +205,7 @@ public void profileRun() EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer2.getLazyAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + EasyMock.expect(druidServer2.iterateAllSegments()).andReturn(Collections.emptyList()).anyTimes(); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); @@ -246,7 +246,7 @@ public void profileRun() toPeon ) ) - .withAvailableSegmentsInTest(segments) + .withUsedSegmentsInTest(segments) .withDynamicConfigs( CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java index 0a3089808c6c..9ede2d8e176f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -535,7 +535,7 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( .boxed() .collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get)) ) - .withAvailableSegmentsInTest(segments) + .withUsedSegmentsInTest(segments) .withDynamicConfigs( CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( MAX_SEGMENTS_TO_MOVE @@ -558,7 +558,7 @@ private static void mockDruidServer( EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes(); EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce(); EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce(); - EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(druidServer.iterateAllSegments()).andReturn(segments).anyTimes(); EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); if (!segments.isEmpty()) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 51a79307c752..725671c37338 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -63,7 +63,7 @@ public class DruidCoordinatorRuleRunnerTest { private DruidCoordinator coordinator; private LoadQueuePeon mockPeon; - private List availableSegments; + private List usedSegments; private DruidCoordinatorRuleRunner ruleRunner; private ServiceEmitter emitter; private MetadataRuleManager databaseRuleManager; @@ -78,9 +78,9 @@ public void setUp() databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); DateTime start = DateTimes.of("2012-01-01"); - availableSegments = new ArrayList<>(); + usedSegments = new ArrayList<>(); for (int i = 0; i < 24; i++) { - availableSegments.add( + usedSegments.add( new DataSegment( "test", new Interval(start, start.plusHours(1)), @@ -198,7 +198,7 @@ public void testRunThreeTiersOneReplicant() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) @@ -304,7 +304,7 @@ public void testRunTwoTiersTwoReplicants() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) @@ -361,8 +361,8 @@ public void testRunTwoTiersWithExistingSegments() "normal", 0 ); - for (DataSegment availableSegment : availableSegments) { - normServer.addDataSegment(availableSegment); + for (DataSegment segment : usedSegments) { + normServer.addDataSegment(segment); } DruidCluster druidCluster = new DruidCluster( @@ -403,7 +403,7 @@ public void testRunTwoTiersWithExistingSegments() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -478,7 +478,7 @@ public void testRunTwoTiersTierDoesNotExist() new DruidCoordinatorRuntimeParams.Builder() .withEmitter(emitter) .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) @@ -540,7 +540,7 @@ public void testRunRuleDoesNotExist() new DruidCoordinatorRuntimeParams.Builder() .withEmitter(emitter) .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .build(); @@ -558,7 +558,7 @@ public void testDropRemove() mockEmptyPeon(); EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.tryMarkSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); EasyMock.replay(coordinator); @@ -582,7 +582,7 @@ public void testDropRemove() "normal", 0 ); - for (DataSegment segment : availableSegments) { + for (DataSegment segment : usedSegments) { server.addDataSegment(segment); } @@ -609,7 +609,7 @@ public void testDropRemove() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -653,7 +653,7 @@ public void testDropTooManyInSameTier() "normal", 0 ); - server1.addDataSegment(availableSegments.get(0)); + server1.addDataSegment(usedSegments.get(0)); DruidServer server2 = new DruidServer( "serverNorm2", @@ -664,7 +664,7 @@ public void testDropTooManyInSameTier() "normal", 0 ); - for (DataSegment segment : availableSegments) { + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } @@ -695,7 +695,7 @@ public void testDropTooManyInSameTier() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -742,7 +742,7 @@ public void testDropTooManyInDifferentTiers() "hot", 0 ); - server1.addDataSegment(availableSegments.get(0)); + server1.addDataSegment(usedSegments.get(0)); DruidServer server2 = new DruidServer( "serverNorm2", "hostNorm2", @@ -752,7 +752,7 @@ public void testDropTooManyInDifferentTiers() "normal", 0 ); - for (DataSegment segment : availableSegments) { + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } @@ -786,7 +786,7 @@ public void testDropTooManyInDifferentTiers() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -840,7 +840,7 @@ public void testDontDropInDifferentTiers() "normal", 0 ); - for (DataSegment segment : availableSegments) { + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } DruidCluster druidCluster = new DruidCluster( @@ -865,7 +865,7 @@ public void testDontDropInDifferentTiers() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -908,7 +908,7 @@ public void testDropServerActuallyServesSegment() "normal", 0 ); - server1.addDataSegment(availableSegments.get(0)); + server1.addDataSegment(usedSegments.get(0)); DruidServer server2 = new DruidServer( "serverNorm2", "hostNorm2", @@ -918,7 +918,7 @@ public void testDropServerActuallyServesSegment() "normal", 0 ); - server2.addDataSegment(availableSegments.get(1)); + server2.addDataSegment(usedSegments.get(1)); DruidServer server3 = new DruidServer( "serverNorm3", "hostNorm3", @@ -928,8 +928,8 @@ public void testDropServerActuallyServesSegment() "normal", 0 ); - server3.addDataSegment(availableSegments.get(1)); - server3.addDataSegment(availableSegments.get(2)); + server3.addDataSegment(usedSegments.get(1)); + server3.addDataSegment(usedSegments.get(2)); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); @@ -963,7 +963,7 @@ public void testDropServerActuallyServesSegment() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -1047,7 +1047,7 @@ public void testReplicantThrottle() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) @@ -1077,7 +1077,7 @@ public void testReplicantThrottle() new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withEmitter(emitter) - .withAvailableSegmentsInTest(Collections.singletonList(overFlowSegment)) + .withUsedSegmentsInTest(Collections.singletonList(overFlowSegment)) .withDatabaseRuleManager(databaseRuleManager) .withBalancerStrategy(balancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) @@ -1114,7 +1114,7 @@ public void testReplicantThrottleAcrossTiers() .build() ) .atLeastOnce(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.tryMarkSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); @@ -1180,7 +1180,7 @@ public void testReplicantThrottleAcrossTiers() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withBalancerStrategy(balancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) @@ -1231,8 +1231,8 @@ public void testDropReplicantThrottle() 1, 0 ); - List longerAvailableSegments = Lists.newArrayList(availableSegments); - longerAvailableSegments.add(overFlowSegment); + List longerUsedSegments = Lists.newArrayList(usedSegments); + longerUsedSegments.add(overFlowSegment); DruidServer server1 = new DruidServer( "serverNorm1", @@ -1243,8 +1243,8 @@ public void testDropReplicantThrottle() "normal", 0 ); - for (DataSegment availableSegment : longerAvailableSegments) { - server1.addDataSegment(availableSegment); + for (DataSegment segment : longerUsedSegments) { + server1.addDataSegment(segment); } DruidServer server2 = new DruidServer( "serverNorm2", @@ -1255,8 +1255,8 @@ public void testDropReplicantThrottle() "normal", 0 ); - for (DataSegment availableSegment : longerAvailableSegments) { - server2.addDataSegment(availableSegment); + for (DataSegment segment : longerUsedSegments) { + server2.addDataSegment(segment); } DruidCluster druidCluster = new DruidCluster( @@ -1286,7 +1286,7 @@ public void testDropReplicantThrottle() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(longerAvailableSegments) + .withUsedSegmentsInTest(longerUsedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) @@ -1305,7 +1305,7 @@ public void testDropReplicantThrottle() @Test public void testRulesRunOnNonOvershadowedSegmentsOnly() { - Set availableSegments = new HashSet<>(); + Set usedSegments = new HashSet<>(); DataSegment v1 = new DataSegment( "test", Intervals.of("2012-01-01/2012-01-02"), @@ -1328,8 +1328,8 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() IndexIO.CURRENT_VERSION_ID, 1 ); - availableSegments.add(v1); - availableSegments.add(v2); + usedSegments.add(v1); + usedSegments.add(v2); mockCoordinator(); mockPeon.loadSegment(EasyMock.eq(v2), EasyMock.anyObject()); @@ -1369,7 +1369,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withBalancerStrategy(balancerStrategy) @@ -1385,9 +1385,9 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty()); Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty()); - Assert.assertEquals(2, availableSegments.size()); - Assert.assertEquals(availableSegments, params.getAvailableSegments()); - Assert.assertEquals(availableSegments, afterParams.getAvailableSegments()); + Assert.assertEquals(2, usedSegments.size()); + Assert.assertEquals(usedSegments, params.getUsedSegments()); + Assert.assertEquals(usedSegments, afterParams.getUsedSegments()); EasyMock.verify(mockPeon); exec.shutdown(); @@ -1396,7 +1396,7 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() private void mockCoordinator() { EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.tryMarkSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); } 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 1f4f7e2844b2..9e86e77210ed 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 @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import it.unimi.dsi.fastutil.objects.Object2IntMap; import it.unimi.dsi.fastutil.objects.Object2LongMap; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; @@ -46,7 +47,7 @@ import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -85,7 +86,7 @@ public class DruidCoordinatorTest extends CuratorTestBase private static final long COORDINATOR_PERIOD = 100; private DruidCoordinator coordinator; - private MetadataSegmentManager databaseSegmentManager; + private MetadataSegments metadataSegments; private SingleServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; @@ -105,7 +106,7 @@ public void setUp() throws Exception { druidServer = EasyMock.createMock(DruidServer.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); + metadataSegments = EasyMock.createNiceMock(MetadataSegments.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -181,7 +182,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + metadataSegments, serverInventoryView, metadataRuleManager, curator, @@ -247,8 +248,8 @@ public void testMoveSegment() ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); EasyMock.replay(druidDataSource); - EasyMock.expect(databaseSegmentManager.getDataSource(EasyMock.anyString())).andReturn(druidDataSource); - EasyMock.replay(databaseSegmentManager); + EasyMock.expect(metadataSegments.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock.replay(metadataSegments); scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); EasyMock.replay(scheduledExecutorFactory); EasyMock.replay(metadataRuleManager); @@ -319,7 +320,7 @@ public void testCoordinatorRun() throws Exception EasyMock.replay(metadataRuleManager); - // Setup MetadataSegmentManager + // Setup MetadataSegments DruidDataSource[] dataSources = { new DruidDataSource(dataSource, Collections.emptyMap()) }; @@ -336,7 +337,7 @@ public void testCoordinatorRun() throws Exception ); dataSources[0].addSegment(dataSegment); - setupMetadataSegmentManagerMock(dataSources[0]); + setupMetadataSegmentsMock(dataSources[0]); ImmutableDruidDataSource immutableDruidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(immutableDruidDataSource.getSegments()) .andReturn(ImmutableSet.of(dataSegment)).atLeastOnce(); @@ -390,9 +391,10 @@ public void childEvent(CuratorFramework curatorFramework, PathChildrenCacheEvent Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getId().toString())); - Map segmentAvailability = coordinator.getSegmentAvailability(); - Assert.assertEquals(1, segmentAvailability.size()); - Assert.assertEquals(0L, segmentAvailability.get(dataSource)); + Object2IntMap numsUnavailableUsedSegmentsPerDataSource = + coordinator.computeNumsUnavailableUsedSegmentsPerDataSource(); + Assert.assertEquals(1, numsUnavailableUsedSegmentsPerDataSource.size()); + Assert.assertEquals(0, numsUnavailableUsedSegmentsPerDataSource.getInt(dataSource)); Map> underReplicationCountsPerDataSourcePerTier = coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); @@ -461,7 +463,7 @@ public void testCoordinatorTieredRun() throws Exception DruidDataSource[] druidDataSources = {new DruidDataSource(dataSource, Collections.emptyMap())}; dataSegments.values().forEach(druidDataSources[0]::addSegment); - setupMetadataSegmentManagerMock(druidDataSources[0]); + setupMetadataSegmentsMock(druidDataSources[0]); EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) .andReturn(ImmutableList.of(hotTier, coldTier)).atLeastOnce(); @@ -526,26 +528,26 @@ public void testCoordinatorTieredRun() throws Exception leaderUnannouncerLatch.await(); EasyMock.verify(serverInventoryView); - EasyMock.verify(databaseSegmentManager); + EasyMock.verify(metadataSegments); EasyMock.verify(metadataRuleManager); } - private void setupMetadataSegmentManagerMock(DruidDataSource dataSource) + private void setupMetadataSegmentsMock(DruidDataSource dataSource) { - EasyMock.expect(databaseSegmentManager.isStarted()).andReturn(true).anyTimes(); + EasyMock.expect(metadataSegments.isStarted()).andReturn(true).anyTimes(); EasyMock - .expect(databaseSegmentManager.iterateAllSegments()) + .expect(metadataSegments.iterateAllUsedSegments()) .andReturn(dataSource.getSegments()) .anyTimes(); EasyMock - .expect(databaseSegmentManager.getDataSources()) + .expect(metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) .anyTimes(); EasyMock - .expect(databaseSegmentManager.getAllDataSourceNames()) + .expect(metadataSegments.retrieveAllDataSourceNames()) .andReturn(Collections.singleton(dataSource.getName())) .anyTimes(); - EasyMock.replay(databaseSegmentManager); + EasyMock.replay(metadataSegments); } @Nullable diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java index 001dc2af24c7..7d90000a4972 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java @@ -138,7 +138,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.getLazyAllSegments()).andReturn(segments1).anyTimes(); + EasyMock.expect(druidServer1.iterateAllSegments()).andReturn(segments1).anyTimes(); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -146,7 +146,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer2.getLazyAllSegments()).andReturn(segments2).anyTimes(); + EasyMock.expect(druidServer2.iterateAllSegments()).andReturn(segments2).anyTimes(); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); @@ -154,7 +154,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer3.getLazyAllSegments()).andReturn(segments3).anyTimes(); + EasyMock.expect(druidServer3.iterateAllSegments()).andReturn(segments3).anyTimes(); EasyMock.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); @@ -162,7 +162,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer4.getLazyAllSegments()).andReturn(segments4).anyTimes(); + EasyMock.expect(druidServer4.iterateAllSegments()).andReturn(segments4).anyTimes(); EasyMock.expect(druidServer4.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer4); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java index b8699fb08666..13942b010eb4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java @@ -115,7 +115,7 @@ private CachingCostBalancerStrategy createCachingCostBalancerStrategy( { ClusterCostCache.Builder builder = ClusterCostCache.builder(); serverHolders.forEach( - s -> s.getServer().getLazyAllSegments().forEach(segment -> builder.addSegment(s.getServer().getName(), segment)) + s -> s.getServer().iterateAllSegments().forEach(segment -> builder.addSegment(s.getServer().getName(), segment)) ); return new CachingCostBalancerStrategy(builder.build(), listeningExecutorService); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java index 0ae0ee6c07d8..4a98e6bf88f6 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java @@ -48,7 +48,7 @@ public class DruidCoordinatorCleanupOvershadowedTest { DruidCoordinatorCleanupOvershadowed druidCoordinatorCleanupOvershadowed; DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class); - private List availableSegments; + private List usedSegments; DateTime start = DateTimes.of("2012-01-01"); DruidCluster druidCluster; private LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); @@ -71,7 +71,7 @@ public class DruidCoordinatorCleanupOvershadowedTest public void testRun() { druidCoordinatorCleanupOvershadowed = new DruidCoordinatorCleanupOvershadowed(coordinator); - availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); + usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); // Dummy values for comparisons in TreeSet EasyMock.expect(mockPeon.getLoadQueueSize()) @@ -103,8 +103,8 @@ public void testRun() .andReturn(ImmutableSet.of(segmentV1, segmentV2)) .anyTimes(); EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); - coordinator.removeSegment(segmentV1); - coordinator.removeSegment(segmentV0); + coordinator.tryMarkSegmentAsUnused(segmentV1); + coordinator.tryMarkSegmentAsUnused(segmentV0); EasyMock.expectLastCall(); EasyMock.replay(mockPeon, coordinator, druidServer, druidDataSource); @@ -119,7 +119,7 @@ public void testRun() DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder() - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withCoordinatorStats(new CoordinatorStats()) .withDruidCluster(druidCluster) .build(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index 52b78e2ff584..32deb2420d19 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -338,7 +338,7 @@ private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor { DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder() - .withDataSources(dataSources) + .setDataSourcesWithUsedSegments(dataSources) .withCompactionConfig(CoordinatorCompactionConfig.from(createCompactionConfigs(keepSegmentGranularity))) .build(); return compactor.run(params).getCoordinatorStats(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java index 8593e301b972..8d04eba96fb5 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.easymock.EasyMock; import org.joda.time.Duration; @@ -84,23 +84,21 @@ public void testFindIntervalForKillTask() ); } - private void testFindIntervalForKillTask(List segmentManagerResult, Interval expected) + private void testFindIntervalForKillTask(List metadataSegmentsResult, Interval expected) { - MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegments metadataSegments = EasyMock.createMock(MetadataSegments.class); EasyMock.expect( - segmentManager.getUnusedSegmentIntervals( + metadataSegments.getUnusedSegmentIntervals( EasyMock.anyString(), EasyMock.anyObject(Interval.class), EasyMock.anyInt() ) - ).andReturn( - segmentManagerResult - ); - EasyMock.replay(segmentManager); + ).andReturn(metadataSegmentsResult); + EasyMock.replay(metadataSegments); IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class); DruidCoordinatorSegmentKiller coordinatorSegmentKiller = new DruidCoordinatorSegmentKiller( - segmentManager, + metadataSegments, indexingServiceClient, new TestDruidCoordinatorConfig( null, 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 adb344792722..174d4608af5e 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 @@ -286,7 +286,7 @@ public void testBroadcastToSingleDataSource() .withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( + .withUsedSegmentsInTest( smallSegment, largeSegments.get(0), largeSegments.get(1), @@ -337,7 +337,7 @@ public void testBroadcastDecommissioning() .withDruidCluster(secondCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(secondCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( + .withUsedSegmentsInTest( smallSegment, largeSegments.get(0), largeSegments.get(1) @@ -366,7 +366,7 @@ public void testBroadcastToMultipleDataSources() .withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( + .withUsedSegmentsInTest( smallSegment, largeSegments.get(0), largeSegments.get(1), @@ -404,7 +404,7 @@ public void testBroadcastToAllServers() .withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( + .withUsedSegmentsInTest( smallSegment, largeSegments.get(0), largeSegments.get(1), 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 73d47ccc38bf..da5e01208d0a 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 @@ -182,7 +182,7 @@ public void testLoad() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -253,7 +253,7 @@ public void testLoadPrimaryAssignDoesNotOverAssign() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -303,7 +303,7 @@ public void testLoadPrimaryAssignDoesNotOverAssign() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -393,7 +393,7 @@ public void testLoadPriority() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -482,7 +482,7 @@ public void testDrop() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -541,7 +541,7 @@ public void testLoadWithNonExistentTier() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -614,7 +614,7 @@ public void testDropWithNonExistentTier() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -671,7 +671,7 @@ public void testMaxLoadingQueueSize() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) + .withUsedSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) .build(); @@ -728,7 +728,7 @@ public void testLoadDecommissioning() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -785,7 +785,7 @@ public void testLoadReplicaDuringDecommissioning() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + .withUsedSegmentsInTest(segment).build(), segment ); @@ -838,7 +838,7 @@ public void testDropDuringDecommissioning() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment1, segment2) + .withUsedSegmentsInTest(segment1, segment2) .build(); CoordinatorStats stats = rule.run( null, @@ -904,7 +904,7 @@ public void testRedundantReplicaDropDuringDecommissioning() .withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy) .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment1) + .withUsedSegmentsInTest(segment1) .build(); CoordinatorStats stats = rule.run( null, 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 af97906f9cc7..647039539ba5 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 @@ -254,7 +254,7 @@ private void assertConfig( int expectedReplicationThrottleLimit, int expectedBalancerComputeThreads, boolean expectedEmitingBalancingStats, - Set expectedKillableDatasources, + Set expectedSpecificDataSourcesToKill, boolean expectedKillAllDataSources, int expectedMaxSegmentsInNodeLoadingQueue, Set decommissioning, @@ -269,7 +269,7 @@ private void assertConfig( Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit()); Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads()); Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats()); - Assert.assertEquals(expectedKillableDatasources, config.getKillableDataSources()); + Assert.assertEquals(expectedSpecificDataSourcesToKill, config.getSpecificDataSourcesToKill()); Assert.assertEquals(expectedKillAllDataSources, config.isKillAllDataSources()); Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue()); Assert.assertEquals(decommissioning, config.getDecommissioningNodes()); diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index 116c20bb86e0..086ba605ef6c 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -174,7 +174,6 @@ public void testGetFullQueryableDataSources() null, null, null, - new AuthConfig(), AuthTestUtils.TEST_AUTHORIZER_MAPPER ); Response response = DataSourcesResource.getQueryableDataSources("full", null, request); @@ -259,7 +258,6 @@ public Access authorize(AuthenticationResult authenticationResult1, Resource res null, null, null, - new AuthConfig(), authMapper ); Response response = DataSourcesResource.getQueryableDataSources("full", null, request); @@ -306,7 +304,6 @@ public void testGetSimpleQueryableDataSources() null, null, null, - new AuthConfig(), AuthTestUtils.TEST_AUTHORIZER_MAPPER ); Response response = DataSourcesResource.getQueryableDataSources(null, "simple", request); @@ -332,8 +329,8 @@ public void testFullGetTheDataSource() EasyMock.replay(inventoryView, server); DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = DataSourcesResource.getTheDataSource("datasource1", "full"); + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = DataSourcesResource.getDataSource("datasource1", "full"); ImmutableDruidDataSource result = (ImmutableDruidDataSource) response.getEntity(); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(dataSource1.toImmutableDruidDataSource(), result); @@ -348,8 +345,8 @@ public void testNullGetTheDataSource() EasyMock.replay(inventoryView, server); DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Assert.assertEquals(204, DataSourcesResource.getTheDataSource("none", null).getStatus()); + new DataSourcesResource(inventoryView, null, null, null, null); + Assert.assertEquals(204, DataSourcesResource.getDataSource("none", null).getStatus()); EasyMock.verify(inventoryView, server); } @@ -366,8 +363,8 @@ public void testSimpleGetTheDataSource() EasyMock.replay(inventoryView, server); DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = DataSourcesResource.getTheDataSource("datasource1", null); + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = DataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); Assert.assertEquals(1, ((Map) (result.get("tiers").get(null))).get("segmentCount")); @@ -406,8 +403,8 @@ public void testSimpleGetTheDataSourceManyTiers() EasyMock.replay(inventoryView, server, server2, server3); DataSourcesResource DataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = DataSourcesResource.getTheDataSource("datasource1", null); + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = DataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result = (Map>) response.getEntity(); Assert.assertEquals(2, ((Map) (result.get("tiers").get("cold"))).get("segmentCount")); @@ -438,18 +435,18 @@ public void testGetSegmentDataSourceIntervals() expectedIntervals.add(Intervals.of("2010-01-22T00:00:00.000Z/2010-01-23T00:00:00.000Z")); expectedIntervals.add(Intervals.of("2010-01-01T00:00:00.000Z/2010-01-02T00:00:00.000Z")); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); + new DataSourcesResource(inventoryView, null, null, null, null); - Response response = dataSourcesResource.getSegmentDataSourceIntervals("invalidDataSource", null, null); + Response response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals("invalidDataSource", null, null); Assert.assertEquals(response.getEntity(), null); - response = dataSourcesResource.getSegmentDataSourceIntervals("datasource1", null, null); + response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals("datasource1", null, null); TreeSet actualIntervals = (TreeSet) response.getEntity(); Assert.assertEquals(2, actualIntervals.size()); Assert.assertEquals(expectedIntervals.get(0), actualIntervals.first()); Assert.assertEquals(expectedIntervals.get(1), actualIntervals.last()); - response = dataSourcesResource.getSegmentDataSourceIntervals("datasource1", "simple", null); + response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals("datasource1", "simple", null); TreeMap> results = (TreeMap) response.getEntity(); Assert.assertEquals(2, results.size()); Assert.assertEquals(expectedIntervals.get(0), results.firstKey()); @@ -457,7 +454,7 @@ public void testGetSegmentDataSourceIntervals() Assert.assertEquals(1, results.firstEntry().getValue().get(DataSourcesResource.SimpleProperties.count)); Assert.assertEquals(1, results.lastEntry().getValue().get(DataSourcesResource.SimpleProperties.count)); - response = dataSourcesResource.getSegmentDataSourceIntervals("datasource1", null, "full"); + response = dataSourcesResource.getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals("datasource1", null, "full"); Map> results2 = ((Map>) response.getEntity()); int i = 1; for (Map.Entry> entry : results2.entrySet()) { @@ -482,8 +479,8 @@ public void testGetSegmentDataSourceSpecificInterval() EasyMock.replay(inventoryView); DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + new DataSourcesResource(inventoryView, null, null, null, null); + Response response = dataSourcesResource.getServedSegmentsInInterval( "invalidDataSource", "2010-01-01/P1D", null, @@ -491,7 +488,7 @@ public void testGetSegmentDataSourceSpecificInterval() ); Assert.assertEquals(null, response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + response = dataSourcesResource.getServedSegmentsInInterval( "datasource1", "2010-03-01/P1D", null, @@ -499,16 +496,16 @@ public void testGetSegmentDataSourceSpecificInterval() ); // interval not present in the datasource Assert.assertEquals(ImmutableSet.of(), response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1D", null, null); + response = dataSourcesResource.getServedSegmentsInInterval("datasource1", "2010-01-01/P1D", null, null); Assert.assertEquals(ImmutableSet.of(dataSegmentList.get(0).getId()), response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1M", null, null); + response = dataSourcesResource.getServedSegmentsInInterval("datasource1", "2010-01-01/P1M", null, null); Assert.assertEquals( ImmutableSet.of(dataSegmentList.get(1).getId(), dataSegmentList.get(0).getId()), response.getEntity() ); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + response = dataSourcesResource.getServedSegmentsInInterval( "datasource1", "2010-01-01/P1M", "simple", @@ -526,7 +523,7 @@ public void testGetSegmentDataSourceSpecificInterval() ); } - response = dataSourcesResource.getSegmentDataSourceSpecificInterval("datasource1", "2010-01-01/P1M", null, "full"); + response = dataSourcesResource.getServedSegmentsInInterval("datasource1", "2010-01-01/P1M", null, "full"); Map> results1 = ((Map>) response.getEntity()); i = 1; for (Map.Entry> entry : results1.entrySet()) { @@ -556,10 +553,9 @@ public void testDeleteDataSourceSpecificInterval() null, null, indexingServiceClient, - new AuthConfig(), null ); - Response response = DataSourcesResource.deleteDataSourceSpecificInterval("datasource1", interval); + Response response = DataSourcesResource.killSegmentsInInterval("datasource1", interval); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(null, response.getEntity()); @@ -576,10 +572,9 @@ public void testDeleteDataSource() null, null, indexingServiceClient, - new AuthConfig(), null ); - Response response = DataSourcesResource.deleteDataSource("datasource", "true", "???"); + Response response = DataSourcesResource.tryMarkAsUnusedAllSegmentsOrKillSegmentsInInterval("datasource", "true", "???"); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); Assert.assertTrue(response.getEntity().toString().contains("java.lang.IllegalArgumentException")); @@ -598,7 +593,6 @@ public void testIsHandOffComplete() null, databaseRuleManager, null, - new AuthConfig(), null ); 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 4097e952096a..f49545c95bdd 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -52,9 +52,9 @@ import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManagerConfig; import org.apache.druid.metadata.MetadataRuleManagerProvider; -import org.apache.druid.metadata.MetadataSegmentManager; -import org.apache.druid.metadata.MetadataSegmentManagerConfig; -import org.apache.druid.metadata.MetadataSegmentManagerProvider; +import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.MetadataSegmentsConfig; +import org.apache.druid.metadata.MetadataSegmentsProvider; import org.apache.druid.metadata.MetadataStorage; import org.apache.druid.metadata.MetadataStorageProvider; import org.apache.druid.server.audit.AuditManagerProvider; @@ -145,7 +145,7 @@ public void configure(Binder binder) binder.bind(MetadataStorage.class) .toProvider(MetadataStorageProvider.class); - JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentManagerConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentsConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.lookups", LookupCoordinatorManagerConfig.class); JsonConfigProvider.bind(binder, "druid.coordinator.balancer", BalancerStrategyFactory.class); @@ -163,8 +163,8 @@ public void configure(Binder binder) binder.bind(RedirectInfo.class).to(CoordinatorRedirectInfo.class).in(LazySingleton.class); } - binder.bind(MetadataSegmentManager.class) - .toProvider(MetadataSegmentManagerProvider.class) + binder.bind(MetadataSegments.class) + .toProvider(MetadataSegmentsProvider.class) .in(ManageLifecycle.class); binder.bind(MetadataRuleManager.class) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 6316b40f991f..6436b5bbe3fd 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -479,7 +479,7 @@ public Enumerable scan(DataContext root) final List druidServers = serverView.getDruidServers(); final int serverSegmentsTableSize = SERVER_SEGMENTS_SIGNATURE.getRowOrder().size(); for (ImmutableDruidServer druidServer : druidServers) { - for (DataSegment segment : druidServer.getLazyAllSegments()) { + for (DataSegment segment : druidServer.iterateAllSegments()) { Object[] row = new Object[serverSegmentsTableSize]; row[0] = druidServer.getHost(); row[1] = segment.getId(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java index ba44081fc712..34170ab0418c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java @@ -271,7 +271,8 @@ public void testSegmentMetadataHolderNumRows() final Pair pair = druidServers .stream() .flatMap(druidServer -> druidServer - .getLazyAllSegments().stream() + .iterateAllSegments() + .stream() .filter(segment -> segment.equals(existingSegment)) .map(segment -> Pair.of(druidServer, segment)) ) diff --git a/web-console/README.md b/web-console/README.md index ad02c1034316..7617ba7de792 100644 --- a/web-console/README.md +++ b/web-console/README.md @@ -58,7 +58,7 @@ GET /druid/indexer/v1/supervisor?full GET /druid/indexer/v1/workers GET /druid/coordinator/v1/loadqueue?simple GET /druid/coordinator/v1/config -GET /druid/coordinator/v1/metadata/datasources?includeDisabled +GET /druid/coordinator/v1/metadata/datasources?includeUnused GET /druid/coordinator/v1/rules GET /druid/coordinator/v1/config/compaction GET /druid/coordinator/v1/tiers diff --git a/web-console/old-console/css/enable.css b/web-console/old-console/css/use_data_source.css similarity index 97% rename from web-console/old-console/css/enable.css rename to web-console/old-console/css/use_data_source.css index dfcc3eddec80..9504f1adf120 100644 --- a/web-console/old-console/css/enable.css +++ b/web-console/old-console/css/use_data_source.css @@ -16,6 +16,6 @@ * limitations under the License. */ -#select_datasource { +#select_data_source { margin: 20px 0 20px 0; } \ No newline at end of file diff --git a/web-console/old-console/index.html b/web-console/old-console/index.html index 2b9a8fe79b1f..48bb1b518874 100644 --- a/web-console/old-console/index.html +++ b/web-console/old-console/index.html @@ -42,7 +42,7 @@ Configure Coordinator Parameters
Permanent Segment Deletion diff --git a/web-console/old-console/js/kill-0.0.1.js b/web-console/old-console/js/kill-0.0.1.js index f59f945879a3..50e59cd6859f 100644 --- a/web-console/old-console/js/kill-0.0.1.js +++ b/web-console/old-console/js/kill-0.0.1.js @@ -59,7 +59,7 @@ $(document).ready(function() { } }); - $.getJSON("/druid/coordinator/v1/metadata/datasources?includeDisabled", function(data) { + $.getJSON("/druid/coordinator/v1/metadata/datasources?includeUnused", function(data) { $.each(data, function(index, datasource) { $('#datasources').append($('').val(datasource).text(datasource)); }); diff --git a/web-console/old-console/js/enable-0.0.1.js b/web-console/old-console/js/use-data-source-0.0.1.js similarity index 70% rename from web-console/old-console/js/enable-0.0.1.js rename to web-console/old-console/js/use-data-source-0.0.1.js index ba2c18b32afc..1c739a70a257 100644 --- a/web-console/old-console/js/enable-0.0.1.js +++ b/web-console/old-console/js/use-data-source-0.0.1.js @@ -30,13 +30,13 @@ $(document).ready(function() { } }); - $("#enable_dialog").dialog({ + $("#use_dialog").dialog({ autoOpen: false, modal:true, resizeable: false, buttons: { Yes : function() { - var selected = $('#datasources option:selected').text(); + var selected = $('#data_sources option:selected').text(); $.ajax({ type: 'POST', url:'/druid/coordinator/v1/datasources/' + selected, @@ -44,12 +44,12 @@ $(document).ready(function() { contentType:"application/json; charset=utf-8", dataType:"text", error: function(xhr, status, error) { - $("#enable_dialog").dialog("close"); + $("#use_dialog").dialog("close"); $("#error_dialog").html(xhr.responseText); $("#error_dialog").dialog("open"); }, success: function(data, status, xhr) { - $("#enable_dialog").dialog("close"); + $("#use_dialog").dialog("close"); } }); }, @@ -59,13 +59,13 @@ $(document).ready(function() { } }); - $("#disable_dialog").dialog({ + $("#unuse_dialog").dialog({ autoOpen: false, modal:true, resizeable: false, buttons: { Yes : function() { - var selected = $('#datasources option:selected').text(); + var selected = $('#data_sources option:selected').text(); $.ajax({ type: 'DELETE', url:'/druid/coordinator/v1/datasources/' + selected, @@ -73,12 +73,12 @@ $(document).ready(function() { contentType:"application/json; charset=utf-8", dataType:"text", error: function(xhr, status, error) { - $("#disable_dialog").dialog("close"); + $("#unuse_dialog").dialog("close"); $("#error_dialog").html(xhr.responseText); $("#error_dialog").dialog("open"); }, success: function(data, status, xhr) { - $("#disable_dialog").dialog("close"); + $("#unuse_dialog").dialog("close"); } }); }, @@ -88,27 +88,27 @@ $(document).ready(function() { } }); - $.getJSON("/druid/coordinator/v1/metadata/datasources", function(enabled_datasources) { - $.each(enabled_datasources, function(index, datasource) { - $('#enabled_datasources').append($('
  • ' + datasource + '
  • ')); + $.getJSON("/druid/coordinator/v1/metadata/datasources", function(used_data_sources) { + $.each(used_data_sources, function(index, data_source) { + $('#used_data_sources').append($('
  • ' + data_source + '
  • ')); }); - $.getJSON("/druid/coordinator/v1/metadata/datasources?includeDisabled", function(db_datasources) { - var disabled_datasources = _.difference(db_datasources, enabled_datasources); - $.each(disabled_datasources, function(index, datasource) { - $('#disabled_datasources').append($('
  • ' + datasource + '
  • ')); + $.getJSON("/druid/coordinator/v1/metadata/datasources?includeUnused", function(all_data_sources) { + var unused_data_sources = _.difference(all_data_sources, used_data_sources); + $.each(unused_data_sources, function(index, data_source) { + $('#unused_data_sources').append($('
  • ' + data_source + '
  • ')); }); - $.each(db_datasources, function(index, datasource) { - $('#datasources').append($('').val(datasource).text(datasource)); + $.each(all_data_sources, function(index, data_source) { + $('#data_sources').append($('').val(data_source).text(data_source)); }); }); }); - $("#enable").click(function() { - $("#enable_dialog").dialog("open"); + $("#use").click(function() { + $("#use_dialog").dialog("open"); }); - $('#disable').click(function (){ - $("#disable_dialog").dialog("open") + $("#unuse").click(function() { + $("#unuse_dialog").dialog("open"); }); }); diff --git a/web-console/old-console/kill.html b/web-console/old-console/kill.html index f78f14b95057..5bcaee178e59 100644 --- a/web-console/old-console/kill.html +++ b/web-console/old-console/kill.html @@ -20,7 +20,7 @@ - Druid Coordinator Console - Enable/Disable Datasources + Druid Coordinator Console - Permanently Delete Segments diff --git a/web-console/old-console/enable.html b/web-console/old-console/use-data-source.html similarity index 60% rename from web-console/old-console/enable.html rename to web-console/old-console/use-data-source.html index eb10b8412b67..44a53987d50e 100644 --- a/web-console/old-console/enable.html +++ b/web-console/old-console/use-data-source.html @@ -20,52 +20,52 @@ - Druid Coordinator Console - Enable/Disable Datasources + Druid Coordinator Console - Mark all segments in data sources as used or unused - + - +
    -
    Enable/Disable Datasources
    +
    Mark all segments in data sources as used or unused
    -

    Enabled Datasources:

    -
      +

      Data sources with used segments:

      +
    -

    Disabled Datasources:

    -
      +

      Data sources without used segments:

      +
    -
    - Select Data Source: -
    - -
    -

    Are you sure you want to enable the selected datasource?

    + +
    +

    Are you sure you want to mark as used all segments belonging to the selected data source?

    - -
    -

    Are you sure you want to disable the selected datasource?

    + +
    +

    Are you sure you want to mark as unused all segments belonging to the selected data source?

    diff --git a/web-console/src/views/datasource-view.tsx b/web-console/src/views/datasource-view.tsx index 3ee04a237f9c..9e915fb21200 100644 --- a/web-console/src/views/datasource-view.tsx +++ b/web-console/src/views/datasource-view.tsx @@ -58,16 +58,16 @@ export interface DatasourcesViewState { datasourcesError: string | null; datasourcesFilter: Filter[]; - showDisabled: boolean; + showUnused: boolean; retentionDialogOpenOn: { datasource: string, rules: any[] } | null; compactionDialogOpenOn: {datasource: string, configData: any} | null; dropDataDatasource: string | null; - enableDatasource: string | null; + dataSourceToMarkAsUsedAllSegmentsIn: string | null; killDatasource: string | null; } export class DatasourcesView extends React.Component { - static DISABLED_COLOR = '#0a1500'; + static UNUSED_COLOR = '#0a1500'; static FULLY_AVAILABLE_COLOR = '#57d500'; static PARTIALLY_AVAILABLE_COLOR = '#ffbf00'; @@ -93,11 +93,11 @@ export class DatasourcesView extends React.Component x.datasource); - const disabledResp = await axios.get('/druid/coordinator/v1/metadata/datasources?includeDisabled'); - const disabled: string[] = disabledResp.data.filter((d: string) => !seen[d]); + const allDataSourcesResp = await axios.get('/druid/coordinator/v1/metadata/datasources?includeUnused'); + const unused: string[] = allDataSourcesResp.data.filter((d: string) => !seen[d]); const rulesResp = await axios.get('/druid/coordinator/v1/rules'); const rules = rulesResp.data; @@ -120,7 +120,7 @@ export class DatasourcesView extends React.Component ({ datasource: d, disabled: true }))); + const allDatasources = datasources.concat(unused.map(d => ({ datasource: d, unused: true }))); allDatasources.forEach((ds: any) => { ds.rules = rules[ds.datasource] || []; ds.compaction = compaction[ds.datasource]; @@ -182,27 +182,27 @@ GROUP BY 1`); ; } - renderEnableAction() { - const { enableDatasource } = this.state; + renderUseAction() { + const { dataSourceToMarkAsUsedAllSegmentsIn } = this.state; return { - const resp = await axios.post(`/druid/coordinator/v1/datasources/${enableDatasource}`, {}); + dataSourceToMarkAsUsedAllSegmentsIn ? async () => { + const resp = await axios.post(`/druid/coordinator/v1/datasources/${dataSourceToMarkAsUsedAllSegmentsIn}`, {}); return resp.data; } : null } - confirmButtonText="Enable datasource" - successText="Datasource has been enabled" - failText="Could not enable datasource" + confirmButtonText="Mark as used all segments belonging to data source" + successText="All segments belonging to data source has been marked as used" + failText="Failed to mark as used all segments belonging to data source" intent={Intent.PRIMARY} onClose={(success) => { - this.setState({ enableDatasource: null }); + this.setState({ dataSourceToMarkAsUsedAllSegmentsIn: null }); if (success) this.datasourceQueryManager.rerunLastQuery(); }} >

    - {`Are you sure you want to enable datasource '${enableDatasource}'?`} + {`Are you sure you want to mark as used all segments belonging to data source '${dataSourceToMarkAsUsedAllSegmentsIn}'?`}

    ; } @@ -341,11 +341,11 @@ GROUP BY 1`); renderDatasourceTable() { const { goToSegments } = this.props; - const { datasources, defaultRules, datasourcesLoading, datasourcesError, datasourcesFilter, showDisabled } = this.state; + const { datasources, defaultRules, datasourcesLoading, datasourcesError, datasourcesFilter, showUnused } = this.state; let data = datasources || []; - if (!showDisabled) { - data = data.filter(d => !d.disabled); + if (!showUnused) { + data = data.filter(d => !d.unused); } return <> @@ -374,12 +374,12 @@ GROUP BY 1`); filterable: false, accessor: (row) => row.num_available_segments / row.num_segments, Cell: (row) => { - const { datasource, num_available_segments, num_segments, disabled } = row.original; + const { datasource, num_available_segments, num_segments, unused } = row.original; - if (disabled) { + if (unused) { return - ●  - Disabled + ●  + Unused ; } @@ -473,10 +473,10 @@ GROUP BY 1`); filterable: false, Cell: row => { const datasource = row.value; - const { disabled } = row.original; - if (disabled) { + const { unused } = row.original; + if (unused) { return ; } else { @@ -491,7 +491,7 @@ GROUP BY 1`); className="-striped -highlight" /> {this.renderDropDataAction()} - {this.renderEnableAction()} + {this.renderUseAction()} {this.renderKillAction()} {this.renderRetentionDialog()} {this.renderCompactionDialog()} @@ -500,7 +500,7 @@ GROUP BY 1`); render() { const { goToSql } = this.props; - const { showDisabled } = this.state; + const { showUnused } = this.state; return
    @@ -516,9 +516,9 @@ GROUP BY 1`); onClick={() => goToSql(this.datasourceQueryManager.getLastQuery())} /> this.setState({ showDisabled: !showDisabled })} + checked={showUnused} + label="Show data sources without used segments" + onChange={() => this.setState({ showUnused: !showUnused })} />
    {this.renderDatasourceTable()} From 25de70d6334cee9c94d2375d7fc37260a1ed03ee Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 20 Mar 2019 18:31:39 -0300 Subject: [PATCH 02/40] Fix brace --- .../org/apache/druid/timeline/VersionedIntervalTimeline.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java index bf20fa5a4e17..8d370019efa4 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -143,7 +143,8 @@ public Collection iterateAllObjects() ); } - public int getNumObjects() { + public int getNumObjects() + { return numObjects.get(); } From fe13d9c7d64e76d42de1efb3aff2ef2fec7a20a8 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 20 Mar 2019 21:00:25 -0300 Subject: [PATCH 03/40] Import order --- .../apache/druid/guice/SQLMetadataStorageDruidModule.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java index 10fe2ee60ed3..74fa2b9726c0 100644 --- a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java @@ -29,10 +29,10 @@ import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManagerProvider; -import org.apache.druid.metadata.MetadataSegments; -import org.apache.druid.metadata.MetadataSegmentsProvider; import org.apache.druid.metadata.MetadataSegmentPublisher; import org.apache.druid.metadata.MetadataSegmentPublisherProvider; +import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.MetadataSegmentsProvider; import org.apache.druid.metadata.MetadataStorageActionHandlerFactory; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageProvider; @@ -40,11 +40,11 @@ import org.apache.druid.metadata.SQLMetadataConnector; import org.apache.druid.metadata.SQLMetadataRuleManager; import org.apache.druid.metadata.SQLMetadataRuleManagerProvider; -import org.apache.druid.metadata.SqlMetadataSegments; -import org.apache.druid.metadata.SqlMetadataSegmentsProvider; import org.apache.druid.metadata.SQLMetadataSegmentPublisher; import org.apache.druid.metadata.SQLMetadataSegmentPublisherProvider; import org.apache.druid.metadata.SQLMetadataSupervisorManager; +import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlMetadataSegmentsProvider; import org.apache.druid.server.audit.AuditManagerProvider; import org.apache.druid.server.audit.SQLAuditManager; import org.apache.druid.server.audit.SQLAuditManagerConfig; From 050bd3b238a0b66dfabe3ed855e1871214d84163 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 21 Mar 2019 17:09:49 -0300 Subject: [PATCH 04/40] Rename withKillDataSourceWhitelist to withSpecificDataSourcesToKill --- .../druid/server/coordinator/CoordinatorDynamicConfig.java | 7 +++---- .../server/coordinator/DruidCoordinatorRuleRunnerTest.java | 2 +- .../druid/server/http/CoordinatorDynamicConfigTest.java | 2 +- 3 files changed, 5 insertions(+), 6 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 9ba84de16606..c0ae5fdc2e89 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 @@ -82,7 +82,6 @@ public CoordinatorDynamicConfig( @JsonProperty("replicationThrottleLimit") int replicationThrottleLimit, @JsonProperty("balancerComputeThreads") int balancerComputeThreads, @JsonProperty("emitBalancingStats") boolean emitBalancingStats, - // Type is Object here so that we can support both string and list as // coordinator console can not send array of strings in the update request. // See https://github.com/apache/incubator-druid/issues/3055 @@ -279,7 +278,7 @@ public String toString() ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + ", killAllDataSources=" + killAllDataSources + - ", killDataSourceWhitelist=" + specificDataSourcesToKill + + ", specificDataSourcesToKill=" + specificDataSourcesToKill + ", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + ", decommissioningNodes=" + decommissioningNodes + @@ -482,9 +481,9 @@ public Builder withEmitBalancingStats(boolean emitBalancingStats) return this; } - public Builder withKillDataSourceWhitelist(Set killDataSourceWhitelist) + public Builder withSpecificDataSourcesToKill(Set specificDataSourcesToKill) { - this.specificDataSourcesToKill = killDataSourceWhitelist; + this.specificDataSourcesToKill = specificDataSourcesToKill; return this; } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 725671c37338..5a31c1c3d79e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -1421,7 +1421,7 @@ private CoordinatorDynamicConfig createCoordinatorDynamicConfig() .withReplicationThrottleLimit(24) .withBalancerComputeThreads(0) .withEmitBalancingStats(false) - .withKillDataSourceWhitelist(null) + .withSpecificDataSourcesToKill(null) .withKillAllDataSources(false) .withMaxSegmentsInNodeLoadingQueue(1000) .build(); 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 647039539ba5..bc25effd9b60 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 @@ -224,7 +224,7 @@ public void testBuilderDefaults() public void testUpdate() { CoordinatorDynamicConfig current = CoordinatorDynamicConfig.builder() - .withKillDataSourceWhitelist(ImmutableSet.of("x")) + .withSpecificDataSourcesToKill(ImmutableSet.of("x")) .build(); Assert.assertEquals( From 562f5b41d61dcd7a25b3483c9015c88549ee0585 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 21 Mar 2019 17:22:41 -0300 Subject: [PATCH 05/40] Fix tests --- .../client/ImmutableDruidServerTests.java | 37 +++++++++++++++++++ .../coordinator/CostBalancerStrategyTest.java | 3 +- ...iskNormalizedCostBalancerStrategyTest.java | 3 +- .../DruidCoordinatorBalancerProfiler.java | 9 +++-- .../DruidCoordinatorBalancerTest.java | 3 +- .../ReservoirSegmentSamplerTest.java | 9 +++-- 6 files changed, 53 insertions(+), 11 deletions(-) create mode 100644 server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java diff --git a/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java new file mode 100644 index 000000000000..8cc5c03c137a --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java @@ -0,0 +1,37 @@ +/* + * 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.client; + +import org.apache.druid.timeline.DataSegment; +import org.easymock.EasyMock; + +import java.util.Collection; + +public final class ImmutableDruidServerTests +{ + + public static void expectSegments(ImmutableDruidServer mockServer, Collection segments) + { + EasyMock.expect(mockServer.iterateAllSegments()).andReturn(segments).anyTimes(); + EasyMock.expect(mockServer.getNumSegments()).andReturn(segments.size()).anyTimes(); + } + + private ImmutableDruidServerTests() {} +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java index 4259689d4d3c..3074fb2c0fe1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java @@ -23,6 +23,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -94,7 +95,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme segments.put(segment.getId(), segment); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); } - EasyMock.expect(druidServer.iterateAllSegments()).andReturn(segments.values()).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments.values()); EasyMock.replay(druidServer); serverHolderList.add(new ServerHolder(druidServer, fromPeon)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java index 6d61a4f3a85a..bd2aa6eaf14a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java @@ -23,6 +23,7 @@ import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -88,7 +89,7 @@ public static List setupDummyCluster(int serverCount, int maxSegme segments.add(segment); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); } - EasyMock.expect(druidServer.iterateAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments); EasyMock.replay(druidServer); serverHolderList.add(new ServerHolder(druidServer, fromPeon)); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java index de33a5256dd4..c8718f91c385 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java @@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -116,9 +117,9 @@ public void bigProfiler() EasyMock.expect(server.getName()).andReturn(Integer.toString(i)).atLeastOnce(); EasyMock.expect(server.getHost()).andReturn(Integer.toString(i)).anyTimes(); if (i == 0) { - EasyMock.expect(server.iterateAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(server, segments); } else { - EasyMock.expect(server.iterateAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + ImmutableDruidServerTests.expectSegments(server, Collections.emptyList()); } EasyMock.expect(server.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(server); @@ -197,7 +198,7 @@ public void profileRun() EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.iterateAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer1, segments); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -205,7 +206,7 @@ public void profileRun() EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer2.iterateAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer2, Collections.emptyList()); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java index 9ede2d8e176f..68a7f6a70ba1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java @@ -25,6 +25,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; @@ -558,7 +559,7 @@ private static void mockDruidServer( EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes(); EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce(); EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).atLeastOnce(); - EasyMock.expect(druidServer.iterateAllSegments()).andReturn(segments).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer, segments); EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); if (!segments.isEmpty()) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java index 7d90000a4972..73e829ce0e0c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java @@ -21,6 +21,7 @@ import com.google.common.collect.Lists; import org.apache.druid.client.ImmutableDruidServer; +import org.apache.druid.client.ImmutableDruidServerTests; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -138,7 +139,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer1.iterateAllSegments()).andReturn(segments1).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer1, segments1); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -146,7 +147,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer2.iterateAllSegments()).andReturn(segments2).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer2, segments2); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); @@ -154,7 +155,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer3.iterateAllSegments()).andReturn(segments3).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer3, segments3); EasyMock.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer3); @@ -162,7 +163,7 @@ public void getRandomBalancerSegmentHolderTest() EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).atLeastOnce(); - EasyMock.expect(druidServer4.iterateAllSegments()).andReturn(segments4).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer4, segments4); EasyMock.expect(druidServer4.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer4); From a6138daa3c336eace337af5bf9c041bd80f83d51 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 25 Mar 2019 18:29:02 -0300 Subject: [PATCH 06/40] Fix tests by adding proper methods without interval parameters to IndexerMetadataStorageCoordinator instead of hacking with Intervals.ETERNITY --- .../MaterializedViewSupervisor.java | 13 +- .../common/actions/SegmentAllocateAction.java | 2 +- .../IndexerMetadataStorageAdapter.java | 2 +- .../IndexerMetadataStorageAdapterTest.java | 4 +- ...TestIndexerMetadataStorageCoordinator.java | 18 ++- .../IndexerMetadataStorageCoordinator.java | 49 ++++-- .../IndexerSQLMetadataStorageCoordinator.java | 147 +++++++++++------- ...exerSQLMetadataStorageCoordinatorTest.java | 2 +- 8 files changed, 149 insertions(+), 88 deletions(-) diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index c6bb1c39435c..d366d0d4ff87 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -37,7 +37,6 @@ import org.apache.druid.indexing.overlord.supervisor.SupervisorReport; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; @@ -69,6 +68,7 @@ public class MaterializedViewSupervisor implements Supervisor private static final int DEFAULT_MAX_TASK_COUNT = 1; // there is a lag between derivatives and base dataSource, to prevent repeatedly building for some delay data. private static final long DEFAULT_MIN_DATA_LAG_MS = 24 * 3600 * 1000L; + private final MetadataSupervisorManager metadataSupervisorManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final SqlMetadataSegments metadataSegments; @@ -317,16 +317,11 @@ Pair, Map>> checkSegment { // Pair version, interval -> list> Pair, Map>> derivativeSegmentsSnapshot = - getVersionAndBaseSegments( - metadataStorageCoordinator.getUsedSegmentsForInterval(dataSource, Intervals.ETERNITY) - ); + getVersionAndBaseSegments(metadataStorageCoordinator.getUsedSegments(dataSource)); // Pair max(created_date), interval -> list> Pair, Map>> baseSegmentsSnapshot = getMaxCreateDateAndBaseSegments( - metadataStorageCoordinator.getUsedSegmentAndCreatedDateForInterval( - spec.getBaseDataSource(), - Intervals.ETERNITY - ) + metadataStorageCoordinator.getUsedSegmentsAndCreatedDates(spec.getBaseDataSource()) ); // baseSegments are used to create HadoopIndexTask Map> baseSegments = baseSegmentsSnapshot.rhs; @@ -468,7 +463,7 @@ private void clearTasks() private void clearSegments() { log.info("Clear all metadata of dataSource %s", dataSource); - metadataStorageCoordinator.deletePendingSegments(dataSource, Intervals.ETERNITY); + metadataStorageCoordinator.deletePendingSegments(dataSource); if (!metadataSegments.tryMarkAsUnusedAllSegmentsInDataSource(dataSource)) { log.error("Failed to mark all segments in " + dataSource + " as unused."); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java index 6623483667e4..927bdacd4531 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java @@ -179,7 +179,7 @@ public SegmentIdWithShardSpec perform( } // Could not allocate a pending segment. There's a chance that this is because someone else inserted a segment - // overlapping with this row between when we called "mdc.getUsedSegmentsForInterval" and now. Check it again, + // overlapping with this row between when we called "msc.getUsedSegmentsForInterval" and now. Check it again, // and if it's different, repeat. if (!ImmutableSet.copyOf(msc.getUsedSegmentsForInterval(dataSource, rowInterval)).equals(usedSegmentsForRow)) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java index cd550c2ad7bf..4671c5bb7a96 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java @@ -64,6 +64,6 @@ public int deletePendingSegments(String dataSource, Interval deleteInterval) activeTaskInterval.getStart() ); - return indexerMetadataStorageCoordinator.deletePendingSegments(dataSource, deleteInterval); + return indexerMetadataStorageCoordinator.deletePendingSegmentsCreatedInInterval(dataSource, deleteInterval); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java index 8cb45ade0f09..0c54dd8b0bc5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapterTest.java @@ -81,7 +81,7 @@ public void testDeletePendingSegments() final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01"); EasyMock .expect( - indexerMetadataStorageCoordinator.deletePendingSegments(EasyMock.anyString(), EasyMock.eq(deleteInterval)) + indexerMetadataStorageCoordinator.deletePendingSegmentsCreatedInInterval(EasyMock.anyString(), EasyMock.eq(deleteInterval)) ) .andReturn(10); EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); @@ -114,7 +114,7 @@ public void testDeletePendingSegmentsOfRunningTasks() final Interval deleteInterval = Intervals.of("2017-01-01/2017-12-01"); EasyMock .expect( - indexerMetadataStorageCoordinator.deletePendingSegments(EasyMock.anyString(), EasyMock.eq(deleteInterval)) + indexerMetadataStorageCoordinator.deletePendingSegmentsCreatedInInterval(EasyMock.anyString(), EasyMock.eq(deleteInterval)) ) .andReturn(10); EasyMock.replay(taskStorageQueryAdapter, indexerMetadataStorageCoordinator); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 0eeecd5375bd..1e497459a4d5 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -77,11 +77,17 @@ public List getUsedSegmentsForInterval(String dataSource, Interval } @Override - public List> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval) + public List getUsedSegments(String dataSource) { return ImmutableList.of(); } - + + @Override + public List> getUsedSegmentsAndCreatedDates(String dataSource) + { + return ImmutableList.of(); + } + @Override public List getUsedSegmentsForIntervals(String dataSource, List intervals) { @@ -133,7 +139,13 @@ public SegmentIdWithShardSpec allocatePendingSegment( } @Override - public int deletePendingSegments(String dataSource, Interval deleteInterval) + public int deletePendingSegmentsCreatedInInterval(String dataSource, Interval deleteInterval) + { + throw new UnsupportedOperationException(); + } + + @Override + public int deletePendingSegments(String dataSource) { throw new UnsupportedOperationException(); } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index f628b4c1c760..6ecdea914908 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -34,14 +34,14 @@ public interface IndexerMetadataStorageCoordinator { /** - * Get all segments which may include any data in the interval and are flagged as used. + * Get all segments which may include any data in the interval and are marked as used. * * @param dataSource The datasource to query - * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, end is exclusive - * - * @return The DataSegments which include data in the requested interval. These segments may contain data outside the requested interval. + * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, + * end is exclusive * - * @throws IOException + * @return The DataSegments which include data in the requested interval. These segments may contain data outside the + * requested interval. */ default List getUsedSegmentsForInterval(String dataSource, Interval interval) { @@ -49,14 +49,23 @@ default List getUsedSegmentsForInterval(String dataSource, Interval } /** - * Get all used segments and the created_date of these segments in a given datasource and interval + * Get all used segments in the data source. + * + * @param dataSource The data source to query + * + * @return all segments belonging to the given data source + * @see #getUsedSegmentsForInterval(String, Interval) similar to this method but also accepts data interval. + */ + List getUsedSegments(String dataSource); + + /** + * Get all used segments and the created_date of these segments belonging to the given data source. * * @param dataSource The datasource to query - * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, end is exclusive * - * @return The DataSegments and the related created_date of segments which include data in the requested interval + * @return DataSegments and the related created_date of segments */ - List> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval); + List> getUsedSegmentsAndCreatedDates(String dataSource); /** * Get all segments which may include any data in the interval and are flagged as used. @@ -66,8 +75,6 @@ default List getUsedSegmentsForInterval(String dataSource, Interval * * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the * requested interval. - * - * @throws IOException */ List getUsedSegmentsForIntervals(String dataSource, List intervals); @@ -111,15 +118,29 @@ SegmentIdWithShardSpec allocatePendingSegment( ); /** - * Delete pending segments created in the given interval for the given dataSource from the pending segments table. - * The {@code created_date} field of the pending segments table is checked to find segments to be deleted. + * Delete pending segments created in the given interval belonging to the given data source from the pending segments + * table. The {@code created_date} field of the pending segments table is checked to find segments to be deleted. + * + * Note that the semantic of the interval (for `created_date`s) is different from the semantic of the interva + * parameters in some other methods in this class, such as {@link #getUsedSegmentsForInterval} (where the interval + * is about the time column value in rows belonging to the segment). * * @param dataSource dataSource * @param deleteInterval interval to check the {@code created_date} of pendingSegments * * @return number of deleted pending segments */ - int deletePendingSegments(String dataSource, Interval deleteInterval); + int deletePendingSegmentsCreatedInInterval(String dataSource, Interval deleteInterval); + + /** + * Delete all pending segments belonging to the given data source from the pending segments table. + * The {@code created_date} field of the pending segments table is checked to find segments to be deleted. + * + * @return number of deleted pending segments + * @see #deletePendingSegmentsCreatedInInterval(String, Interval) similar to this method but also accepts interval for + * segments' `created_date`s + */ + int deletePendingSegments(String dataSource); /** * Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 3cdbd0d022e1..5f2f200e7c4e 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -59,15 +59,14 @@ import org.skife.jdbi.v2.TransactionStatus; import org.skife.jdbi.v2.exceptions.CallbackFailedException; import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; import org.skife.jdbi.v2.util.ByteArrayMapper; import org.skife.jdbi.v2.util.StringMapper; import javax.annotation.Nullable; import java.io.IOException; import java.sql.ResultSet; -import java.sql.SQLException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -116,14 +115,28 @@ public void start() @Override public List getUsedSegmentsForIntervals(final String dataSource, final List intervals) + { + if (intervals == null || intervals.isEmpty()) { + throw new IAE("null/empty intervals"); + } + return doGetUsedSegments(dataSource, intervals); + } + + @Override + public List getUsedSegments(String dataSource) + { + return doGetUsedSegments(dataSource, Collections.emptyList()); + } + + /** + * @param intervals empty list means unrestricted interval. + */ + private List doGetUsedSegments(final String dataSource, final List intervals) { return connector.retryWithHandle( handle -> { - final VersionedIntervalTimeline timeline = getTimelineForIntervalsWithHandle( - handle, - dataSource, - intervals - ); + final VersionedIntervalTimeline timeline = + getTimelineForIntervalsWithHandle(handle, dataSource, intervals); return intervals .stream() @@ -178,29 +191,25 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi final List intervals ) { - if (intervals == null || intervals.isEmpty()) { - throw new IAE("null/empty intervals"); - } - final StringBuilder sb = new StringBuilder(); - sb.append("SELECT payload FROM %s WHERE used = true AND dataSource = ? AND ("); - for (int i = 0; i < intervals.size(); i++) { - sb.append( - StringUtils.format("(start <= ? AND %1$send%1$s >= ?)", connector.getQuoteString()) - ); - if (i == intervals.size() - 1) { - sb.append(")"); - } else { - sb.append(" OR "); + sb.append("SELECT payload FROM %s WHERE used = true AND dataSource = ?"); + if (!intervals.isEmpty()) { + sb.append(" AND ("); + for (int i = 0; i < intervals.size(); i++) { + sb.append( + StringUtils.format("(start <= ? AND %1$send%1$s >= ?)", connector.getQuoteString()) + ); + if (i == intervals.size() - 1) { + sb.append(")"); + } else { + sb.append(" OR "); + } } } - Query> sql = handle.createQuery( - StringUtils.format( - sb.toString(), - dbTables.getSegmentsTable() - ) - ).bind(0, dataSource); + Query> sql = handle + .createQuery(StringUtils.format(sb.toString(), dbTables.getSegmentsTable())) + .bind(0, dataSource); for (int i = 0; i < intervals.size(); i++) { Interval interval = intervals.get(i); @@ -714,13 +723,13 @@ private SegmentIdWithShardSpec createNewSegment( } @Override - public int deletePendingSegments(String dataSource, Interval deleteInterval) + public int deletePendingSegmentsCreatedInInterval(String dataSource, Interval deleteInterval) { return connector.getDBI().inTransaction( (handle, status) -> handle .createStatement( StringUtils.format( - "delete from %s where datasource = :dataSource and created_date >= :start and created_date < :end", + "DELETE FROM %s WHERE datasource = :dataSource AND created_date >= :start AND created_date < :end", dbTables.getPendingSegmentsTable() ) ) @@ -731,6 +740,19 @@ public int deletePendingSegments(String dataSource, Interval deleteInterval) ); } + @Override + public int deletePendingSegments(String dataSource) + { + return connector.getDBI().inTransaction( + (handle, status) -> handle + .createStatement( + StringUtils.format("DELETE FROM %s WHERE datasource = :dataSource", dbTables.getPendingSegmentsTable()) + ) + .bind("dataSource", dataSource) + .execute() + ); + } + /** * Attempts to insert a single segment to the database. If the segment already exists, will do nothing; although, * this checking is imperfect and callers must be prepared to retry their entire transaction on exceptions. @@ -1115,37 +1137,48 @@ public List fold( } @Override - public List> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval) + public List> getUsedSegmentsAndCreatedDates(String dataSource) + { + return doGetUsedSegmentsAndCreatedDates(dataSource, null); + } + + /** + * @param interval if null, assumed unrestricted interval + */ + private List> doGetUsedSegmentsAndCreatedDates( + String dataSource, + @Nullable Interval interval + ) { + String rawQueryString = "SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource AND used = true"; + if (interval != null) { + rawQueryString += StringUtils.format(" AND start >= :start AND %1$send%1$s <= :end", connector.getQuoteString()); + } + final String queryString = StringUtils.format(rawQueryString, dbTables.getSegmentsTable()); return connector.retryWithHandle( - handle -> handle.createQuery( - StringUtils.format( - "SELECT created_date, payload FROM %1$s WHERE dataSource = :dataSource " + - "AND start >= :start AND %2$send%2$s <= :end AND used = true", - dbTables.getSegmentsTable(), connector.getQuoteString() - ) - ) - .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .map(new ResultSetMapper>() - { - @Override - public Pair map(int index, ResultSet r, StatementContext ctx) - throws SQLException - { - try { - return new Pair<>( - jsonMapper.readValue(r.getBytes("payload"), DataSegment.class), - r.getString("created_date") - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - }) - .list() + handle -> { + Query> query = handle + .createQuery(queryString) + .bind("dataSource", dataSource); + if (interval != null) { + query = query + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + } + return query + .map((int index, ResultSet r, StatementContext ctx) -> { + try { + return new Pair<>( + jsonMapper.readValue(r.getBytes("payload"), DataSegment.class), + r.getString("created_date") + ); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }) + .list(); + } ); } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 879ddfb13b4c..7972c09ed8e1 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -934,7 +934,7 @@ public void testDeletePendingSegment() throws InterruptedException prevSegmentId = identifier.toString(); } - final int numDeleted = coordinator.deletePendingSegments(dataSource, new Interval(begin, secondBegin)); + final int numDeleted = coordinator.deletePendingSegmentsCreatedInInterval(dataSource, new Interval(begin, secondBegin)); Assert.assertEquals(10, numDeleted); } } From cb6fe7f26f5bb6c5390ea41069a2afd9eb078330 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 28 Mar 2019 13:56:25 -0300 Subject: [PATCH 07/40] More aligned names of DruidCoordinatorHelpers, rename several CoordinatorDynamicConfig parameters --- .idea/inspectionProfiles/Druid.xml | 8 +- docs/content/configuration/index.md | 2 +- .../indexing/common/task/CompactionTask.java | 6 + ...lTask.java => KillUnusedSegmentsTask.java} | 10 +- .../druid/indexing/common/task/Task.java | 2 +- .../task/batch/parallel/TaskMonitor.java | 2 +- .../overlord/http/OverlordResource.java | 11 +- .../indexing/common/task/TaskSerdeTest.java | 10 +- ...stractParallelIndexSupervisorTaskTest.java | 2 +- .../indexing/overlord/TaskLifecycleTest.java | 4 +- ...Query.java => ClientCompactTaskQuery.java} | 8 +- ...=> ClientKillUnusedSegmentsTaskQuery.java} | 7 +- ...{ClientQuery.java => ClientTaskQuery.java} | 13 +- .../indexing/HttpIndexingServiceClient.java | 31 +-- .../indexing/IndexingServiceClient.java | 4 +- .../client/indexing/TaskPayloadResponse.java | 6 +- .../druid/metadata/SqlMetadataSegments.java | 3 +- .../coordinator/CoordinatorDynamicConfig.java | 186 ++++++++++-------- .../server/coordinator/DruidCoordinator.java | 12 +- ...dCoordinatorKillStalePendingSegments.java} | 16 +- .../DruidCoordinatorRuntimeParams.java | 38 ++-- .../helper/DruidCoordinatorBalancer.java | 2 +- ...atorMarkAsUnusedOvershadowedSegments.java} | 56 +++--- .../helper/DruidCoordinatorRuleRunner.java | 8 +- .../DruidCoordinatorSegmentCompactor.java | 4 +- ...DruidCoordinatorUnloadUnusedSegments.java} | 11 +- ...DruidCoordinatorUnusedSegmentsKiller.java} | 14 +- .../server/http/DataSourcesResource.java | 4 +- ...=> ClientKillUnusedSegmentsQueryTest.java} | 14 +- .../indexing/NoopIndexingServiceClient.java | 4 +- .../DruidCoordinatorRuleRunnerTest.java | 22 ++- ...MarkAsUnusedOvershadowedSegmentsTest.java} | 8 +- ...dCoordinatorUnusedSegmentsKillerTest.java} | 4 +- .../http/CoordinatorDynamicConfigTest.java | 23 ++- .../server/http/DataSourcesResourceTest.java | 2 +- .../org/apache/druid/cli/CliCoordinator.java | 8 +- 36 files changed, 323 insertions(+), 242 deletions(-) rename indexing-service/src/main/java/org/apache/druid/indexing/common/task/{KillTask.java => KillUnusedSegmentsTask.java} (87%) rename server/src/main/java/org/apache/druid/client/indexing/{ClientCompactQuery.java => ClientCompactTaskQuery.java} (89%) rename server/src/main/java/org/apache/druid/client/indexing/{ClientKillQuery.java => ClientKillUnusedSegmentsTaskQuery.java} (79%) rename server/src/main/java/org/apache/druid/client/indexing/{ClientQuery.java => ClientTaskQuery.java} (60%) rename server/src/main/java/org/apache/druid/server/coordinator/{DruidCoordinatorCleanupPendingSegments.java => DruidCoordinatorKillStalePendingSegments.java} (83%) rename server/src/main/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorCleanupOvershadowed.java => DruidCoordinatorMarkAsUnusedOvershadowedSegments.java} (50%) rename server/src/main/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorCleanupUnusedSegments.java => DruidCoordinatorUnloadUnusedSegments.java} (85%) rename server/src/main/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorSegmentKiller.java => DruidCoordinatorUnusedSegmentsKiller.java} (91%) rename server/src/test/java/org/apache/druid/client/indexing/{ClientKillQueryTest.java => ClientKillUnusedSegmentsQueryTest.java} (75%) rename server/src/test/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorCleanupOvershadowedTest.java => DruidCoordinatorMarkAsUnusedOvershadowedSegmentsTest.java} (93%) rename server/src/test/java/org/apache/druid/server/coordinator/helper/{DruidCoordinatorSegmentKillerTest.java => DruidCoordinatorUnusedSegmentsKillerTest.java} (95%) diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml index 77078ae296f9..c046c952081c 100644 --- a/.idea/inspectionProfiles/Druid.xml +++ b/.idea/inspectionProfiles/Druid.xml @@ -92,6 +92,7 @@ + - - - + + diff --git a/docs/content/configuration/index.md b/docs/content/configuration/index.md index 0166099cb723..50d85d077865 100644 --- a/docs/content/configuration/index.md +++ b/docs/content/configuration/index.md @@ -791,7 +791,7 @@ Issuing a GET request at the same URL will return the spec that is currently in |Property|Description|Default| |--------|-----------|-------| -|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be active before it can start removing (marking unused) segments in metadata storage.|900000 (15 mins)| +|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be a leader before it can start marking as unused overshadowed segments in metadata storage.|900000 (15 mins)| |`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L| |`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.html).|100| |`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5| diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 53c3641916fd..7e2027ef638c 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -30,6 +30,7 @@ import com.google.common.collect.HashBiMap; import com.google.common.collect.Lists; import org.apache.druid.client.coordinator.CoordinatorClient; +import org.apache.druid.client.indexing.ClientCompactTaskQuery; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -103,6 +104,11 @@ import java.util.stream.IntStream; import java.util.stream.StreamSupport; +/** + * The client representation of this task is {@link ClientCompactTaskQuery}. JSON + * serialization fields of this class must correspond to those of {@link + * ClientCompactTaskQuery}. + */ public class CompactionTask extends AbstractTask { private static final Logger log = new Logger(CompactionTask.class); diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java similarity index 87% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java index 44c90bc8a47b..f6b2dd1e3803 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java @@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -37,13 +38,16 @@ import java.util.Map; /** + * The client representation of this task is {@link ClientKillUnusedSegmentsTaskQuery}. + * JSON serialization fields of this class must correspond to those of {@link + * ClientKillUnusedSegmentsTaskQuery}, except for "id" and "context" fields. */ -public class KillTask extends AbstractFixedIntervalTask +public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask { - private static final Logger log = new Logger(KillTask.class); + private static final Logger log = new Logger(KillUnusedSegmentsTask.class); @JsonCreator - public KillTask( + public KillUnusedSegmentsTask( @JsonProperty("id") String id, @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java index c32f27dc6d24..75d5dd81cc95 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java @@ -46,7 +46,7 @@ */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "kill", value = KillTask.class), + @JsonSubTypes.Type(name = "kill", value = KillUnusedSegmentsTask.class), @JsonSubTypes.Type(name = "move", value = MoveTask.class), @JsonSubTypes.Type(name = "archive", value = ArchiveTask.class), @JsonSubTypes.Type(name = "restore", value = RestoreTask.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java index ff1683668b25..839968869c53 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java @@ -191,7 +191,7 @@ public void stop() iterator.remove(); final String taskId = entry.runningTask.getId(); log.info("Request to kill subtask[%s]", taskId); - indexingServiceClient.killTask(taskId); + indexingServiceClient.cancelTask(taskId); numRunningTasks--; numKilledTasks++; } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java index e5abac4070e7..344ad55c3e60 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java @@ -32,6 +32,7 @@ import org.apache.druid.audit.AuditEntry; import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; +import org.apache.druid.client.indexing.ClientTaskQuery; import org.apache.druid.common.config.ConfigManager.SetResult; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.indexer.RunnerTaskState; @@ -141,14 +142,16 @@ public OverlordResource( this.authorizerMapper = authorizerMapper; } + /** + * Warning, magic: {@link org.apache.druid.client.indexing.HttpIndexingServiceClient#runTask} may call this method + * remotely with {@link ClientTaskQuery} objects, but we deserialize {@link Task} objects. See the comment for {@link + * ClientTaskQuery} for details. + */ @POST @Path("/task") @Consumes(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON) - public Response taskPost( - final Task task, - @Context final HttpServletRequest req - ) + public Response taskPost(final Task task, @Context final HttpServletRequest req) { final String dataSource = task.getDataSource(); final ResourceAction resourceAction = new ResourceAction( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java index 1d15b6808b37..076a9c10986b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import org.apache.druid.client.indexing.ClientKillQuery; +import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery; import org.apache.druid.guice.FirehoseModule; import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIngestionSpec; @@ -327,7 +327,7 @@ public void testIndexTaskwithResourceSerde() throws Exception @Test public void testKillTaskSerde() throws Exception { - final KillTask task = new KillTask( + final KillUnusedSegmentsTask task = new KillUnusedSegmentsTask( null, "foo", Intervals.of("2010-01-01/P1D"), @@ -337,7 +337,7 @@ public void testKillTaskSerde() throws Exception final String json = jsonMapper.writeValueAsString(task); Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change - final KillTask task2 = (KillTask) jsonMapper.readValue(json, Task.class); + final KillUnusedSegmentsTask task2 = (KillUnusedSegmentsTask) jsonMapper.readValue(json, Task.class); Assert.assertEquals("foo", task.getDataSource()); Assert.assertEquals(Intervals.of("2010-01-01/P1D"), task.getInterval()); @@ -347,9 +347,9 @@ public void testKillTaskSerde() throws Exception Assert.assertEquals(task.getDataSource(), task2.getDataSource()); Assert.assertEquals(task.getInterval(), task2.getInterval()); - final KillTask task3 = (KillTask) jsonMapper.readValue( + final KillUnusedSegmentsTask task3 = (KillUnusedSegmentsTask) jsonMapper.readValue( jsonMapper.writeValueAsString( - new ClientKillQuery( + new ClientKillUnusedSegmentsTaskQuery( "foo", Intervals.of("2010-01-01/P1D") ) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java index 8792956bc95a..a2de4517e002 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java @@ -197,7 +197,7 @@ public TaskStatusResponse getTaskStatus(String taskId) } @Override - public String killTask(String taskId) + public String cancelTask(String taskId) { final Future taskStatusFuture = tasks.remove(taskId); if (taskStatusFuture != null) { diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 6f2b99218a26..6c8b9f77f744 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -63,7 +63,7 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; -import org.apache.druid.indexing.common.task.KillTask; +import org.apache.druid.indexing.common.task.KillUnusedSegmentsTask; import org.apache.druid.indexing.common.task.NoopTestTaskFileWriter; import org.apache.druid.indexing.common.task.RealtimeIndexTask; import org.apache.druid.indexing.common.task.Task; @@ -856,7 +856,7 @@ public DataSegment apply(String input) segmentFiles.add(file); } - final Task killTask = new KillTask(null, "test_kill_task", Intervals.of("2011-04-01/P4D"), null); + final Task killTask = new KillUnusedSegmentsTask(null, "test_kill_task", Intervals.of("2011-04-01/P4D"), null); final TaskStatus status = runTask(killTask); Assert.assertEquals("merged statusCode", TaskState.SUCCESS, status.getStatusCode()); diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactTaskQuery.java similarity index 89% rename from server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientCompactTaskQuery.java index c87ff90d136a..e5113040ce06 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactTaskQuery.java @@ -27,7 +27,11 @@ import java.util.List; import java.util.Map; -public class ClientCompactQuery implements ClientQuery +/** + * Client representation of {@link org.apache.druid.indexing.common.task.CompactionTask}. JSON serialization fields of + * this class must correspond to those of {@link org.apache.druid.indexing.common.task.CompactionTask}. + */ +public class ClientCompactTaskQuery implements ClientTaskQuery { private final String dataSource; private final List segments; @@ -38,7 +42,7 @@ public class ClientCompactQuery implements ClientQuery private final Map context; @JsonCreator - public ClientCompactQuery( + public ClientCompactTaskQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, @JsonProperty("keepSegmentGranularity") boolean keepSegmentGranularity, diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientKillUnusedSegmentsTaskQuery.java similarity index 79% rename from server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientKillUnusedSegmentsTaskQuery.java index 06d88f9535a3..3d184bc2fde4 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientKillUnusedSegmentsTaskQuery.java @@ -24,14 +24,17 @@ import org.joda.time.Interval; /** + * Client representation of {@link org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}. JSON searialization + * fields of this class must correspond to those of {@link + * org.apache.druid.indexing.common.task.KillUnusedSegmentsTask}, except for "id" and "context" fields. */ -public class ClientKillQuery implements ClientQuery +public class ClientKillUnusedSegmentsTaskQuery implements ClientTaskQuery { private final String dataSource; private final Interval interval; @JsonCreator - public ClientKillQuery( + public ClientKillUnusedSegmentsTaskQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java similarity index 60% rename from server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java index aaa8b5c3ce3a..9d55736bf99a 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java @@ -24,14 +24,19 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; /** - * org.apache.druid.indexing.common.task.Task representation for clients + * {@link org.apache.druid.indexing.common.task.Task} representations for clients. The magic conversion happens right + * at the moment of making a REST query: {@link HttpIndexingServiceClient#runTask} serializes ClientTaskQuery + * objects and {@link org.apache.druid.indexing.overlord.http.OverlordResource#taskPost} deserializes {@link + * org.apache.druid.indexing.common.task.Task} objects from the same bytes. Therefore JSON serialization fields of + * ClientTaskQuery objects must match with those of the corresponding {@link + * org.apache.druid.indexing.common.task.Task} objects. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { - @Type(name = "kill", value = ClientKillQuery.class), - @Type(name = "compact", value = ClientCompactQuery.class) + @Type(name = "kill", value = ClientKillUnusedSegmentsTaskQuery.class), + @Type(name = "compact", value = ClientCompactTaskQuery.class) }) -public interface ClientQuery +public interface ClientTaskQuery { String getType(); diff --git a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java index ac4349b2b6df..9a92c3d9857c 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java @@ -60,9 +60,9 @@ public HttpIndexingServiceClient( } @Override - public void killSegments(String dataSource, Interval interval) + public void killUnusedSegments(String dataSource, Interval interval) { - runTask(new ClientKillQuery(dataSource, interval)); + runTask(new ClientKillUnusedSegmentsTaskQuery(dataSource, interval)); } @Override @@ -87,7 +87,7 @@ public String compactSegments( context.put("priority", compactionTaskPriority); return runTask( - new ClientCompactQuery( + new ClientCompactTaskQuery( dataSource, segments, keepSegmentGranularity, @@ -102,11 +102,12 @@ public String compactSegments( public String runTask(Object taskObject) { try { + // Warning, magic: here we may serialize ClientTaskQuery objects, but OverlordResource.taskPost() deserializes Task + // objects from the same data. See the comment for ClientTaskQuery for details. final FullResponseHolder response = druidLeaderClient.go( - druidLeaderClient.makeRequest( - HttpMethod.POST, - "/druid/indexer/v1/task" - ).setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject)) + druidLeaderClient + .makeRequest(HttpMethod.POST, "/druid/indexer/v1/task") + .setContent(MediaType.APPLICATION_JSON, jsonMapper.writeValueAsBytes(taskObject)) ); if (!response.getStatus().equals(HttpResponseStatus.OK)) { @@ -126,7 +127,7 @@ public String runTask(Object taskObject) } @Override - public String killTask(String taskId) + public String cancelTask(String taskId) { try { final FullResponseHolder response = druidLeaderClient.go( @@ -140,22 +141,22 @@ public String killTask(String taskId) ); if (!response.getStatus().equals(HttpResponseStatus.OK)) { - throw new ISE("Failed to kill task[%s]", taskId); + throw new ISE("Failed to cancel task[%s]", taskId); } final Map resultMap = jsonMapper.readValue( response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ); - final String killedTaskId = (String) resultMap.get("task"); - Preconditions.checkNotNull(killedTaskId, "Null task id returned for task[%s]", taskId); + final String cancelledTaskId = (String) resultMap.get("task"); + Preconditions.checkNotNull(cancelledTaskId, "Null task id returned for task[%s]", taskId); Preconditions.checkState( - taskId.equals(killedTaskId), - "Requested to kill task[%s], but another task[%s] was killed!", + taskId.equals(cancelledTaskId), + "Requested to cancel task[%s], but another task[%s] was cancelled!", taskId, - killedTaskId + cancelledTaskId ); - return killedTaskId; + return cancelledTaskId; } catch (Exception e) { throw new RuntimeException(e); diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java index df2baf6b4f3a..3549a4b4985f 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java @@ -30,7 +30,7 @@ public interface IndexingServiceClient { - void killSegments(String dataSource, Interval interval); + void killUnusedSegments(String dataSource, Interval interval); int killPendingSegments(String dataSource, DateTime end); @@ -47,7 +47,7 @@ String compactSegments( String runTask(Object taskObject); - String killTask(String taskId); + String cancelTask(String taskId); List getRunningTasks(); diff --git a/server/src/main/java/org/apache/druid/client/indexing/TaskPayloadResponse.java b/server/src/main/java/org/apache/druid/client/indexing/TaskPayloadResponse.java index 1b938af32626..45249593aa2f 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/TaskPayloadResponse.java +++ b/server/src/main/java/org/apache/druid/client/indexing/TaskPayloadResponse.java @@ -25,12 +25,12 @@ public class TaskPayloadResponse { private final String task; - private final ClientQuery payload; + private final ClientTaskQuery payload; @JsonCreator public TaskPayloadResponse( @JsonProperty("task") final String task, - @JsonProperty("payload") final ClientQuery payload + @JsonProperty("payload") final ClientTaskQuery payload ) { this.task = task; @@ -44,7 +44,7 @@ public String getTask() } @JsonProperty - public ClientQuery getPayload() + public ClientTaskQuery getPayload() { return payload; } diff --git a/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java b/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java index 752791938a70..3dfdb7c71e8f 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java @@ -595,7 +595,8 @@ public List inTransaction(Handle handle, TransactionStatus status) Iterator iter = handle .createQuery( StringUtils.format( - "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource and start >= :start and %2$send%2$s <= :end and used = false ORDER BY start, %2$send%2$s", + "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource AND start >= :start " + + "AND %2$send%2$s <= :end AND used = false ORDER BY start, %2$send%2$s", getSegmentsTable(), connector.getQuoteString() ) 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 c0ae5fdc2e89..b4c3177eefbf 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 @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.common.config.JacksonConfigManager; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorUnusedSegmentsKiller; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -48,7 +49,7 @@ public class CoordinatorDynamicConfig { public static final String CONFIG_KEY = "coordinator.config"; - private final long millisToWaitBeforeDeleting; + private final long millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments; private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; @@ -56,13 +57,13 @@ public class CoordinatorDynamicConfig private final int replicationThrottleLimit; private final int balancerComputeThreads; private final boolean emitBalancingStats; - private final boolean killAllDataSources; - private final Set specificDataSourcesToKill; + private final boolean killUnusedSegmentsInAllDataSources; + private final Set specificDataSourcesToKillUnusedSegmentsIn; private final Set decommissioningNodes; private final int decommissioningMaxPercentOfMaxSegmentsToMove; - // The pending segments of the dataSources in this list are not killed. - private final Set protectedPendingSegmentDatasources; + /** Stale pending segments belonging to the data sources in this list are not killed (they are "protected"). */ + private final Set dataSourcesToNotKillStalePendingSegmentsIn; /** * The maximum number of segments that could be queued for loading to any given server. @@ -74,7 +75,10 @@ public class CoordinatorDynamicConfig @JsonCreator public CoordinatorDynamicConfig( - @JsonProperty("millisToWaitBeforeDeleting") long millisToWaitBeforeDeleting, + // Keeping the legacy 'millisToWaitBeforeDeleting' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("millisToWaitBeforeDeleting") + long millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments, @JsonProperty("mergeBytesLimit") long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") int maxSegmentsToMove, @@ -82,18 +86,26 @@ public CoordinatorDynamicConfig( @JsonProperty("replicationThrottleLimit") int replicationThrottleLimit, @JsonProperty("balancerComputeThreads") int balancerComputeThreads, @JsonProperty("emitBalancingStats") boolean emitBalancingStats, - // Type is Object here so that we can support both string and list as - // coordinator console can not send array of strings in the update request. - // See https://github.com/apache/incubator-druid/issues/3055 - @JsonProperty("killDataSourceWhitelist") Object specificDataSourcesToKill, - @JsonProperty("killAllDataSources") boolean killAllDataSources, - @JsonProperty("killPendingSegmentsSkipList") Object protectedPendingSegmentDatasources, + // Type is Object here so that we can support both string and list as Coordinator console can not send array of + // strings in the update request. See https://github.com/apache/incubator-druid/issues/3055. + // Keeping the legacy 'killDataSourceWhitelist' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("killDataSourceWhitelist") Object specificDataSourcesToKillUnusedSegmentsIn, + // Keeping the legacy 'killAllDataSources' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("killAllDataSources") boolean killUnusedSegmentsInAllDataSources, + // Type is Object here so that we can support both string and list as Coordinator console can not send array of + // strings in the update request, as well as for specificDataSourcesToKillUnusedSegmentsIn. + // Keeping the legacy 'killPendingSegmentsSkipList' property name for backward compatibility. When the project is + // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152 + @JsonProperty("killPendingSegmentsSkipList") Object dataSourcesToNotKillStalePendingSegmentsIn, @JsonProperty("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue, @JsonProperty("decommissioningNodes") Object decommissioningNodes, @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") int decommissioningMaxPercentOfMaxSegmentsToMove ) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments = + millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -101,9 +113,10 @@ public CoordinatorDynamicConfig( this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; - this.killAllDataSources = killAllDataSources; - this.specificDataSourcesToKill = parseJsonStringOrArray(specificDataSourcesToKill); - this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources); + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; + this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn); + this.dataSourcesToNotKillStalePendingSegmentsIn = + parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes); Preconditions.checkArgument( @@ -112,8 +125,10 @@ public CoordinatorDynamicConfig( ); this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; - if (this.killAllDataSources && !this.specificDataSourcesToKill.isEmpty()) { - throw new IAE("can't have killAllDataSources and non-empty specificDataSourcesToKill"); + if (this.killUnusedSegmentsInAllDataSources && !this.specificDataSourcesToKillUnusedSegmentsIn.isEmpty()) { + throw new IAE( + "can't have killUnusedSegmentsInAllDataSources and non-empty specificDataSourcesToKillUnusedSegmentsIn" + ); } } @@ -151,10 +166,10 @@ public static CoordinatorDynamicConfig current(final JacksonConfigManager config return Preconditions.checkNotNull(watch(configManager).get(), "Got null config from watcher?!"); } - @JsonProperty - public long getMillisToWaitBeforeDeleting() + @JsonProperty("millisToWaitBeforeDeleting") + public long getMillisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments() { - return millisToWaitBeforeDeleting; + return millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments; } @JsonProperty @@ -201,28 +216,28 @@ public int getBalancerComputeThreads() /** * List of specific data sources for which kill tasks are sent in - * {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. + * {@link DruidCoordinatorUnusedSegmentsKiller}. */ @JsonProperty("killDataSourceWhitelist") - public Set getSpecificDataSourcesToKill() + public Set getSpecificDataSourcesToKillUnusedSegmentsIn() { - return specificDataSourcesToKill; + return specificDataSourcesToKillUnusedSegmentsIn; } @JsonProperty - public boolean isKillAllDataSources() + public boolean isKillUnusedSegmentsInAllDataSources() { - return killAllDataSources; + return killUnusedSegmentsInAllDataSources; } /** * List of dataSources for which pendingSegments are NOT cleaned up - * in {@link DruidCoordinatorCleanupPendingSegments}. + * in {@link DruidCoordinatorKillStalePendingSegments}. */ - @JsonProperty - public Set getProtectedPendingSegmentDatasources() + @JsonProperty("killPendingSegmentsSkipList") + public Set getDataSourcesToNotKillStalePendingSegmentsIn() { - return protectedPendingSegmentDatasources; + return dataSourcesToNotKillStalePendingSegmentsIn; } @JsonProperty @@ -232,9 +247,9 @@ public int getMaxSegmentsInNodeLoadingQueue() } /** - * 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 - * {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove}. + * 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 {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove}. * * @return list of host:port entries */ @@ -269,7 +284,8 @@ public int getDecommissioningMaxPercentOfMaxSegmentsToMove() public String toString() { return "CoordinatorDynamicConfig{" + - "millisToWaitBeforeDeleting=" + millisToWaitBeforeDeleting + + "millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments=" + + millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments + ", mergeBytesLimit=" + mergeBytesLimit + ", mergeSegmentsLimit=" + mergeSegmentsLimit + ", maxSegmentsToMove=" + maxSegmentsToMove + @@ -277,9 +293,9 @@ public String toString() ", replicationThrottleLimit=" + replicationThrottleLimit + ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + - ", killAllDataSources=" + killAllDataSources + - ", specificDataSourcesToKill=" + specificDataSourcesToKill + - ", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + + ", killUnusedSegmentsInAllDataSources=" + killUnusedSegmentsInAllDataSources + + ", specificDataSourcesToKillUnusedSegmentsIn=" + specificDataSourcesToKillUnusedSegmentsIn + + ", dataSourcesToNotKillStalePendingSegmentsIn=" + dataSourcesToNotKillStalePendingSegmentsIn + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + ", decommissioningNodes=" + decommissioningNodes + ", decommissioningMaxPercentOfMaxSegmentsToMove=" + decommissioningMaxPercentOfMaxSegmentsToMove + @@ -298,7 +314,8 @@ public boolean equals(Object o) CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o; - if (millisToWaitBeforeDeleting != that.millisToWaitBeforeDeleting) { + if (millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments != + that.millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments) { return false; } if (mergeBytesLimit != that.mergeBytesLimit) { @@ -322,16 +339,16 @@ public boolean equals(Object o) if (emitBalancingStats != that.emitBalancingStats) { return false; } - if (killAllDataSources != that.killAllDataSources) { + if (killUnusedSegmentsInAllDataSources != that.killUnusedSegmentsInAllDataSources) { return false; } if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { return false; } - if (!Objects.equals(specificDataSourcesToKill, that.specificDataSourcesToKill)) { + if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) { return false; } - if (!Objects.equals(protectedPendingSegmentDatasources, that.protectedPendingSegmentDatasources)) { + if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) { return false; } if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) { @@ -344,7 +361,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( - millisToWaitBeforeDeleting, + millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, @@ -352,10 +369,10 @@ public int hashCode() replicationThrottleLimit, balancerComputeThreads, emitBalancingStats, - killAllDataSources, + killUnusedSegmentsInAllDataSources, maxSegmentsInNodeLoadingQueue, - specificDataSourcesToKill, - protectedPendingSegmentDatasources, + specificDataSourcesToKillUnusedSegmentsIn, + dataSourcesToNotKillStalePendingSegmentsIn, decommissioningNodes, decommissioningMaxPercentOfMaxSegmentsToMove ); @@ -368,19 +385,21 @@ public static Builder builder() public static class Builder { - private static final long DEFAULT_MILLIS_TO_WAIT_BEFORE_DELETING = TimeUnit.MINUTES.toMillis(15); - private static final long DEFAULT_MERGE_BYTES_LIMIT = 524288000L; + private static final + long DEFAULT_MILLIS_LAG_SINCE_COORDINATOR_BECOMES_LEADER_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS + = TimeUnit.MINUTES.toMillis(15); + private static final long DEFAULT_MERGE_BYTES_LIMIT = 524_288_000L; private static final int DEFAULT_MERGE_SEGMENTS_LIMIT = 100; private static final int DEFAULT_MAX_SEGMENTS_TO_MOVE = 5; private static final int DEFAULT_REPLICANT_LIFETIME = 15; private static final int DEFAULT_REPLICATION_THROTTLE_LIMIT = 10; private static final int DEFAULT_BALANCER_COMPUTE_THREADS = 1; private static final boolean DEFAULT_EMIT_BALANCING_STATS = false; - private static final boolean DEFAULT_KILL_ALL_DATA_SOURCES = false; + private static final boolean DEFAULT_KILL_UNUSED_SEGMENTS_IN_ALL_DATA_SOURCES = false; private static final int DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 0; private static final int DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70; - private Long millisToWaitBeforeDeleting; + private Long millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments; private Long mergeBytesLimit; private Integer mergeSegmentsLimit; private Integer maxSegmentsToMove; @@ -388,9 +407,9 @@ public static class Builder private Integer replicationThrottleLimit; private Boolean emitBalancingStats; private Integer balancerComputeThreads; - private Object specificDataSourcesToKill; - private Boolean killAllDataSources; - private Object killPendingSegmentsSkipList; + private Object specificDataSourcesToKillUnusedSegmentsIn; + private Boolean killUnusedSegmentsInAllDataSources; + private Object dataSourcesToNotKillStalePendingSegmentsIn; private Integer maxSegmentsInNodeLoadingQueue; private Object decommissioningNodes; private Integer decommissioningMaxPercentOfMaxSegmentsToMove; @@ -401,7 +420,8 @@ public Builder() @JsonCreator public Builder( - @JsonProperty("millisToWaitBeforeDeleting") @Nullable Long millisToWaitBeforeDeleting, + @JsonProperty("millisToWaitBeforeDeleting") + @Nullable Long millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments, @JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove, @@ -409,15 +429,17 @@ public Builder( @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, - @JsonProperty("killDataSourceWhitelist") @Nullable Object specificDataSourcesToKill, - @JsonProperty("killAllDataSources") @Nullable Boolean killAllDataSources, - @JsonProperty("killPendingSegmentsSkipList") @Nullable Object killPendingSegmentsSkipList, + @JsonProperty("killDataSourceWhitelist") @Nullable Object specificDataSourcesToKillUnusedSegmentsIn, + @JsonProperty("killAllDataSources") @Nullable Boolean killUnusedSegmentsInAllDataSources, + @JsonProperty("killPendingSegmentsSkipList") @Nullable Object dataSourcesToNotKillStalePendingSegmentsIn, @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue, @JsonProperty("decommissioningNodes") @Nullable Object decommissioningNodes, - @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") @Nullable Integer decommissioningMaxPercentOfMaxSegmentsToMove + @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") + @Nullable Integer decommissioningMaxPercentOfMaxSegmentsToMove ) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments = + millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -425,17 +447,17 @@ public Builder( this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = balancerComputeThreads; this.emitBalancingStats = emitBalancingStats; - this.killAllDataSources = killAllDataSources; - this.specificDataSourcesToKill = specificDataSourcesToKill; - this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; + this.specificDataSourcesToKillUnusedSegmentsIn = specificDataSourcesToKillUnusedSegmentsIn; + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; + this.dataSourcesToNotKillStalePendingSegmentsIn = dataSourcesToNotKillStalePendingSegmentsIn; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = decommissioningNodes; this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; } - public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting) + public Builder withMillisLagSinceCoordinatorStartBeforeCanMarkAsUnusedOvershadowedSegments(long millisLag) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments = millisLag; return this; } @@ -481,15 +503,15 @@ public Builder withEmitBalancingStats(boolean emitBalancingStats) return this; } - public Builder withSpecificDataSourcesToKill(Set specificDataSourcesToKill) + public Builder withSpecificDataSourcesToKillUnusedSegmentsIn(Set dataSources) { - this.specificDataSourcesToKill = specificDataSourcesToKill; + this.specificDataSourcesToKillUnusedSegmentsIn = dataSources; return this; } - public Builder withKillAllDataSources(boolean killAllDataSources) + public Builder withKillUnusedSegmentsInAllDataSources(boolean killUnusedSegmentsInAllDataSources) { - this.killAllDataSources = killAllDataSources; + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; return this; } @@ -514,7 +536,9 @@ public Builder withDecommissioningMaxPercentOfMaxSegmentsToMove(Integer percent) public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( - millisToWaitBeforeDeleting == null ? DEFAULT_MILLIS_TO_WAIT_BEFORE_DELETING : millisToWaitBeforeDeleting, + millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments == null + ? DEFAULT_MILLIS_LAG_SINCE_COORDINATOR_BECOMES_LEADER_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS + : millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit, mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit, maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove, @@ -522,9 +546,11 @@ public CoordinatorDynamicConfig build() replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit, balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, - specificDataSourcesToKill, - killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, - killPendingSegmentsSkipList, + specificDataSourcesToKillUnusedSegmentsIn, + killUnusedSegmentsInAllDataSources == null + ? DEFAULT_KILL_UNUSED_SEGMENTS_IN_ALL_DATA_SOURCES + : killUnusedSegmentsInAllDataSources, + dataSourcesToNotKillStalePendingSegmentsIn, maxSegmentsInNodeLoadingQueue == null ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE : maxSegmentsInNodeLoadingQueue, @@ -538,7 +564,9 @@ public CoordinatorDynamicConfig build() public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) { return new CoordinatorDynamicConfig( - millisToWaitBeforeDeleting == null ? defaults.getMillisToWaitBeforeDeleting() : millisToWaitBeforeDeleting, + millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments == null + ? defaults.getMillisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments() + : millisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit, mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit, maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove, @@ -546,11 +574,15 @@ public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit, balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, - specificDataSourcesToKill == null ? defaults.getSpecificDataSourcesToKill() : specificDataSourcesToKill, - killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, - killPendingSegmentsSkipList == null - ? defaults.getProtectedPendingSegmentDatasources() - : killPendingSegmentsSkipList, + specificDataSourcesToKillUnusedSegmentsIn == null + ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn() + : specificDataSourcesToKillUnusedSegmentsIn, + killUnusedSegmentsInAllDataSources == null + ? defaults.isKillUnusedSegmentsInAllDataSources() + : killUnusedSegmentsInAllDataSources, + dataSourcesToNotKillStalePendingSegmentsIn == null + ? defaults.getDataSourcesToNotKillStalePendingSegmentsIn() + : dataSourcesToNotKillStalePendingSegmentsIn, maxSegmentsInNodeLoadingQueue == null ? defaults.getMaxSegmentsInNodeLoadingQueue() : maxSegmentsInNodeLoadingQueue, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index 97d2af0376ff..d03be349b898 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -61,8 +61,8 @@ import org.apache.druid.metadata.MetadataSegments; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupOvershadowed; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorCleanupUnusedSegments; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorMarkAsUnusedOvershadowedSegments; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorUnloadUnusedSegments; import org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper; import org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger; import org.apache.druid.server.coordinator.helper.DruidCoordinatorRuleRunner; @@ -606,7 +606,7 @@ private List makeIndexingServiceHelpers() public abstract class CoordinatorRunnable implements Runnable { - private final long startTime = System.currentTimeMillis(); + private final long startTimeNanos = System.nanoTime(); private final List helpers; private final int startingLeaderCounter; @@ -651,7 +651,7 @@ public void run() DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams .newBuilder() - .withStartTime(startTime) + .withStartTimeNanos(startTimeNanos) .withDataSourcesWithUsedSegments(metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) .withDynamicConfigs(getDynamicConfigs()) .withCompactionConfig(getCompactionConfig()) @@ -746,8 +746,8 @@ public CoordinatorHistoricalManagerRunnable(final int startingLeaderCounter) .build(); }, new DruidCoordinatorRuleRunner(DruidCoordinator.this), - new DruidCoordinatorCleanupUnusedSegments(), - new DruidCoordinatorCleanupOvershadowed(DruidCoordinator.this), + new DruidCoordinatorUnloadUnusedSegments(), + new DruidCoordinatorMarkAsUnusedOvershadowedSegments(DruidCoordinator.this), new DruidCoordinatorBalancer(DruidCoordinator.this), new DruidCoordinatorLogger(DruidCoordinator.this) ), diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorKillStalePendingSegments.java similarity index 83% rename from server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java rename to server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorKillStalePendingSegments.java index 34ed7485b133..3014f14e36b2 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorKillStalePendingSegments.java @@ -33,15 +33,15 @@ import java.util.ArrayList; import java.util.List; -public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorHelper +public class DruidCoordinatorKillStalePendingSegments implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidCoordinatorCleanupPendingSegments.class); + private static final Logger log = new Logger(DruidCoordinatorKillStalePendingSegments.class); private static final Period KEEP_PENDING_SEGMENTS_OFFSET = new Period("P1D"); private final IndexingServiceClient indexingServiceClient; @Inject - public DruidCoordinatorCleanupPendingSegments(IndexingServiceClient indexingServiceClient) + public DruidCoordinatorKillStalePendingSegments(IndexingServiceClient indexingServiceClient) { this.indexingServiceClient = indexingServiceClient; } @@ -85,15 +85,15 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // is no running/pending/waiting tasks. Preconditions.checkState(!createdTimes.isEmpty(), "Failed to gather createdTimes of tasks"); - // If there is no running/pending/waiting/complete tasks, pendingSegmentsCleanupEndTime is + // If there is no running/pending/waiting/complete tasks, stalePendingSegmentsCutoffCreationTime is // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET). - final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); + final DateTime stalePendingSegmentsCutoffCreationTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); for (String dataSource : params.getDataSourcesWithUsedSegments().keySet()) { - if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) { + if (!params.getCoordinatorDynamicConfig().getDataSourcesToNotKillStalePendingSegmentsIn().contains(dataSource)) { log.info( "Killed [%d] pendingSegments created until [%s] for dataSource[%s]", - indexingServiceClient.killPendingSegments(dataSource, pendingSegmentsCleanupEndTime), - pendingSegmentsCleanupEndTime, + indexingServiceClient.killPendingSegments(dataSource, stalePendingSegmentsCutoffCreationTime), + stalePendingSegmentsCutoffCreationTime, dataSource ); } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java index e9f91ed212dd..6799b007f32f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java @@ -35,6 +35,7 @@ import java.util.HashMap; import java.util.Map; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; /** */ @@ -55,7 +56,7 @@ public static TreeSet createUsedSegmentsSet(Iterable u return segmentsSet; } - private final long startTime; + private final long startTimeNanos; private final DruidCluster druidCluster; private final MetadataRuleManager databaseRuleManager; private final SegmentReplicantLookup segmentReplicantLookup; @@ -72,7 +73,7 @@ public static TreeSet createUsedSegmentsSet(Iterable u private final BalancerStrategy balancerStrategy; private DruidCoordinatorRuntimeParams( - long startTime, + long startTimeNanos, DruidCluster druidCluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, @@ -88,7 +89,7 @@ private DruidCoordinatorRuntimeParams( BalancerStrategy balancerStrategy ) { - this.startTime = startTime; + this.startTimeNanos = startTimeNanos; this.druidCluster = druidCluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; @@ -104,9 +105,9 @@ private DruidCoordinatorRuntimeParams( this.balancerStrategy = balancerStrategy; } - public long getStartTime() + public long getStartTimeNanos() { - return startTime; + return startTimeNanos; } public DruidCluster getDruidCluster() @@ -178,9 +179,13 @@ public BalancerStrategy getBalancerStrategy() return balancerStrategy; } - public boolean hasDeletionWaitTimeElapsed() + public boolean lagSinceCoordinatorStartElapsedBeforeCanMarkAsUnusedOvershadowedSegements() { - return (System.currentTimeMillis() - getStartTime() > coordinatorDynamicConfig.getMillisToWaitBeforeDeleting()); + long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); + long lagNanos = TimeUnit.MILLISECONDS.toNanos( + coordinatorDynamicConfig.getMillisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments() + ); + return nanosElapsedSinceCoordinatorStart > lagNanos; } public static Builder newBuilder() @@ -191,7 +196,7 @@ public static Builder newBuilder() public Builder buildFromExisting() { return new Builder( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, @@ -211,7 +216,7 @@ public Builder buildFromExisting() public Builder buildFromExistingWithoutUsedSegments() { return new Builder( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, @@ -230,7 +235,7 @@ public Builder buildFromExistingWithoutUsedSegments() public static class Builder { - private long startTime; + private @Nullable Long startTimeNanos; private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; private SegmentReplicantLookup segmentReplicantLookup; @@ -247,7 +252,7 @@ public static class Builder Builder() { - this.startTime = 0; + this.startTimeNanos = null; this.druidCluster = null; this.databaseRuleManager = null; this.segmentReplicantLookup = null; @@ -263,7 +268,7 @@ public static class Builder } Builder( - long startTime, + long startTimeNanos, DruidCluster cluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, @@ -279,7 +284,7 @@ public static class Builder BalancerStrategy balancerStrategy ) { - this.startTime = startTime; + this.startTimeNanos = startTimeNanos; this.druidCluster = cluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; @@ -297,8 +302,9 @@ public static class Builder public DruidCoordinatorRuntimeParams build() { + Preconditions.checkNotNull(startTimeNanos, "startTime must be set"); return new DruidCoordinatorRuntimeParams( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, @@ -315,9 +321,9 @@ public DruidCoordinatorRuntimeParams build() ); } - public Builder withStartTime(long time) + public Builder withStartTimeNanos(long startTimeNanos) { - startTime = time; + this.startTimeNanos = startTimeNanos; return this; } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index 803d5c067ecd..11e080554e8d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -198,7 +198,7 @@ private Pair balanceServers( // DruidCoordinatorRuntimeParams.getUsedSegments originate from MetadataSegments, i. e. that's a set of segments // that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be // any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such - // discrepancies eventually via DruidCoordinatorCleanupUnusedSegments). Therefore the picked segmentToMoveHolder's + // discrepancies eventually via DruidCoordinatorUnloadUnusedSegments). Therefore the picked segmentToMoveHolder's // segment may not need to be balanced. boolean needToBalancePickedSegment = params.getUsedSegments().contains(segmentToMoveHolder.getSegment()); if (needToBalancePickedSegment) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorMarkAsUnusedOvershadowedSegments.java similarity index 50% rename from server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java rename to server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorMarkAsUnusedOvershadowedSegments.java index 72aad9ea9dc6..2c186fd1c37e 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorMarkAsUnusedOvershadowedSegments.java @@ -34,11 +34,11 @@ import java.util.Map; import java.util.SortedSet; -public class DruidCoordinatorCleanupOvershadowed implements DruidCoordinatorHelper +public class DruidCoordinatorMarkAsUnusedOvershadowedSegments implements DruidCoordinatorHelper { private final DruidCoordinator coordinator; - public DruidCoordinatorCleanupOvershadowed(DruidCoordinator coordinator) + public DruidCoordinatorMarkAsUnusedOvershadowedSegments(DruidCoordinator coordinator) { this.coordinator = coordinator; } @@ -46,41 +46,41 @@ public DruidCoordinatorCleanupOvershadowed(DruidCoordinator coordinator) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - CoordinatorStats stats = new CoordinatorStats(); + // Mark as unused overshadowed segments only if we've had enough time to make sure we aren't flapping with old data. + if (!params.lagSinceCoordinatorStartElapsedBeforeCanMarkAsUnusedOvershadowedSegements()) { + return params; + } - // Delete segments that are old - // Unservice old partitions if we've had enough time to make sure we aren't flapping with old data - if (params.hasDeletionWaitTimeElapsed()) { - DruidCluster cluster = params.getDruidCluster(); - Map> timelines = new HashMap<>(); + CoordinatorStats stats = new CoordinatorStats(); - for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { - for (ServerHolder serverHolder : serverHolders) { - ImmutableDruidServer server = serverHolder.getServer(); + DruidCluster cluster = params.getDruidCluster(); + Map> timelines = new HashMap<>(); - for (ImmutableDruidDataSource dataSource : server.getDataSources()) { - VersionedIntervalTimeline timeline = timelines.get(dataSource.getName()); - if (timeline == null) { - timeline = new VersionedIntervalTimeline<>(Comparator.naturalOrder()); - timelines.put(dataSource.getName(), timeline); - } + for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { + for (ServerHolder serverHolder : serverHolders) { + ImmutableDruidServer server = serverHolder.getServer(); - VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); + for (ImmutableDruidDataSource dataSource : server.getDataSources()) { + VersionedIntervalTimeline timeline = timelines.get(dataSource.getName()); + if (timeline == null) { + timeline = new VersionedIntervalTimeline<>(Comparator.naturalOrder()); + timelines.put(dataSource.getName(), timeline); } + + VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); } } + } - //Remove all segments in db that are overshadowed by served segments - for (DataSegment dataSegment : params.getUsedSegments()) { - VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { - coordinator.tryMarkSegmentAsUnused(dataSegment); - stats.addToGlobalStat("overShadowedCount", 1); - } + // Mark all segments as unused in db that are overshadowed by served segments + for (DataSegment dataSegment : params.getUsedSegments()) { + VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); + if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { + coordinator.tryMarkSegmentAsUnused(dataSegment); + stats.addToGlobalStat("overShadowedCount", 1); } } - return params.buildFromExisting() - .withCoordinatorStats(stats) - .build(); + + return params.buildFromExisting().withCoordinatorStats(stats).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java index b0212dddc04f..4e748dae91b6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java @@ -47,7 +47,7 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper { private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class); - private static int MAX_MISSING_RULES = 10; + private static final int MAX_MISSING_RULES = 10; private final ReplicationThrottler replicatorThrottler; @@ -86,8 +86,10 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params; } - // Find used segments which are overshadowed by other used segments. Those would not need to be loaded and dropped. - // Segments overshadowed by *served* used segments is dropped automatically by DruidCoordinatorCleanupOvershadowed. + // Find used segments which are overshadowed by other used segments. Those would not need to be loaded and + // eventually will be unloaded from Historical servers. Segments overshadowed by *served* used segments are marked + // as unused in DruidCoordinatorMarkAsUnusedOvershadowedSegments, and then eventually Coordinator sends commands to + // Historical nodes to unload such segments in DruidCoordinatorUnloadUnusedSegments. Set overshadowed = determineOvershadowedSegments(params); for (String tier : cluster.getTierNames()) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java index b1ac299d2162..d48a594b471c 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -22,7 +22,7 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import it.unimi.dsi.fastutil.objects.Object2LongMap; -import org.apache.druid.client.indexing.ClientCompactQuery; +import org.apache.druid.client.indexing.ClientCompactTaskQuery; import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.TaskPayloadResponse; @@ -97,7 +97,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) throw new ISE("WTH? got a null paylord from overlord for task[%s]", status.getId()); } if (COMPACT_TASK_TYPE.equals(response.getPayload().getType())) { - final ClientCompactQuery compactQuery = (ClientCompactQuery) response.getPayload(); + final ClientCompactTaskQuery compactQuery = (ClientCompactTaskQuery) response.getPayload(); final Interval interval = JodaUtils.umbrellaInterval( compactQuery.getSegments() .stream() diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java similarity index 85% rename from server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnusedSegments.java rename to server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java index 2086ec17f84a..846a8f87d070 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java @@ -34,9 +34,9 @@ /** */ -public class DruidCoordinatorCleanupUnusedSegments implements DruidCoordinatorHelper +public class DruidCoordinatorUnloadUnusedSegments implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidCoordinatorCleanupUnusedSegments.class); + private static final Logger log = new Logger(DruidCoordinatorUnloadUnusedSegments.class); @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) @@ -54,9 +54,10 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) return params.buildFromExisting().withCoordinatorStats(stats).build(); } - // Drop segments that are no longer marked as used, *if* the usedSegments collection has been populated. Used - // segments might not have been loaded yet since it's done asynchronously (in SqlMetadataSegments). But it's also - // done atomically (see SqlMetadataSegments code), so if there are any segments at all, we should have all of them. + // Unload segments that are no longer marked as used from historical servers, *if* the usedSegments collection has + // been populated. Used segments might not have been loaded yet since it's done asynchronously (in + // SqlMetadataSegments). But it's also done atomically (see SqlMetadataSegments code), so if there are any segments + // at all, we should have all of them. for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { ImmutableDruidServer server = serverHolder.getServer(); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java similarity index 91% rename from server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java rename to server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java index 33d2f0030fab..87fef359caf6 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java @@ -31,14 +31,15 @@ import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collection; import java.util.List; /** */ -public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper +public class DruidCoordinatorUnusedSegmentsKiller implements DruidCoordinatorHelper { - private static final Logger log = new Logger(DruidCoordinatorSegmentKiller.class); + private static final Logger log = new Logger(DruidCoordinatorUnusedSegmentsKiller.class); private final long period; private final long retainDuration; @@ -50,7 +51,7 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper private final IndexingServiceClient indexingServiceClient; @Inject - public DruidCoordinatorSegmentKiller( + public DruidCoordinatorUnusedSegmentsKiller( MetadataSegments metadataSegments, IndexingServiceClient indexingServiceClient, DruidCoordinatorConfig config @@ -82,8 +83,8 @@ public DruidCoordinatorSegmentKiller( @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources(); - Collection specificDataSourcesToKill = params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKill(); + boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources(); + Collection specificDataSourcesToKill = params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn(); if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) { log.error( @@ -106,7 +107,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill); if (intervalToKill != null) { try { - indexingServiceClient.killSegments(dataSource, intervalToKill); + indexingServiceClient.killUnusedSegments(dataSource, intervalToKill); } catch (Exception ex) { log.error(ex, "Failed to submit kill task for dataSource [%s]", dataSource); @@ -122,6 +123,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } @VisibleForTesting + @Nullable Interval findIntervalForKillTask(String dataSource, int limit) { List unusedSegmentIntervals = metadataSegments.getUnusedSegmentIntervals( diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index fc4096108e68..ba28383f06bf 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -205,7 +205,7 @@ public Response tryMarkAsUnusedAllSegmentsOrKillSegmentsInInterval( boolean killSegments = kill != null && Boolean.valueOf(kill); if (killSegments) { try { - indexingServiceClient.killSegments(dataSourceName, Intervals.of(interval)); + indexingServiceClient.killUnusedSegments(dataSourceName, Intervals.of(interval)); } catch (IllegalArgumentException e) { return Response.status(Response.Status.BAD_REQUEST) @@ -253,7 +253,7 @@ public Response killSegmentsInInterval( } final Interval theInterval = Intervals.of(interval.replace('_', '/')); try { - indexingServiceClient.killSegments(dataSourceName, theInterval); + indexingServiceClient.killUnusedSegments(dataSourceName, theInterval); } catch (Exception e) { return Response.serverError() diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientKillUnusedSegmentsQueryTest.java similarity index 75% rename from server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java rename to server/src/test/java/org/apache/druid/client/indexing/ClientKillUnusedSegmentsQueryTest.java index ebd3b6f4438e..53d77aa12646 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientKillUnusedSegmentsQueryTest.java @@ -27,40 +27,40 @@ import org.junit.Before; import org.junit.Test; -public class ClientKillQueryTest +public class ClientKillUnusedSegmentsQueryTest { private static final String DATA_SOURCE = "data_source"; public static final DateTime START = DateTimes.nowUtc(); private static final Interval INTERVAL = new Interval(START, START.plus(1)); - ClientKillQuery clientKillQuery; + ClientKillUnusedSegmentsTaskQuery clientKillUnusedSegmentsQuery; @Before public void setUp() { - clientKillQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL); + clientKillUnusedSegmentsQuery = new ClientKillUnusedSegmentsTaskQuery(DATA_SOURCE, INTERVAL); } @After public void tearDown() { - clientKillQuery = null; + clientKillUnusedSegmentsQuery = null; } @Test public void testGetType() { - Assert.assertEquals("kill", clientKillQuery.getType()); + Assert.assertEquals("kill", clientKillUnusedSegmentsQuery.getType()); } @Test public void testGetDataSource() { - Assert.assertEquals(DATA_SOURCE, clientKillQuery.getDataSource()); + Assert.assertEquals(DATA_SOURCE, clientKillUnusedSegmentsQuery.getDataSource()); } @Test public void testGetInterval() { - Assert.assertEquals(INTERVAL, clientKillQuery.getInterval()); + Assert.assertEquals(INTERVAL, clientKillUnusedSegmentsQuery.getInterval()); } } diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java index 78a91648d785..7bd1e214dd04 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java @@ -31,7 +31,7 @@ public class NoopIndexingServiceClient implements IndexingServiceClient { @Override - public void killSegments(String dataSource, Interval interval) + public void killUnusedSegments(String dataSource, Interval interval) { } @@ -68,7 +68,7 @@ public String runTask(Object taskObject) } @Override - public String killTask(String taskId) + public String cancelTask(String taskId) { return null; } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java index 5a31c1c3d79e..f8267d8530b1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java @@ -61,6 +61,10 @@ */ public class DruidCoordinatorRuleRunnerTest { + public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = CoordinatorDynamicConfig + .builder() + .withMillisLagSinceCoordinatorStartBeforeCanMarkAsUnusedOvershadowedSegments(0L) + .build(); private DruidCoordinator coordinator; private LoadQueuePeon mockPeon; private List usedSegments; @@ -608,7 +612,7 @@ public void testDropRemove() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -694,7 +698,7 @@ public void testDropTooManyInSameTier() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisLagSinceCoordinatorStartBeforeCanMarkAsUnusedOvershadowedSegments(0L).build()) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -785,7 +789,7 @@ public void testDropTooManyInDifferentTiers() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -864,7 +868,7 @@ public void testDontDropInDifferentTiers() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -962,7 +966,7 @@ public void testDropServerActuallyServesSegment() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -1285,7 +1289,7 @@ public void testDropReplicantThrottle() DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LAG_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withUsedSegmentsInTest(longerUsedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) @@ -1413,7 +1417,7 @@ private void mockEmptyPeon() private CoordinatorDynamicConfig createCoordinatorDynamicConfig() { return CoordinatorDynamicConfig.builder() - .withMillisToWaitBeforeDeleting(0) + .withMillisLagSinceCoordinatorStartBeforeCanMarkAsUnusedOvershadowedSegments(0) .withMergeBytesLimit(0) .withMergeSegmentsLimit(0) .withMaxSegmentsToMove(0) @@ -1421,8 +1425,8 @@ private CoordinatorDynamicConfig createCoordinatorDynamicConfig() .withReplicationThrottleLimit(24) .withBalancerComputeThreads(0) .withEmitBalancingStats(false) - .withSpecificDataSourcesToKill(null) - .withKillAllDataSources(false) + .withSpecificDataSourcesToKillUnusedSegmentsIn(null) + .withKillUnusedSegmentsInAllDataSources(false) .withMaxSegmentsInNodeLoadingQueue(1000) .build(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorMarkAsUnusedOvershadowedSegmentsTest.java similarity index 93% rename from server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorMarkAsUnusedOvershadowedSegmentsTest.java index 4a98e6bf88f6..abd3e32d1a3b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorMarkAsUnusedOvershadowedSegmentsTest.java @@ -44,9 +44,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -public class DruidCoordinatorCleanupOvershadowedTest +public class DruidCoordinatorMarkAsUnusedOvershadowedSegmentsTest { - DruidCoordinatorCleanupOvershadowed druidCoordinatorCleanupOvershadowed; + DruidCoordinatorMarkAsUnusedOvershadowedSegments druidCoordinatorMarkAsUnusedOvershadowedSegments; DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class); private List usedSegments; DateTime start = DateTimes.of("2012-01-01"); @@ -70,7 +70,7 @@ public class DruidCoordinatorCleanupOvershadowedTest @Test public void testRun() { - druidCoordinatorCleanupOvershadowed = new DruidCoordinatorCleanupOvershadowed(coordinator); + druidCoordinatorMarkAsUnusedOvershadowedSegments = new DruidCoordinatorMarkAsUnusedOvershadowedSegments(coordinator); usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); // Dummy values for comparisons in TreeSet @@ -123,7 +123,7 @@ public void testRun() .withCoordinatorStats(new CoordinatorStats()) .withDruidCluster(druidCluster) .build(); - druidCoordinatorCleanupOvershadowed.run(params); + druidCoordinatorMarkAsUnusedOvershadowedSegments.run(params); EasyMock.verify(coordinator, druidDataSource, druidServer); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java similarity index 95% rename from server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java rename to server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java index 8d04eba96fb5..f698258dcaca 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java @@ -34,7 +34,7 @@ /** */ -public class DruidCoordinatorSegmentKillerTest +public class DruidCoordinatorUnusedSegmentsKillerTest { @Test public void testFindIntervalForKillTask() @@ -97,7 +97,7 @@ private void testFindIntervalForKillTask(List metadataSegmentsResult, EasyMock.replay(metadataSegments); IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class); - DruidCoordinatorSegmentKiller coordinatorSegmentKiller = new DruidCoordinatorSegmentKiller( + DruidCoordinatorUnusedSegmentsKiller coordinatorSegmentKiller = new DruidCoordinatorUnusedSegmentsKiller( metadataSegments, indexingServiceClient, new TestDruidCoordinatorConfig( 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 bc25effd9b60..1cb1111be659 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 @@ -223,9 +223,10 @@ public void testBuilderDefaults() @Test public void testUpdate() { - CoordinatorDynamicConfig current = CoordinatorDynamicConfig.builder() - .withSpecificDataSourcesToKill(ImmutableSet.of("x")) - .build(); + CoordinatorDynamicConfig current = CoordinatorDynamicConfig + .builder() + .withSpecificDataSourcesToKillUnusedSegmentsIn(ImmutableSet.of("x")) + .build(); Assert.assertEquals( current, @@ -246,7 +247,7 @@ public void testEqualsAndHashCodeSanity() private void assertConfig( CoordinatorDynamicConfig config, - long expectedMillisToWaitBeforeDeleting, + long expectedMillisLagBeforeCanMaskAsUnusedOvershadowedSegments, long expectedMergeBytesLimit, int expectedMergeSegmentsLimit, int expectedMaxSegmentsToMove, @@ -261,7 +262,10 @@ private void assertConfig( int decommissioningMaxPercentOfMaxSegmentsToMove ) { - Assert.assertEquals(expectedMillisToWaitBeforeDeleting, config.getMillisToWaitBeforeDeleting()); + Assert.assertEquals( + expectedMillisLagBeforeCanMaskAsUnusedOvershadowedSegments, + config.getMillisLagSinceCoordinatorBecomesLeaderBeforeCanMarkAsUnusedOvershadowedSegments() + ); Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit()); Assert.assertEquals(expectedMaxSegmentsToMove, config.getMaxSegmentsToMove()); @@ -269,10 +273,13 @@ private void assertConfig( Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit()); Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads()); Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats()); - Assert.assertEquals(expectedSpecificDataSourcesToKill, config.getSpecificDataSourcesToKill()); - Assert.assertEquals(expectedKillAllDataSources, config.isKillAllDataSources()); + Assert.assertEquals(expectedSpecificDataSourcesToKill, config.getSpecificDataSourcesToKillUnusedSegmentsIn()); + Assert.assertEquals(expectedKillAllDataSources, config.isKillUnusedSegmentsInAllDataSources()); Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue()); Assert.assertEquals(decommissioning, config.getDecommissioningNodes()); - Assert.assertEquals(decommissioningMaxPercentOfMaxSegmentsToMove, config.getDecommissioningMaxPercentOfMaxSegmentsToMove()); + Assert.assertEquals( + decommissioningMaxPercentOfMaxSegmentsToMove, + config.getDecommissioningMaxPercentOfMaxSegmentsToMove() + ); } } diff --git a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java index 086ba605ef6c..1dc39bf399db 100644 --- a/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/DataSourcesResourceTest.java @@ -544,7 +544,7 @@ public void testDeleteDataSourceSpecificInterval() Interval theInterval = Intervals.of(interval.replace('_', '/')); IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); - indexingServiceClient.killSegments("datasource1", theInterval); + indexingServiceClient.killUnusedSegments("datasource1", theInterval); EasyMock.expectLastCall().once(); EasyMock.replay(indexingServiceClient, server); 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 f49545c95bdd..47452446d3ae 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -61,11 +61,11 @@ import org.apache.druid.server.coordinator.BalancerStrategyFactory; import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig; import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorCleanupPendingSegments; +import org.apache.druid.server.coordinator.DruidCoordinatorKillStalePendingSegments; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.LoadQueueTaskMaster; import org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper; -import org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller; +import org.apache.druid.server.coordinator.helper.DruidCoordinatorUnusedSegmentsKiller; import org.apache.druid.server.http.ClusterResource; import org.apache.druid.server.http.CoordinatorCompactionConfigsResource; import org.apache.druid.server.http.CoordinatorDynamicConfigsResource; @@ -224,11 +224,11 @@ public void configure(Binder binder) conditionalMultibind.addConditionBinding( "druid.coordinator.kill.on", Predicates.equalTo("true"), - DruidCoordinatorSegmentKiller.class + DruidCoordinatorUnusedSegmentsKiller.class ).addConditionBinding( "druid.coordinator.kill.pendingSegments.on", Predicates.equalTo("true"), - DruidCoordinatorCleanupPendingSegments.class + DruidCoordinatorKillStalePendingSegments.class ); bindAnnouncer( From d959bb05975dd47c7b908ea0844acf99a9aafa5e Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 28 Mar 2019 19:19:08 -0300 Subject: [PATCH 08/40] Rename ClientCompactTaskQuery to ClientCompactionTaskQuery for consistency with CompactionTask; ClientCompactQueryTuningConfig to ClientCompactionTaskQueryTuningConfig --- .../indexing/common/task/CompactionTask.java | 6 +++--- ...ry.java => ClientCompactionTaskQuery.java} | 10 ++++----- ...lientCompactionTaskQueryTuningConfig.java} | 21 +++++++++++-------- .../client/indexing/ClientTaskQuery.java | 2 +- .../indexing/HttpIndexingServiceClient.java | 4 ++-- .../indexing/IndexingServiceClient.java | 2 +- .../DataSourceCompactionConfig.java | 12 +++++------ .../DruidCoordinatorSegmentCompactor.java | 8 +++---- .../indexing/NoopIndexingServiceClient.java | 2 +- .../DataSourceCompactionConfigTest.java | 6 +++--- .../DruidCoordinatorSegmentCompactorTest.java | 4 ++-- 11 files changed, 40 insertions(+), 37 deletions(-) rename server/src/main/java/org/apache/druid/client/indexing/{ClientCompactTaskQuery.java => ClientCompactionTaskQuery.java} (89%) rename server/src/main/java/org/apache/druid/client/indexing/{ClientCompactQueryTuningConfig.java => ClientCompactionTaskQueryTuningConfig.java} (88%) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 7e2027ef638c..4eeb3a8e700e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -30,7 +30,7 @@ import com.google.common.collect.HashBiMap; import com.google.common.collect.Lists; import org.apache.druid.client.coordinator.CoordinatorClient; -import org.apache.druid.client.indexing.ClientCompactTaskQuery; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling; import org.apache.druid.data.input.impl.DimensionsSpec; @@ -105,9 +105,9 @@ import java.util.stream.StreamSupport; /** - * The client representation of this task is {@link ClientCompactTaskQuery}. JSON + * The client representation of this task is {@link ClientCompactionTaskQuery}. JSON * serialization fields of this class must correspond to those of {@link - * ClientCompactTaskQuery}. + * ClientCompactionTaskQuery}. */ public class CompactionTask extends AbstractTask { diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java similarity index 89% rename from server/src/main/java/org/apache/druid/client/indexing/ClientCompactTaskQuery.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java index e5113040ce06..50058837ac87 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQuery.java @@ -31,23 +31,23 @@ * Client representation of {@link org.apache.druid.indexing.common.task.CompactionTask}. JSON serialization fields of * this class must correspond to those of {@link org.apache.druid.indexing.common.task.CompactionTask}. */ -public class ClientCompactTaskQuery implements ClientTaskQuery +public class ClientCompactionTaskQuery implements ClientTaskQuery { private final String dataSource; private final List segments; private final boolean keepSegmentGranularity; @Nullable private final Long targetCompactionSizeBytes; - private final ClientCompactQueryTuningConfig tuningConfig; + private final ClientCompactionTaskQueryTuningConfig tuningConfig; private final Map context; @JsonCreator - public ClientCompactTaskQuery( + public ClientCompactionTaskQuery( @JsonProperty("dataSource") String dataSource, @JsonProperty("segments") List segments, @JsonProperty("keepSegmentGranularity") boolean keepSegmentGranularity, @JsonProperty("targetCompactionSizeBytes") @Nullable Long targetCompactionSizeBytes, - @JsonProperty("tuningConfig") ClientCompactQueryTuningConfig tuningConfig, + @JsonProperty("tuningConfig") ClientCompactionTaskQueryTuningConfig tuningConfig, @JsonProperty("context") Map context ) { @@ -93,7 +93,7 @@ public Long getTargetCompactionSizeBytes() } @JsonProperty - public ClientCompactQueryTuningConfig getTuningConfig() + public ClientCompactionTaskQueryTuningConfig getTuningConfig() { return tuningConfig; } diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java similarity index 88% rename from server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java rename to server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java index 9bae161c76a0..84238e36e6d8 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionTaskQueryTuningConfig.java @@ -22,14 +22,17 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.segment.IndexSpec; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig.UserCompactTuningConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; import java.util.Objects; -public class ClientCompactQueryTuningConfig +public class ClientCompactionTaskQueryTuningConfig { - // These default values should be synchronized with those of IndexTuningConfig + /** + * These default values should be synchronized with those of {@link + * org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig} + */ private static final int DEFAULT_MAX_ROWS_IN_MEMORY = 75_000; private static final int DEFAULT_MAX_TOTAL_ROWS = 20_000_000; private static final IndexSpec DEFAULT_INDEX_SPEC = new IndexSpec(); @@ -44,12 +47,12 @@ public class ClientCompactQueryTuningConfig private final int maxPendingPersists; private final long pushTimeout; - public static ClientCompactQueryTuningConfig from( - @Nullable UserCompactTuningConfig userCompactTuningConfig, + public static ClientCompactionTaskQueryTuningConfig from( + @Nullable DataSourceCompactionConfig.UserCompactionTaskQueryTuningConfig userCompactTuningConfig, @Nullable Integer maxRowsPerSegment ) { - return new ClientCompactQueryTuningConfig( + return new ClientCompactionTaskQueryTuningConfig( maxRowsPerSegment, userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxRowsInMemory(), userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxTotalRows(), @@ -60,7 +63,7 @@ public static ClientCompactQueryTuningConfig from( } @JsonCreator - public ClientCompactQueryTuningConfig( + public ClientCompactionTaskQueryTuningConfig( @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxTotalRows") @Nullable Integer maxTotalRows, @@ -129,7 +132,7 @@ public boolean equals(Object o) if (o == null || getClass() != o.getClass()) { return false; } - ClientCompactQueryTuningConfig that = (ClientCompactQueryTuningConfig) o; + ClientCompactionTaskQueryTuningConfig that = (ClientCompactionTaskQueryTuningConfig) o; return maxRowsInMemory == that.maxRowsInMemory && maxTotalRows == that.maxTotalRows && maxPendingPersists == that.maxPendingPersists && @@ -154,7 +157,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactQueryTuningConfig{" + + return "ClientCompactionTaskQueryTuningConfig{" + "maxRowsPerSegment=" + maxRowsPerSegment + ", maxRowsInMemory=" + maxRowsInMemory + ", maxTotalRows=" + maxTotalRows + diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java index 9d55736bf99a..c2dc1f936cec 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientTaskQuery.java @@ -34,7 +34,7 @@ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { @Type(name = "kill", value = ClientKillUnusedSegmentsTaskQuery.class), - @Type(name = "compact", value = ClientCompactTaskQuery.class) + @Type(name = "compact", value = ClientCompactionTaskQuery.class) }) public interface ClientTaskQuery { diff --git a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java index 9a92c3d9857c..a604a379ec6d 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/HttpIndexingServiceClient.java @@ -71,7 +71,7 @@ public String compactSegments( boolean keepSegmentGranularity, @Nullable Long targetCompactionSizeBytes, int compactionTaskPriority, - @Nullable ClientCompactQueryTuningConfig tuningConfig, + @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, @Nullable Map context ) { @@ -87,7 +87,7 @@ public String compactSegments( context.put("priority", compactionTaskPriority); return runTask( - new ClientCompactTaskQuery( + new ClientCompactionTaskQuery( dataSource, segments, keepSegmentGranularity, diff --git a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java index 3549a4b4985f..25ec9fb5a3cd 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java +++ b/server/src/main/java/org/apache/druid/client/indexing/IndexingServiceClient.java @@ -39,7 +39,7 @@ String compactSegments( boolean keepSegmentGranularity, @Nullable Long targetCompactionSizeBytes, int compactionTaskPriority, - @Nullable ClientCompactQueryTuningConfig tuningConfig, + @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, @Nullable Map context ); diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java index ef33fb024b62..32c5f87cf26f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DataSourceCompactionConfig.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.segment.IndexSpec; import org.joda.time.Period; @@ -54,7 +54,7 @@ public class DataSourceCompactionConfig private final Integer maxRowsPerSegment; private final int maxNumSegmentsToCompact; private final Period skipOffsetFromLatest; - private final UserCompactTuningConfig tuningConfig; + private final UserCompactionTaskQueryTuningConfig tuningConfig; private final Map taskContext; @JsonCreator @@ -67,7 +67,7 @@ public DataSourceCompactionConfig( @JsonProperty("maxRowsPerSegment") @Nullable Integer maxRowsPerSegment, @JsonProperty("maxNumSegmentsToCompact") @Nullable Integer maxNumSegmentsToCompact, @JsonProperty("skipOffsetFromLatest") @Nullable Period skipOffsetFromLatest, - @JsonProperty("tuningConfig") @Nullable UserCompactTuningConfig tuningConfig, + @JsonProperty("tuningConfig") @Nullable UserCompactionTaskQueryTuningConfig tuningConfig, @JsonProperty("taskContext") @Nullable Map taskContext ) { @@ -156,7 +156,7 @@ public Period getSkipOffsetFromLatest() @JsonProperty @Nullable - public UserCompactTuningConfig getTuningConfig() + public UserCompactionTaskQueryTuningConfig getTuningConfig() { return tuningConfig; } @@ -205,10 +205,10 @@ public int hashCode() ); } - public static class UserCompactTuningConfig extends ClientCompactQueryTuningConfig + public static class UserCompactionTaskQueryTuningConfig extends ClientCompactionTaskQueryTuningConfig { @JsonCreator - public UserCompactTuningConfig( + public UserCompactionTaskQueryTuningConfig( @JsonProperty("maxRowsInMemory") @Nullable Integer maxRowsInMemory, @JsonProperty("maxTotalRows") @Nullable Integer maxTotalRows, @JsonProperty("indexSpec") @Nullable IndexSpec indexSpec, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java index d48a594b471c..d82d4b8d51de 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -22,8 +22,8 @@ import com.google.common.collect.Iterables; import com.google.inject.Inject; import it.unimi.dsi.fastutil.objects.Object2LongMap; -import org.apache.druid.client.indexing.ClientCompactTaskQuery; -import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig; +import org.apache.druid.client.indexing.ClientCompactionTaskQuery; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.TaskPayloadResponse; import org.apache.druid.indexer.TaskStatusPlus; @@ -97,7 +97,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) throw new ISE("WTH? got a null paylord from overlord for task[%s]", status.getId()); } if (COMPACT_TASK_TYPE.equals(response.getPayload().getType())) { - final ClientCompactTaskQuery compactQuery = (ClientCompactTaskQuery) response.getPayload(); + final ClientCompactionTaskQuery compactQuery = (ClientCompactionTaskQuery) response.getPayload(); final Interval interval = JodaUtils.umbrellaInterval( compactQuery.getSegments() .stream() @@ -186,7 +186,7 @@ private CoordinatorStats doRun( config.isKeepSegmentGranularity(), config.getTargetCompactionSizeBytes(), config.getTaskPriority(), - ClientCompactQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()), + ClientCompactionTaskQueryTuningConfig.from(config.getTuningConfig(), config.getMaxRowsPerSegment()), config.getTaskContext() ); LOG.info( diff --git a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java index 7bd1e214dd04..9539958ef623 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java +++ b/server/src/test/java/org/apache/druid/client/indexing/NoopIndexingServiceClient.java @@ -48,7 +48,7 @@ public String compactSegments( boolean keepSegmentGranularity, @Nullable Long targetCompactionSizeBytes, int compactionTaskPriority, - @Nullable ClientCompactQueryTuningConfig tuningConfig, + @Nullable ClientCompactionTaskQueryTuningConfig tuningConfig, @Nullable Map context ) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java index eae6043c2148..8edb588b387b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DataSourceCompactionConfigTest.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableMap; import org.apache.druid.jackson.DefaultObjectMapper; -import org.apache.druid.server.coordinator.DataSourceCompactionConfig.UserCompactTuningConfig; +import org.apache.druid.server.coordinator.DataSourceCompactionConfig.UserCompactionTaskQueryTuningConfig; import org.joda.time.Period; import org.junit.Assert; import org.junit.Test; @@ -94,11 +94,11 @@ public void testSerdeWithMaxRowsPerSegment() throws IOException @Test public void testSerdeUserCompactTuningConfig() throws IOException { - final UserCompactTuningConfig config = new UserCompactTuningConfig(null, null, null, null, null); + final DataSourceCompactionConfig.UserCompactionTaskQueryTuningConfig config = new DataSourceCompactionConfig.UserCompactionTaskQueryTuningConfig(null, null, null, null, null); final String json = objectMapper.writeValueAsString(config); // Check maxRowsPerSegment doesn't exist in the JSON string Assert.assertFalse(json.contains("maxRowsPerSegment")); - final UserCompactTuningConfig fromJson = objectMapper.readValue(json, UserCompactTuningConfig.class); + final DataSourceCompactionConfig.UserCompactionTaskQueryTuningConfig fromJson = objectMapper.readValue(json, UserCompactionTaskQueryTuningConfig.class); Assert.assertEquals(config, fromJson); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index 32deb2420d19..a3aa838c2588 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java @@ -22,7 +22,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig; +import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.indexer.TaskStatusPlus; @@ -68,7 +68,7 @@ public String compactSegments( boolean keepSegmentGranularity, @Nullable Long targetCompactionSizeBytes, int compactionTaskPriority, - ClientCompactQueryTuningConfig tuningConfig, + ClientCompactionTaskQueryTuningConfig tuningConfig, Map context ) { From 01c44946029732929fc83f3735898e07b483aa7d Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 28 Mar 2019 19:22:34 -0300 Subject: [PATCH 09/40] More variable and method renames --- .../DruidCoordinatorSegmentCompactor.java | 4 ++-- .../DruidCoordinatorUnusedSegmentsKiller.java | 4 ++-- ...idCoordinatorUnusedSegmentsKillerTest.java | 24 +++++++++---------- 3 files changed, 16 insertions(+), 16 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java index d82d4b8d51de..ff96291ef5ea 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java @@ -97,9 +97,9 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) throw new ISE("WTH? got a null paylord from overlord for task[%s]", status.getId()); } if (COMPACT_TASK_TYPE.equals(response.getPayload().getType())) { - final ClientCompactionTaskQuery compactQuery = (ClientCompactionTaskQuery) response.getPayload(); + final ClientCompactionTaskQuery compactionTaskQuery = (ClientCompactionTaskQuery) response.getPayload(); final Interval interval = JodaUtils.umbrellaInterval( - compactQuery.getSegments() + compactionTaskQuery.getSegments() .stream() .map(DataSegment::getInterval) .sorted(Comparators.intervalsByStartThenEnd()) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java index 87fef359caf6..f8447d771e3d 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java @@ -104,7 +104,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) lastKillTime = System.currentTimeMillis(); for (String dataSource : dataSourcesToKill) { - final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill); + final Interval intervalToKill = findIntervalForKill(dataSource, maxSegmentsToKill); if (intervalToKill != null) { try { indexingServiceClient.killUnusedSegments(dataSource, intervalToKill); @@ -124,7 +124,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) @VisibleForTesting @Nullable - Interval findIntervalForKillTask(String dataSource, int limit) + Interval findIntervalForKill(String dataSource, int limit) { List unusedSegmentIntervals = metadataSegments.getUnusedSegmentIntervals( dataSource, diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java index f698258dcaca..77b14267a214 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java @@ -37,34 +37,34 @@ public class DruidCoordinatorUnusedSegmentsKillerTest { @Test - public void testFindIntervalForKillTask() + public void testFindIntervalForKill() { - testFindIntervalForKillTask(null, null); - testFindIntervalForKillTask(ImmutableList.of(), null); + testFindIntervalForKill(null, null); + testFindIntervalForKill(ImmutableList.of(), null); - testFindIntervalForKillTask(ImmutableList.of(Intervals.of("2014/2015")), Intervals.of("2014/2015")); + testFindIntervalForKill(ImmutableList.of(Intervals.of("2014/2015")), Intervals.of("2014/2015")); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2016/2017")), Intervals.of("2014/2017") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2015/2016")), Intervals.of("2014/2016") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2015/2016"), Intervals.of("2014/2015")), Intervals.of("2014/2016") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of(Intervals.of("2015/2017"), Intervals.of("2014/2016")), Intervals.of("2014/2017") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of( Intervals.of("2015/2019"), Intervals.of("2014/2016"), @@ -73,7 +73,7 @@ public void testFindIntervalForKillTask() Intervals.of("2014/2020") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of( Intervals.of("2015/2019"), Intervals.of("2014/2016"), @@ -84,7 +84,7 @@ public void testFindIntervalForKillTask() ); } - private void testFindIntervalForKillTask(List metadataSegmentsResult, Interval expected) + private void testFindIntervalForKill(List metadataSegmentsResult, Interval expected) { MetadataSegments metadataSegments = EasyMock.createMock(MetadataSegments.class); EasyMock.expect( @@ -117,7 +117,7 @@ private void testFindIntervalForKillTask(List metadataSegmentsResult, Assert.assertEquals( expected, - coordinatorSegmentKiller.findIntervalForKillTask("test", 10000) + coordinatorSegmentKiller.findIntervalForKill("test", 10000) ); } } From 06ec3896375fd1120060923cc073256906d09371 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 1 Apr 2019 14:55:43 -0300 Subject: [PATCH 10/40] Rename MetadataSegments to SegmentsMetadata --- .../MaterializedViewSupervisor.java | 12 +++---- .../MaterializedViewSupervisorSpec.java | 14 ++++---- .../MaterializedViewSupervisorSpecTest.java | 4 +-- .../MaterializedViewSupervisorTest.java | 10 +++--- .../actions/SegmentListActionsTest.java | 2 +- .../common/actions/TaskActionTestKit.java | 18 +++++------ .../CoordinatorResourceTestClient.java | 8 ++--- .../tests/indexer/ITCompactionTaskTest.java | 2 +- .../druid/guice/MetadataConfigModule.java | 4 +-- .../guice/SQLMetadataStorageDruidModule.java | 20 ++++++------ .../metadata/SQLMetadataRuleManager.java | 2 +- ...ataSegments.java => SegmentsMetadata.java} | 5 ++- ...onfig.java => SegmentsMetadataConfig.java} | 2 +- ...der.java => SegmentsMetadataProvider.java} | 4 +-- ...Segments.java => SqlSegmentsMetadata.java} | 16 +++++----- ....java => SqlSegmentsMetadataProvider.java} | 12 +++---- .../server/coordinator/DruidCoordinator.java | 32 +++++++++---------- .../helper/DruidCoordinatorBalancer.java | 2 +- .../DruidCoordinatorUnloadUnusedSegments.java | 2 +- .../DruidCoordinatorUnusedSegmentsKiller.java | 12 +++---- .../server/http/DataSourcesResource.java | 16 +++++----- .../druid/server/http/MetadataResource.java | 20 ++++++------ ...Test.java => SqlSegmentsMetadataTest.java} | 8 ++--- .../CuratorDruidCoordinatorTest.java | 14 ++++---- .../coordinator/DruidCoordinatorTest.java | 32 +++++++++---------- ...idCoordinatorUnusedSegmentsKillerTest.java | 14 ++++---- .../org/apache/druid/cli/CliCoordinator.java | 12 +++---- .../calcite/schema/MetadataSegmentView.java | 17 ++++++---- .../sql/calcite/schema/SystemSchema.java | 7 ++-- 29 files changed, 164 insertions(+), 159 deletions(-) rename server/src/main/java/org/apache/druid/metadata/{MetadataSegments.java => SegmentsMetadata.java} (94%) rename server/src/main/java/org/apache/druid/metadata/{MetadataSegmentsConfig.java => SegmentsMetadataConfig.java} (96%) rename server/src/main/java/org/apache/druid/metadata/{MetadataSegmentsProvider.java => SegmentsMetadataProvider.java} (89%) rename server/src/main/java/org/apache/druid/metadata/{SqlMetadataSegments.java => SqlSegmentsMetadata.java} (97%) rename server/src/main/java/org/apache/druid/metadata/{SqlMetadataSegmentsProvider.java => SqlSegmentsMetadataProvider.java} (87%) rename server/src/test/java/org/apache/druid/metadata/{SqlMetadataSegmentsTest.java => SqlSegmentsMetadataTest.java} (97%) diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java index d366d0d4ff87..0bf6c857e99a 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java @@ -45,7 +45,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlSegmentsMetadata; import org.apache.druid.timeline.DataSegment; import org.joda.time.Duration; import org.joda.time.Interval; @@ -71,7 +71,7 @@ public class MaterializedViewSupervisor implements Supervisor private final MetadataSupervisorManager metadataSupervisorManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final SqlMetadataSegments metadataSegments; + private final SqlSegmentsMetadata sqlSegmentsMetadata; private final MaterializedViewSupervisorSpec spec; private final TaskMaster taskMaster; private final TaskStorage taskStorage; @@ -97,7 +97,7 @@ public MaterializedViewSupervisor( TaskMaster taskMaster, TaskStorage taskStorage, MetadataSupervisorManager metadataSupervisorManager, - SqlMetadataSegments metadataSegments, + SqlSegmentsMetadata sqlSegmentsMetadata, IndexerMetadataStorageCoordinator metadataStorageCoordinator, MaterializedViewTaskConfig config, MaterializedViewSupervisorSpec spec @@ -106,7 +106,7 @@ public MaterializedViewSupervisor( this.taskMaster = taskMaster; this.taskStorage = taskStorage; this.metadataStorageCoordinator = metadataStorageCoordinator; - this.metadataSegments = metadataSegments; + this.sqlSegmentsMetadata = sqlSegmentsMetadata; this.metadataSupervisorManager = metadataSupervisorManager; this.config = config; this.spec = spec; @@ -356,7 +356,7 @@ Pair, Map>> checkSegment // drop derivative segments which interval equals the interval in toDeleteBaseSegments for (Interval interval : toDropInterval.keySet()) { for (DataSegment segment : derivativeSegments.get(interval)) { - metadataSegments.tryMarkSegmentAsUnused(segment.getId()); + sqlSegmentsMetadata.tryMarkSegmentAsUnused(segment.getId()); } } // data of the latest interval will be built firstly. @@ -464,7 +464,7 @@ private void clearSegments() { log.info("Clear all metadata of dataSource %s", dataSource); metadataStorageCoordinator.deletePendingSegments(dataSource); - if (!metadataSegments.tryMarkAsUnusedAllSegmentsInDataSource(dataSource)) { + if (!sqlSegmentsMetadata.tryMarkAsUnusedAllSegmentsInDataSource(dataSource)) { log.error("Failed to mark all segments in " + dataSource + " as unused."); } metadataStorageCoordinator.deleteDataSourceMetadata(dataSource); diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java index 36065f662706..1c76f3d1ef82 100644 --- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java +++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java @@ -42,7 +42,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlSegmentsMetadata; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; @@ -75,7 +75,7 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec private final ObjectMapper objectMapper; private final MetadataSupervisorManager metadataSupervisorManager; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private final SqlMetadataSegments metadataSegments; + private final SqlSegmentsMetadata sqlSegmentsMetadata; private final TaskMaster taskMaster; private final TaskStorage taskStorage; private final MaterializedViewTaskConfig config; @@ -98,7 +98,7 @@ public MaterializedViewSupervisorSpec( @JacksonInject TaskMaster taskMaster, @JacksonInject TaskStorage taskStorage, @JacksonInject MetadataSupervisorManager metadataSupervisorManager, - @JacksonInject SqlMetadataSegments metadataSegments, + @JacksonInject SqlSegmentsMetadata sqlSegmentsMetadata, @JacksonInject IndexerMetadataStorageCoordinator metadataStorageCoordinator, @JacksonInject MaterializedViewTaskConfig config, @JacksonInject AuthorizerMapper authorizerMapper, @@ -136,7 +136,7 @@ public MaterializedViewSupervisorSpec( this.taskMaster = taskMaster; this.taskStorage = taskStorage; this.metadataSupervisorManager = metadataSupervisorManager; - this.metadataSegments = metadataSegments; + this.sqlSegmentsMetadata = sqlSegmentsMetadata; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; this.chatHandlerProvider = chatHandlerProvider; @@ -329,7 +329,7 @@ public Supervisor createSupervisor() taskMaster, taskStorage, metadataSupervisorManager, - metadataSegments, + sqlSegmentsMetadata, metadataStorageCoordinator, config, this @@ -360,7 +360,7 @@ public SupervisorSpec createSuspendedSpec() taskMaster, taskStorage, metadataSupervisorManager, - metadataSegments, + sqlSegmentsMetadata, metadataStorageCoordinator, config, authorizerMapper, @@ -386,7 +386,7 @@ public SupervisorSpec createRunningSpec() taskMaster, taskStorage, metadataSupervisorManager, - metadataSegments, + sqlSegmentsMetadata, metadataStorageCoordinator, config, authorizerMapper, diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java index b8d782449d4f..8262f810f9be 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java @@ -31,7 +31,7 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlSegmentsMetadata; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -69,7 +69,7 @@ public void setup() .addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE) .addValue(ObjectMapper.class, objectMapper) .addValue(MetadataSupervisorManager.class, null) - .addValue(SqlMetadataSegments.class, null) + .addValue(SqlSegmentsMetadata.class, null) .addValue(IndexerMetadataStorageCoordinator.class, null) .addValue(MaterializedViewTaskConfig.class, new MaterializedViewTaskConfig()) .addValue(AuthorizerMapper.class, createMock(AuthorizerMapper.class)) diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java index 251d3ff4f34f..5c5a1206b006 100644 --- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java +++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java @@ -40,7 +40,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.MetadataSupervisorManager; -import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlSegmentsMetadata; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.aggregation.AggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory; @@ -82,7 +82,7 @@ public class MaterializedViewSupervisorTest private TaskMaster taskMaster; private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; private MetadataSupervisorManager metadataSupervisorManager; - private SqlMetadataSegments sqlMetadataSegments; + private SqlSegmentsMetadata sqlSegmentsMetadata; private TaskQueue taskQueue; private MaterializedViewSupervisor supervisor; private MaterializedViewSupervisorSpec spec; @@ -102,7 +102,7 @@ public void setUp() derbyConnector ); metadataSupervisorManager = createMock(MetadataSupervisorManager.class); - sqlMetadataSegments = createMock(SqlMetadataSegments.class); + sqlSegmentsMetadata = createMock(SqlSegmentsMetadata.class); taskQueue = createMock(TaskQueue.class); taskQueue.start(); objectMapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed")); @@ -121,7 +121,7 @@ public void setUp() taskMaster, taskStorage, metadataSupervisorManager, - sqlMetadataSegments, + sqlSegmentsMetadata, indexerMetadataStorageCoordinator, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), @@ -278,7 +278,7 @@ public void testSuspendedDoesntRun() taskMaster, taskStorage, metadataSupervisorManager, - sqlMetadataSegments, + sqlSegmentsMetadata, indexerMetadataStorageCoordinator, new MaterializedViewTaskConfig(), createMock(AuthorizerMapper.class), diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java index 4c2edd0d6940..05dfa60311eb 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java @@ -73,7 +73,7 @@ public void setup() throws IOException expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); - expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegments().tryMarkSegmentAsUnused(s.getId())); + expectedUnusedSegments.forEach(s -> actionTestKit.getSegmentsMetadata().tryMarkSegmentAsUnused(s.getId())); } private DataSegment createSegment(Interval interval, String version) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index 980c67a699de..f3ba60b5913b 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -29,11 +29,11 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.MetadataSegments; -import org.apache.druid.metadata.MetadataSegmentsConfig; +import org.apache.druid.metadata.SegmentsMetadata; +import org.apache.druid.metadata.SegmentsMetadataConfig; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; -import org.apache.druid.metadata.SqlMetadataSegments; +import org.apache.druid.metadata.SqlSegmentsMetadata; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.easymock.EasyMock; @@ -48,7 +48,7 @@ public class TaskActionTestKit extends ExternalResource private TaskLockbox taskLockbox; private TestDerbyConnector testDerbyConnector; private IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private MetadataSegments metadataSegments; + private SegmentsMetadata segmentsMetadata; private TaskActionToolbox taskActionToolbox; public TaskLockbox getTaskLockbox() @@ -61,9 +61,9 @@ public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator() return metadataStorageCoordinator; } - public MetadataSegments getMetadataSegments() + public SegmentsMetadata getSegmentsMetadata() { - return metadataSegments; + return segmentsMetadata; } public TaskActionToolbox getTaskActionToolbox() @@ -86,9 +86,9 @@ public void before() metadataStorageTablesConfig, testDerbyConnector ); - metadataSegments = new SqlMetadataSegments( + segmentsMetadata = new SqlSegmentsMetadata( objectMapper, - Suppliers.ofInstance(new MetadataSegmentsConfig()), + Suppliers.ofInstance(new SegmentsMetadataConfig()), Suppliers.ofInstance(metadataStorageTablesConfig), testDerbyConnector ); @@ -116,7 +116,7 @@ public void after() taskLockbox = null; testDerbyConnector = null; metadataStorageCoordinator = null; - metadataSegments = null; + segmentsMetadata = null; taskActionToolbox = null; } } diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java index 6974b44a35f4..e96f16f7c4b7 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/CoordinatorResourceTestClient.java @@ -72,7 +72,7 @@ private String getCoordinatorURL() ); } - private String getMetadataSegmentsURL(String dataSource) + private String getSegmentsMetadataURL(String dataSource) { return StringUtils.format("%smetadata/datasources/%s/segments", getCoordinatorURL(), StringUtils.urlEncode(dataSource)); } @@ -92,12 +92,12 @@ private String getLoadStatusURL() return StringUtils.format("%s%s", getCoordinatorURL(), "loadstatus"); } - // return a list of the segment dates for the specified datasource - public List getMetadataSegments(final String dataSource) + /** return a list of the segment dates for the specified data source */ + public List getSegments(final String dataSource) { ArrayList segments; try { - StatusResponseHolder response = makeRequest(HttpMethod.GET, getMetadataSegmentsURL(dataSource)); + StatusResponseHolder response = makeRequest(HttpMethod.GET, getSegmentsMetadataURL(dataSource)); segments = jsonMapper.readValue( response.getContent(), new TypeReference>() diff --git a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java index db6ebff48abb..4a86d61d6524 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/indexer/ITCompactionTaskTest.java @@ -164,7 +164,7 @@ private void checkCompactionFinished(int numExpectedSegments) { RetryUtil.retryUntilTrue( () -> { - int metadataSegmentCount = coordinator.getMetadataSegments(fullDatasourceName).size(); + int metadataSegmentCount = coordinator.getSegments(fullDatasourceName).size(); LOG.info("Current metadata segment count: %d, expected: %d", metadataSegmentCount, numExpectedSegments); return metadataSegmentCount == numExpectedSegments; }, diff --git a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java index 396922d255ee..9a572fdcc931 100644 --- a/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java +++ b/server/src/main/java/org/apache/druid/guice/MetadataConfigModule.java @@ -22,7 +22,7 @@ import com.google.inject.Binder; import com.google.inject.Module; import org.apache.druid.metadata.MetadataRuleManagerConfig; -import org.apache.druid.metadata.MetadataSegmentsConfig; +import org.apache.druid.metadata.SegmentsMetadataConfig; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -34,7 +34,7 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.metadata.storage.tables", MetadataStorageTablesConfig.class); JsonConfigProvider.bind(binder, "druid.metadata.storage.connector", MetadataStorageConnectorConfig.class); - JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentsConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.segments", SegmentsMetadataConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); } } diff --git a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java index 74fa2b9726c0..f9a8cc305dc3 100644 --- a/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java +++ b/server/src/main/java/org/apache/druid/guice/SQLMetadataStorageDruidModule.java @@ -31,8 +31,8 @@ import org.apache.druid.metadata.MetadataRuleManagerProvider; import org.apache.druid.metadata.MetadataSegmentPublisher; import org.apache.druid.metadata.MetadataSegmentPublisherProvider; -import org.apache.druid.metadata.MetadataSegments; -import org.apache.druid.metadata.MetadataSegmentsProvider; +import org.apache.druid.metadata.SegmentsMetadata; +import org.apache.druid.metadata.SegmentsMetadataProvider; import org.apache.druid.metadata.MetadataStorageActionHandlerFactory; import org.apache.druid.metadata.MetadataStorageConnector; import org.apache.druid.metadata.MetadataStorageProvider; @@ -43,8 +43,8 @@ import org.apache.druid.metadata.SQLMetadataSegmentPublisher; import org.apache.druid.metadata.SQLMetadataSegmentPublisherProvider; import org.apache.druid.metadata.SQLMetadataSupervisorManager; -import org.apache.druid.metadata.SqlMetadataSegments; -import org.apache.druid.metadata.SqlMetadataSegmentsProvider; +import org.apache.druid.metadata.SqlSegmentsMetadata; +import org.apache.druid.metadata.SqlSegmentsMetadataProvider; import org.apache.druid.server.audit.AuditManagerProvider; import org.apache.druid.server.audit.SQLAuditManager; import org.apache.druid.server.audit.SQLAuditManagerConfig; @@ -73,8 +73,8 @@ public void createBindingChoices(Binder binder, String defaultValue) PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataStorageProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(SQLMetadataConnector.class), defaultValue); - PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegments.class), defaultValue); - PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentsProvider.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(SegmentsMetadata.class), defaultValue); + PolyBind.createChoiceWithDefault(binder, prop, Key.get(SegmentsMetadataProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManager.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataRuleManagerProvider.class), defaultValue); PolyBind.createChoiceWithDefault(binder, prop, Key.get(MetadataSegmentPublisher.class), defaultValue); @@ -90,14 +90,14 @@ public void createBindingChoices(Binder binder, String defaultValue) @Override public void configure(Binder binder) { - PolyBind.optionBinder(binder, Key.get(MetadataSegments.class)) + PolyBind.optionBinder(binder, Key.get(SegmentsMetadata.class)) .addBinding(type) - .to(SqlMetadataSegments.class) + .to(SqlSegmentsMetadata.class) .in(LazySingleton.class); - PolyBind.optionBinder(binder, Key.get(MetadataSegmentsProvider.class)) + PolyBind.optionBinder(binder, Key.get(SegmentsMetadataProvider.class)) .addBinding(type) - .to(SqlMetadataSegmentsProvider.class) + .to(SqlSegmentsMetadataProvider.class) .in(LazySingleton.class); PolyBind.optionBinder(binder, Key.get(MetadataRuleManager.class)) diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java index 2aff2f2fce82..4efed35d92f4 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java @@ -148,7 +148,7 @@ public Void withHandle(Handle handle) throws Exception * the theoretical situation of two tasks scheduled in {@link #start()} calling {@link #poll()} concurrently, if * the sequence of {@link #start()} - {@link #stop()} - {@link #start()} actions occurs quickly. * - * {@link SqlMetadataSegments} also have a similar issue. + * {@link SqlSegmentsMetadata} also have a similar issue. */ private long currentStartOrder = -1; private ScheduledExecutorService exec = null; diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegments.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java similarity index 94% rename from server/src/main/java/org/apache/druid/metadata/MetadataSegments.java rename to server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java index a66d203f7b1e..f57ca14c46ba 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegments.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java @@ -31,8 +31,11 @@ import java.util.List; /** + * The difference between this class and {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} is that this + * class resides in Coordinator's memory, while {@link org.apache.druid.sql.calcite.schema.MetadataSegmentView} resides + * in Broker's memory. */ -public interface MetadataSegments +public interface SegmentsMetadata { void start(); diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsConfig.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataConfig.java similarity index 96% rename from server/src/main/java/org/apache/druid/metadata/MetadataSegmentsConfig.java rename to server/src/main/java/org/apache/druid/metadata/SegmentsMetadataConfig.java index 0e6b686181dc..10f5226f19ae 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsConfig.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataConfig.java @@ -24,7 +24,7 @@ /** */ -public class MetadataSegmentsConfig +public class SegmentsMetadataConfig { @JsonProperty private Period pollDuration = new Period("PT1M"); diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsProvider.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataProvider.java similarity index 89% rename from server/src/main/java/org/apache/druid/metadata/MetadataSegmentsProvider.java rename to server/src/main/java/org/apache/druid/metadata/SegmentsMetadataProvider.java index c1e0900a6135..bf1744a1a75a 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentsProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadataProvider.java @@ -23,8 +23,8 @@ /** */ -public interface MetadataSegmentsProvider extends Provider +public interface SegmentsMetadataProvider extends Provider { @Override - MetadataSegments get(); + SegmentsMetadata get(); } diff --git a/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java similarity index 97% rename from server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java rename to server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java index 3dfdb7c71e8f..c5a59f914ae0 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegments.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java @@ -76,9 +76,9 @@ /** */ @ManageLifecycle -public class SqlMetadataSegments implements MetadataSegments +public class SqlSegmentsMetadata implements SegmentsMetadata { - private static final EmittingLogger log = new EmittingLogger(SqlMetadataSegments.class); + private static final EmittingLogger log = new EmittingLogger(SqlSegmentsMetadata.class); /** * Use to synchronize {@link #start()}, {@link #stop()}, {@link #poll()}, and {@link #isStarted()}. These methods @@ -99,16 +99,16 @@ public class SqlMetadataSegments implements MetadataSegments private final Object pollLock = new Object(); private final ObjectMapper jsonMapper; - private final Supplier config; + private final Supplier config; private final Supplier dbTables; private final SQLMetadataConnector connector; private ConcurrentHashMap dataSources = new ConcurrentHashMap<>(); - /** The number of times this SqlMetadataSegments was started. */ + /** The number of times this SqlSegmentsMetadata was started. */ private long startCount = 0; /** - * Equal to the current {@link #startCount} value, if the SqlMetadataSegments is currently started; -1 if + * Equal to the current {@link #startCount} value, if the SqlSegmentsMetadata is currently started; -1 if * currently stopped. * * This field is used to implement a simple stamp mechanism instead of just a boolean "started" flag to prevent @@ -122,9 +122,9 @@ public class SqlMetadataSegments implements MetadataSegments private ScheduledExecutorService exec = null; @Inject - public SqlMetadataSegments( + public SqlSegmentsMetadata( ObjectMapper jsonMapper, - Supplier config, + Supplier config, Supplier dbTables, SQLMetadataConnector connector ) @@ -171,7 +171,7 @@ private Runnable createPollTaskForStartOrder(long startOrder) // poll() is synchronized together with start(), stop() and isStarted() to ensure that when stop() exits, poll() // won't actually run anymore after that (it could only enter the syncrhonized section and exit immediately // because the localStartedOrder doesn't match the new currentStartOrder). It's needed to avoid flakiness in - // SqlMetadataSegmentsTest. See https://github.com/apache/incubator-druid/issues/6028 + // SqlSegmentsMetadataTest. See https://github.com/apache/incubator-druid/issues/6028 ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock(); lock.lock(); try { diff --git a/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegmentsProvider.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataProvider.java similarity index 87% rename from server/src/main/java/org/apache/druid/metadata/SqlMetadataSegmentsProvider.java rename to server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataProvider.java index 3050e9889873..7686322300d8 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlMetadataSegmentsProvider.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadataProvider.java @@ -26,18 +26,18 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; -public class SqlMetadataSegmentsProvider implements MetadataSegmentsProvider +public class SqlSegmentsMetadataProvider implements SegmentsMetadataProvider { private final ObjectMapper jsonMapper; - private final Supplier config; + private final Supplier config; private final Supplier storageConfig; private final SQLMetadataConnector connector; private final Lifecycle lifecycle; @Inject - public SqlMetadataSegmentsProvider( + public SqlSegmentsMetadataProvider( ObjectMapper jsonMapper, - Supplier config, + Supplier config, Supplier storageConfig, SQLMetadataConnector connector, Lifecycle lifecycle @@ -51,7 +51,7 @@ public SqlMetadataSegmentsProvider( } @Override - public MetadataSegments get() + public SegmentsMetadata get() { lifecycle.addHandler( new Lifecycle.Handler() @@ -70,7 +70,7 @@ public void stop() } ); - return new SqlMetadataSegments( + return new SqlSegmentsMetadata( jsonMapper, config, storageConfig, diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java index d03be349b898..9e465af3f163 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java @@ -58,7 +58,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer; import org.apache.druid.server.coordinator.helper.DruidCoordinatorMarkAsUnusedOvershadowedSegments; @@ -125,7 +125,7 @@ public class DruidCoordinator private final DruidCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; - private final MetadataSegments metadataSegments; + private final SegmentsMetadata segmentsMetadata; private final ServerInventoryView serverInventoryView; private final MetadataRuleManager metadataRuleManager; private final CuratorFramework curator; @@ -151,7 +151,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegments metadataSegments, + SegmentsMetadata segmentsMetadata, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -171,7 +171,7 @@ public DruidCoordinator( config, zkPaths, configManager, - metadataSegments, + segmentsMetadata, serverInventoryView, metadataRuleManager, curator, @@ -193,7 +193,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegments metadataSegments, + SegmentsMetadata segmentsMetadata, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -214,7 +214,7 @@ public DruidCoordinator( this.zkPaths = zkPaths; this.configManager = configManager; - this.metadataSegments = metadataSegments; + this.segmentsMetadata = segmentsMetadata; this.serverInventoryView = serverInventoryView; this.metadataRuleManager = metadataRuleManager; this.curator = curator; @@ -302,7 +302,7 @@ public Object2IntMap computeNumsUnavailableUsedSegmentsPerDataSource() public Map getLoadStatus() { Map loadStatus = new HashMap<>(); - for (ImmutableDruidDataSource dataSource : metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) { + for (ImmutableDruidDataSource dataSource : segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()) { final Set segments = Sets.newHashSet(dataSource.getSegments()); final int numUsedSegments = segments.size(); @@ -345,7 +345,7 @@ public CoordinatorCompactionConfig getCompactionConfig() public void tryMarkSegmentAsUnused(DataSegment segment) { log.info("Removing Segment[%s]", segment.getId()); - metadataSegments.tryMarkSegmentAsUnused(segment.getId()); + segmentsMetadata.tryMarkSegmentAsUnused(segment.getId()); } public String getCurrentLeader() @@ -373,13 +373,13 @@ public void moveSegment( throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); } - ImmutableDruidDataSource dataSource = metadataSegments.prepareImmutableDataSourceWithUsedSegments(segment.getDataSource()); + ImmutableDruidDataSource dataSource = segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(segment.getDataSource()); if (dataSource == null) { throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); } - // get segment information from MetadataSegments instead of getting it from fromServer's. - // This is useful when MetadataSegments and fromServer DataSegment's are different for same + // get segment information from SegmentsMetadata instead of getting it from fromServer's. + // This is useful when SegmentsMetadata and fromServer DataSegment's are different for same // identifier (say loadSpec differs because of deep storage migration). final DataSegment segmentToLoad = dataSource.getSegment(segment.getId()); if (segmentToLoad == null) { @@ -461,7 +461,7 @@ public void moveSegment( */ public Iterable iterateAllUsedSegments() { - return metadataSegments.iterateAllUsedSegments(); + return segmentsMetadata.iterateAllUsedSegments(); } @LifecycleStart @@ -518,7 +518,7 @@ private void becomeLeader() log.info("I am the leader of the coordinators, all must bow!"); log.info("Starting coordination in [%s]", config.getCoordinatorStartDelay()); - metadataSegments.start(); + segmentsMetadata.start(); metadataRuleManager.start(); lookupCoordinatorManager.start(); serviceAnnouncer.announce(self); @@ -586,7 +586,7 @@ private void stopBeingLeader() serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); metadataRuleManager.stop(); - metadataSegments.stop(); + segmentsMetadata.stop(); } } @@ -630,7 +630,7 @@ public void run() } List allStarted = Arrays.asList( - metadataSegments.isStarted(), + segmentsMetadata.isStarted(), serverInventoryView.isStarted() ); for (Boolean aBoolean : allStarted) { @@ -652,7 +652,7 @@ public void run() DruidCoordinatorRuntimeParams .newBuilder() .withStartTimeNanos(startTimeNanos) - .withDataSourcesWithUsedSegments(metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) + .withDataSourcesWithUsedSegments(segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()) .withDynamicConfigs(getDynamicConfigs()) .withCompactionConfig(getCompactionConfig()) .withEmitter(emitter) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java index 11e080554e8d..caab9fa73a31 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java @@ -195,7 +195,7 @@ private Pair balanceServers( log.info("All servers to move segments from are empty, ending run."); break; } - // DruidCoordinatorRuntimeParams.getUsedSegments originate from MetadataSegments, i. e. that's a set of segments + // DruidCoordinatorRuntimeParams.getUsedSegments originate from SegmentsMetadata, i. e. that's a set of segments // that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be // any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such // discrepancies eventually via DruidCoordinatorUnloadUnusedSegments). Therefore the picked segmentToMoveHolder's diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java index 846a8f87d070..30c2215d2d89 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnloadUnusedSegments.java @@ -56,7 +56,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Unload segments that are no longer marked as used from historical servers, *if* the usedSegments collection has // been populated. Used segments might not have been loaded yet since it's done asynchronously (in - // SqlMetadataSegments). But it's also done atomically (see SqlMetadataSegments code), so if there are any segments + // SqlSegmentsMetadata). But it's also done atomically (see SqlSegmentsMetadata code), so if there are any segments // at all, we should have all of them. for (SortedSet serverHolders : cluster.getSortedHistoricalsByTier()) { for (ServerHolder serverHolder : serverHolders) { diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java index f8447d771e3d..eb4fa4a76a69 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java @@ -26,7 +26,7 @@ import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.joda.time.Interval; @@ -47,12 +47,12 @@ public class DruidCoordinatorUnusedSegmentsKiller implements DruidCoordinatorHel private long lastKillTime = 0; - private final MetadataSegments metadataSegments; + private final SegmentsMetadata segmentsMetadata; private final IndexingServiceClient indexingServiceClient; @Inject public DruidCoordinatorUnusedSegmentsKiller( - MetadataSegments metadataSegments, + SegmentsMetadata segmentsMetadata, IndexingServiceClient indexingServiceClient, DruidCoordinatorConfig config ) @@ -76,7 +76,7 @@ public DruidCoordinatorUnusedSegmentsKiller( this.maxSegmentsToKill ); - this.metadataSegments = metadataSegments; + this.segmentsMetadata = segmentsMetadata; this.indexingServiceClient = indexingServiceClient; } @@ -95,7 +95,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) Collection dataSourcesToKill = specificDataSourcesToKill; if (killAllDataSources) { - dataSourcesToKill = metadataSegments.retrieveAllDataSourceNames(); + dataSourcesToKill = segmentsMetadata.retrieveAllDataSourceNames(); } if (dataSourcesToKill != null && @@ -126,7 +126,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) @Nullable Interval findIntervalForKill(String dataSource, int limit) { - List unusedSegmentIntervals = metadataSegments.getUnusedSegmentIntervals( + List unusedSegmentIntervals = segmentsMetadata.getUnusedSegmentIntervals( dataSource, new Interval(DateTimes.EPOCH, DateTimes.nowUtc().minus(retainDuration)), limit diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java index ba28383f06bf..cbeffaeacd19 100644 --- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java +++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java @@ -39,7 +39,7 @@ import org.apache.druid.java.util.common.guava.FunctionalIterable; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -92,7 +92,7 @@ public class DataSourcesResource private static final Logger log = new Logger(DataSourcesResource.class); private final CoordinatorServerView serverInventoryView; - private final MetadataSegments metadataSegments; + private final SegmentsMetadata segmentsMetadata; private final MetadataRuleManager metadataRuleManager; private final IndexingServiceClient indexingServiceClient; private final AuthorizerMapper authorizerMapper; @@ -100,14 +100,14 @@ public class DataSourcesResource @Inject public DataSourcesResource( CoordinatorServerView serverInventoryView, - MetadataSegments metadataSegments, + SegmentsMetadata segmentsMetadata, MetadataRuleManager metadataRuleManager, @Nullable IndexingServiceClient indexingServiceClient, AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; - this.metadataSegments = metadataSegments; + this.segmentsMetadata = segmentsMetadata; this.metadataRuleManager = metadataRuleManager; this.indexingServiceClient = indexingServiceClient; this.authorizerMapper = authorizerMapper; @@ -173,7 +173,7 @@ public Response getDataSource( @ResourceFilters(DatasourceResourceFilter.class) public Response markAsUsedAllSegments(@PathParam("dataSourceName") final String dataSourceName) { - if (!metadataSegments.tryMarkAsUsedAllSegmentsInDataSource(dataSourceName)) { + if (!segmentsMetadata.tryMarkAsUsedAllSegmentsInDataSource(dataSourceName)) { return Response.serverError().build(); } @@ -231,7 +231,7 @@ public Response tryMarkAsUnusedAllSegmentsOrKillSegmentsInInterval( .build(); } } else { - if (!metadataSegments.tryMarkAsUnusedAllSegmentsInDataSource(dataSourceName)) { + if (!segmentsMetadata.tryMarkAsUnusedAllSegmentsInDataSource(dataSourceName)) { return Response.serverError().build(); } } @@ -426,7 +426,7 @@ public Response removeSegment( @PathParam("segmentId") String segmentId ) { - if (metadataSegments.tryMarkSegmentAsUnused(dataSourceName, segmentId)) { + if (segmentsMetadata.tryMarkSegmentAsUnused(dataSourceName, segmentId)) { return Response.ok().build(); } return Response.noContent().build(); @@ -441,7 +441,7 @@ public Response markSegmentAsUsed( @PathParam("segmentId") String segmentId ) { - if (!metadataSegments.tryMarkSegmentAsUsed(segmentId)) { + if (!segmentsMetadata.tryMarkSegmentAsUsed(segmentId)) { return Response.serverError().build(); } diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index a6460bfe419c..c4268b2e9818 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -31,7 +31,7 @@ import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.server.JettyUtils; import org.apache.druid.server.http.security.DatasourceResourceFilter; import org.apache.druid.server.security.AuthorizationUtils; @@ -67,20 +67,20 @@ @Path("/druid/coordinator/v1/metadata") public class MetadataResource { - private final MetadataSegments metadataSegments; + private final SegmentsMetadata segmentsMetadata; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthorizerMapper authorizerMapper; private final ObjectMapper jsonMapper; @Inject public MetadataResource( - MetadataSegments metadataSegments, + SegmentsMetadata segmentsMetadata, IndexerMetadataStorageCoordinator metadataStorageCoordinator, AuthorizerMapper authorizerMapper, @Json ObjectMapper jsonMapper ) { - this.metadataSegments = metadataSegments; + this.segmentsMetadata = segmentsMetadata; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; this.jsonMapper = jsonMapper; @@ -99,9 +99,9 @@ public Response getDataSources( Collection druidDataSources = null; final Set dataSourceNamesPreAuth; if (includeUnused != null) { - dataSourceNamesPreAuth = new TreeSet<>(metadataSegments.retrieveAllDataSourceNames()); + dataSourceNamesPreAuth = new TreeSet<>(segmentsMetadata.retrieveAllDataSourceNames()); } else { - druidDataSources = metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments(); + druidDataSources = segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); dataSourceNamesPreAuth = druidDataSources .stream() .map(ImmutableDruidDataSource::getName) @@ -142,7 +142,7 @@ public Response getAllUsedSegments( ) { Collection dataSourcesWithUsedSegments = - metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments(); + segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments(); if (dataSources != null && !dataSources.isEmpty()) { dataSourcesWithUsedSegments = dataSourcesWithUsedSegments .stream() @@ -186,7 +186,7 @@ public Response getAllUsedSegments( public Response getDataSourceWithUsedSegments(@PathParam("dataSourceName") final String dataSourceName) { ImmutableDruidDataSource dataSource = - metadataSegments.prepareImmutableDataSourceWithUsedSegments(dataSourceName); + segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -204,7 +204,7 @@ public Response getUsedSegmentsInDataSource( ) { ImmutableDruidDataSource dataSource = - metadataSegments.prepareImmutableDataSourceWithUsedSegments(dataSourceName); + segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -250,7 +250,7 @@ public Response isSegmentUsed( @PathParam("segmentId") String segmentId ) { - DruidDataSource dataSource = metadataSegments.getDataSourceWithUsedSegments(dataSourceName); + DruidDataSource dataSource = segmentsMetadata.getDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } diff --git a/server/src/test/java/org/apache/druid/metadata/SqlMetadataSegmentsTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java similarity index 97% rename from server/src/test/java/org/apache/druid/metadata/SqlMetadataSegmentsTest.java rename to server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java index 716338e77aa6..2b5da0ecf56f 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlMetadataSegmentsTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java @@ -41,12 +41,12 @@ import java.io.IOException; -public class SqlMetadataSegmentsTest +public class SqlSegmentsMetadataTest { @Rule public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private SqlMetadataSegments manager; + private SqlSegmentsMetadata manager; private SQLMetadataSegmentPublisher publisher; private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); @@ -86,9 +86,9 @@ public class SqlMetadataSegmentsTest public void setUp() throws Exception { TestDerbyConnector connector = derbyConnectorRule.getConnector(); - manager = new SqlMetadataSegments( + manager = new SqlSegmentsMetadata( jsonMapper, - Suppliers.ofInstance(new MetadataSegmentsConfig()), + Suppliers.ofInstance(new SegmentsMetadataConfig()), derbyConnectorRule.metadataTablesConfigSupplier(), connector ); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index d96a0f00cc58..58fb2cc7179b 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java @@ -45,7 +45,7 @@ import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.segment.TestHelper; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -82,7 +82,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase { private DruidCoordinator coordinator; - private MetadataSegments metadataSegments; + private SegmentsMetadata segmentsMetadata; private ScheduledExecutorFactory scheduledExecutorFactory; private ConcurrentMap loadManagementPeons; private LoadQueuePeon sourceLoadQueuePeon; @@ -124,7 +124,7 @@ public CuratorDruidCoordinatorTest() @Before public void setUp() throws Exception { - metadataSegments = EasyMock.createNiceMock(MetadataSegments.class); + segmentsMetadata = EasyMock.createNiceMock(SegmentsMetadata.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -211,7 +211,7 @@ public String getBase() } }, configManager, - metadataSegments, + segmentsMetadata, baseView, metadataRuleManager, curator, @@ -364,8 +364,8 @@ public void testMoveSegment() throws Exception ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(sourceSegments.get(2)); EasyMock.replay(druidDataSource); - EasyMock.expect(metadataSegments.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); - EasyMock.replay(metadataSegments); + EasyMock.expect(segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock.replay(segmentsMetadata); coordinator.moveSegment( source.toImmutableDruidServer(), @@ -494,7 +494,7 @@ public String getBase() } }, configManager, - metadataSegments, + segmentsMetadata, baseView, metadataRuleManager, curator, 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 9e86e77210ed..1ebce0edc0d1 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 @@ -47,7 +47,7 @@ import org.apache.druid.java.util.emitter.core.Event; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.ServerType; @@ -86,7 +86,7 @@ public class DruidCoordinatorTest extends CuratorTestBase private static final long COORDINATOR_PERIOD = 100; private DruidCoordinator coordinator; - private MetadataSegments metadataSegments; + private SegmentsMetadata segmentsMetadata; private SingleServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; @@ -106,7 +106,7 @@ public void setUp() throws Exception { druidServer = EasyMock.createMock(DruidServer.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); - metadataSegments = EasyMock.createNiceMock(MetadataSegments.class); + segmentsMetadata = EasyMock.createNiceMock(SegmentsMetadata.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); EasyMock.expect( @@ -182,7 +182,7 @@ public String getBase() } }, configManager, - metadataSegments, + segmentsMetadata, serverInventoryView, metadataRuleManager, curator, @@ -248,8 +248,8 @@ public void testMoveSegment() ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); EasyMock.replay(druidDataSource); - EasyMock.expect(metadataSegments.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); - EasyMock.replay(metadataSegments); + EasyMock.expect(segmentsMetadata.prepareImmutableDataSourceWithUsedSegments(EasyMock.anyString())).andReturn(druidDataSource); + EasyMock.replay(segmentsMetadata); scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); EasyMock.replay(scheduledExecutorFactory); EasyMock.replay(metadataRuleManager); @@ -320,7 +320,7 @@ public void testCoordinatorRun() throws Exception EasyMock.replay(metadataRuleManager); - // Setup MetadataSegments + // Setup SegmentsMetadata DruidDataSource[] dataSources = { new DruidDataSource(dataSource, Collections.emptyMap()) }; @@ -337,7 +337,7 @@ public void testCoordinatorRun() throws Exception ); dataSources[0].addSegment(dataSegment); - setupMetadataSegmentsMock(dataSources[0]); + setupSegmentsMetadataMock(dataSources[0]); ImmutableDruidDataSource immutableDruidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(immutableDruidDataSource.getSegments()) .andReturn(ImmutableSet.of(dataSegment)).atLeastOnce(); @@ -463,7 +463,7 @@ public void testCoordinatorTieredRun() throws Exception DruidDataSource[] druidDataSources = {new DruidDataSource(dataSource, Collections.emptyMap())}; dataSegments.values().forEach(druidDataSources[0]::addSegment); - setupMetadataSegmentsMock(druidDataSources[0]); + setupSegmentsMetadataMock(druidDataSources[0]); EasyMock.expect(metadataRuleManager.getRulesWithDefault(EasyMock.anyString())) .andReturn(ImmutableList.of(hotTier, coldTier)).atLeastOnce(); @@ -528,26 +528,26 @@ public void testCoordinatorTieredRun() throws Exception leaderUnannouncerLatch.await(); EasyMock.verify(serverInventoryView); - EasyMock.verify(metadataSegments); + EasyMock.verify(segmentsMetadata); EasyMock.verify(metadataRuleManager); } - private void setupMetadataSegmentsMock(DruidDataSource dataSource) + private void setupSegmentsMetadataMock(DruidDataSource dataSource) { - EasyMock.expect(metadataSegments.isStarted()).andReturn(true).anyTimes(); + EasyMock.expect(segmentsMetadata.isStarted()).andReturn(true).anyTimes(); EasyMock - .expect(metadataSegments.iterateAllUsedSegments()) + .expect(segmentsMetadata.iterateAllUsedSegments()) .andReturn(dataSource.getSegments()) .anyTimes(); EasyMock - .expect(metadataSegments.prepareImmutableDataSourcesWithAllUsedSegments()) + .expect(segmentsMetadata.prepareImmutableDataSourcesWithAllUsedSegments()) .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) .anyTimes(); EasyMock - .expect(metadataSegments.retrieveAllDataSourceNames()) + .expect(segmentsMetadata.retrieveAllDataSourceNames()) .andReturn(Collections.singleton(dataSource.getName())) .anyTimes(); - EasyMock.replay(metadataSegments); + EasyMock.replay(segmentsMetadata); } @Nullable diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java index 77b14267a214..580c6c5abee4 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java @@ -22,7 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.metadata.MetadataSegments; +import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.easymock.EasyMock; import org.joda.time.Duration; @@ -84,21 +84,21 @@ public void testFindIntervalForKill() ); } - private void testFindIntervalForKill(List metadataSegmentsResult, Interval expected) + private void testFindIntervalForKill(List segmentIntervals, Interval expected) { - MetadataSegments metadataSegments = EasyMock.createMock(MetadataSegments.class); + SegmentsMetadata segmentsMetadata = EasyMock.createMock(SegmentsMetadata.class); EasyMock.expect( - metadataSegments.getUnusedSegmentIntervals( + segmentsMetadata.getUnusedSegmentIntervals( EasyMock.anyString(), EasyMock.anyObject(Interval.class), EasyMock.anyInt() ) - ).andReturn(metadataSegmentsResult); - EasyMock.replay(metadataSegments); + ).andReturn(segmentIntervals); + EasyMock.replay(segmentsMetadata); IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class); DruidCoordinatorUnusedSegmentsKiller coordinatorSegmentKiller = new DruidCoordinatorUnusedSegmentsKiller( - metadataSegments, + segmentsMetadata, indexingServiceClient, new TestDruidCoordinatorConfig( null, 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 47452446d3ae..f637f7046aea 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -52,9 +52,9 @@ import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManagerConfig; import org.apache.druid.metadata.MetadataRuleManagerProvider; -import org.apache.druid.metadata.MetadataSegments; -import org.apache.druid.metadata.MetadataSegmentsConfig; -import org.apache.druid.metadata.MetadataSegmentsProvider; +import org.apache.druid.metadata.SegmentsMetadata; +import org.apache.druid.metadata.SegmentsMetadataConfig; +import org.apache.druid.metadata.SegmentsMetadataProvider; import org.apache.druid.metadata.MetadataStorage; import org.apache.druid.metadata.MetadataStorageProvider; import org.apache.druid.server.audit.AuditManagerProvider; @@ -145,7 +145,7 @@ public void configure(Binder binder) binder.bind(MetadataStorage.class) .toProvider(MetadataStorageProvider.class); - JsonConfigProvider.bind(binder, "druid.manager.segments", MetadataSegmentsConfig.class); + JsonConfigProvider.bind(binder, "druid.manager.segments", SegmentsMetadataConfig.class); JsonConfigProvider.bind(binder, "druid.manager.rules", MetadataRuleManagerConfig.class); JsonConfigProvider.bind(binder, "druid.manager.lookups", LookupCoordinatorManagerConfig.class); JsonConfigProvider.bind(binder, "druid.coordinator.balancer", BalancerStrategyFactory.class); @@ -163,8 +163,8 @@ public void configure(Binder binder) binder.bind(RedirectInfo.class).to(CoordinatorRedirectInfo.class).in(LazySingleton.class); } - binder.bind(MetadataSegments.class) - .toProvider(MetadataSegmentsProvider.class) + binder.bind(SegmentsMetadata.class) + .toProvider(SegmentsMetadataProvider.class) .in(ManageLifecycle.class); binder.bind(MetadataRuleManager.class) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java index 50fe3133cd28..35b96e7708a9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/MetadataSegmentView.java @@ -58,8 +58,13 @@ import java.util.concurrent.atomic.AtomicBoolean; /** - * This class polls the coordinator in background to keep the latest published segments. + * This class polls the Coordinator in background to keep the latest published segments. * Provides {@link #getPublishedSegments()} for others to get segments in metadata store. + * + * The difference between this class and {@link org.apache.druid.metadata.SegmentsMetadata} is that this class resides + * in Broker's memory, while {@link org.apache.druid.metadata.SegmentsMetadata} resides in Coordinator's memory. In + * fact, this class polls the data from {@link org.apache.druid.metadata.SegmentsMetadata} object in the memory of the + * currently leading Coordinator via HTTP queries. */ @ManageLifecycle public class MetadataSegmentView @@ -134,7 +139,7 @@ public void stop() private void poll() { log.info("polling published segments from coordinator"); - final JsonParserIterator metadataSegments = getMetadataSegments( + final JsonParserIterator segments = getSegments( coordinatorDruidLeaderClient, jsonMapper, responseHandler, @@ -142,8 +147,8 @@ private void poll() ); final DateTime timestamp = DateTimes.nowUtc(); - while (metadataSegments.hasNext()) { - final DataSegment interned = DataSegmentInterner.intern(metadataSegments.next()); + while (segments.hasNext()) { + final DataSegment interned = DataSegmentInterner.intern(segments.next()); // timestamp is used to filter deleted segments publishedSegments.put(interned, timestamp); } @@ -169,7 +174,7 @@ public Iterator getPublishedSegments() ); return publishedSegments.keySet().iterator(); } else { - return getMetadataSegments( + return getSegments( coordinatorDruidLeaderClient, jsonMapper, responseHandler, @@ -179,7 +184,7 @@ public Iterator getPublishedSegments() } // Note that coordinator must be up to get segments - private JsonParserIterator getMetadataSegments( + private JsonParserIterator getSegments( DruidLeaderClient coordinatorClient, ObjectMapper jsonMapper, BytesAccumulatingResponseHandler responseHandler, diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 6436b5bbe3fd..97025b94e54a 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -229,15 +229,12 @@ public Enumerable scan(DataContext root) } //get published segments from metadata segment cache (if enabled in sql planner config), else directly from coordinator - final Iterator metadataSegments = metadataView.getPublishedSegments(); + final Iterator segments = metadataView.getPublishedSegments(); final Set segmentsAlreadySeen = new HashSet<>(); final FluentIterable publishedSegments = FluentIterable - .from(() -> getAuthorizedPublishedSegments( - metadataSegments, - root - )) + .from(() -> getAuthorizedPublishedSegments(segments, root)) .transform(val -> { try { segmentsAlreadySeen.add(val.getId()); From f4774e198413f48cc47d71161c9a66bfd1086fbd Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 1 Apr 2019 15:31:01 -0300 Subject: [PATCH 11/40] Javadoc update --- .../coordinator/CoordinatorDynamicConfig.java | 21 +++++++++++-------- 1 file changed, 12 insertions(+), 9 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 b4c3177eefbf..178b49ae4f4d 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 @@ -58,11 +58,22 @@ public class CoordinatorDynamicConfig private final int balancerComputeThreads; private final boolean emitBalancingStats; private final boolean killUnusedSegmentsInAllDataSources; + + /** + * List of specific data sources for which kill tasks are sent in + * {@link DruidCoordinatorUnusedSegmentsKiller}. + */ private final Set specificDataSourcesToKillUnusedSegmentsIn; private final Set decommissioningNodes; private final int decommissioningMaxPercentOfMaxSegmentsToMove; - /** Stale pending segments belonging to the data sources in this list are not killed (they are "protected"). */ + /** + * Stale pending segments belonging to the data sources in this list are not killed by {@link + * DruidCoordinatorKillStalePendingSegments}. In other words, segments in these data sources are "protected". + * + * Pending segments are considered "stale" when their created_time is older than {@link + * DruidCoordinatorKillStalePendingSegments#KEEP_PENDING_SEGMENTS_OFFSET} from now. + */ private final Set dataSourcesToNotKillStalePendingSegmentsIn; /** @@ -214,10 +225,6 @@ public int getBalancerComputeThreads() return balancerComputeThreads; } - /** - * List of specific data sources for which kill tasks are sent in - * {@link DruidCoordinatorUnusedSegmentsKiller}. - */ @JsonProperty("killDataSourceWhitelist") public Set getSpecificDataSourcesToKillUnusedSegmentsIn() { @@ -230,10 +237,6 @@ public boolean isKillUnusedSegmentsInAllDataSources() return killUnusedSegmentsInAllDataSources; } - /** - * List of dataSources for which pendingSegments are NOT cleaned up - * in {@link DruidCoordinatorKillStalePendingSegments}. - */ @JsonProperty("killPendingSegmentsSkipList") public Set getDataSourcesToNotKillStalePendingSegmentsIn() { From 8d5200d3e227bbfb1441b15e4ba6f4abe380e0df Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 1 Apr 2019 15:47:05 -0300 Subject: [PATCH 12/40] Simplify SegmentsMetadata.getUnusedSegmentIntervals(), more javadocs --- .../apache/druid/metadata/SegmentsMetadata.java | 6 ++++-- .../apache/druid/metadata/SqlSegmentsMetadata.java | 14 +++++--------- .../coordinator/CoordinatorDynamicConfig.java | 5 +++-- .../DruidCoordinatorUnusedSegmentsKiller.java | 14 ++++++++------ .../druid/metadata/SqlSegmentsMetadataTest.java | 5 +++-- .../DruidCoordinatorUnusedSegmentsKillerTest.java | 3 ++- 6 files changed, 25 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java index f57ca14c46ba..78972fe3be99 100644 --- a/server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java +++ b/server/src/main/java/org/apache/druid/metadata/SegmentsMetadata.java @@ -24,6 +24,7 @@ import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; +import org.joda.time.DateTime; import org.joda.time.Interval; import javax.annotation.Nullable; @@ -115,9 +116,10 @@ public interface SegmentsMetadata Collection retrieveAllDataSourceNames(); /** - * Returns top N unused segment intervals in given interval when ordered by segment start time, end time. + * Returns top N unused segment intervals with the end time no later than the specified maxEndTime when ordered by + * segment start time, end time. */ - List getUnusedSegmentIntervals(String dataSource, Interval interval, int limit); + List getUnusedSegmentIntervals(String dataSource, DateTime maxEndTime, int limit); @VisibleForTesting void poll(); diff --git a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java index c5a59f914ae0..06a51523f133 100644 --- a/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java +++ b/server/src/main/java/org/apache/druid/metadata/SqlSegmentsMetadata.java @@ -42,6 +42,7 @@ import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.skife.jdbi.v2.BaseResultSetMapper; @@ -580,11 +581,7 @@ private String getSegmentsTable() } @Override - public List getUnusedSegmentIntervals( - final String dataSource, - final Interval interval, - final int limit - ) + public List getUnusedSegmentIntervals(final String dataSource, final DateTime maxEndTime, final int limit) { return connector.inReadOnlyTransaction( new TransactionCallback>() @@ -595,8 +592,8 @@ public List inTransaction(Handle handle, TransactionStatus status) Iterator iter = handle .createQuery( StringUtils.format( - "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource AND start >= :start " - + "AND %2$send%2$s <= :end AND used = false ORDER BY start, %2$send%2$s", + "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource AND " + + "%2$send%2$s <= :end AND used = false ORDER BY start, %2$send%2$s", getSegmentsTable(), connector.getQuoteString() ) @@ -604,8 +601,7 @@ public List inTransaction(Handle handle, TransactionStatus status) .setFetchSize(connector.getStreamingFetchSize()) .setMaxRows(limit) .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) + .bind("end", maxEndTime.toString()) .map( new BaseResultSetMapper() { 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 178b49ae4f4d..d83cdc97a4a8 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 @@ -57,11 +57,12 @@ public class CoordinatorDynamicConfig private final int replicationThrottleLimit; private final int balancerComputeThreads; private final boolean emitBalancingStats; + + /** If true, {@link DruidCoordinatorUnusedSegmentsKiller} sends kill tasks for unused segments in all data sources. */ private final boolean killUnusedSegmentsInAllDataSources; /** - * List of specific data sources for which kill tasks are sent in - * {@link DruidCoordinatorUnusedSegmentsKiller}. + * List of specific data sources for which kill tasks are sent in {@link DruidCoordinatorUnusedSegmentsKiller}. */ private final Set specificDataSourcesToKillUnusedSegmentsIn; private final Set decommissioningNodes; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java index eb4fa4a76a69..96562d1496ce 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKiller.java @@ -36,6 +36,10 @@ import java.util.List; /** + * Completely removes information about unused segments whose end time is older than {@link #retainDuration} from now + * from the metadata store. This action is called "to kill a segment". + * + * @see org.apache.druid.indexing.common.task.KillUnusedSegmentsTask */ public class DruidCoordinatorUnusedSegmentsKiller implements DruidCoordinatorHelper { @@ -84,7 +88,8 @@ public DruidCoordinatorUnusedSegmentsKiller( public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources(); - Collection specificDataSourcesToKill = params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn(); + Collection specificDataSourcesToKill = + params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn(); if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) { log.error( @@ -126,11 +131,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) @Nullable Interval findIntervalForKill(String dataSource, int limit) { - List unusedSegmentIntervals = segmentsMetadata.getUnusedSegmentIntervals( - dataSource, - new Interval(DateTimes.EPOCH, DateTimes.nowUtc().minus(retainDuration)), - limit - ); + List unusedSegmentIntervals = + segmentsMetadata.getUnusedSegmentIntervals(dataSource, DateTimes.nowUtc().minus(retainDuration), limit); if (unusedSegmentIntervals != null && unusedSegmentIntervals.size() > 0) { return JodaUtils.umbrellaInterval(unusedSegmentIntervals); diff --git a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java index 2b5da0ecf56f..57de4daf3dde 100644 --- a/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SqlSegmentsMetadataTest.java @@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; +import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -167,12 +168,12 @@ public void testGetUnusedSegmentsForInterval() Assert.assertEquals( ImmutableList.of(segment2.getInterval()), - manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 1) + manager.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 1) ); Assert.assertEquals( ImmutableList.of(segment2.getInterval(), segment1.getInterval()), - manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 5) + manager.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 5) ); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java index 580c6c5abee4..5d4fe2bfd329 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorUnusedSegmentsKillerTest.java @@ -25,6 +25,7 @@ import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.easymock.EasyMock; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.junit.Assert; @@ -90,7 +91,7 @@ private void testFindIntervalForKill(List segmentIntervals, Interval e EasyMock.expect( segmentsMetadata.getUnusedSegmentIntervals( EasyMock.anyString(), - EasyMock.anyObject(Interval.class), + EasyMock.anyObject(DateTime.class), EasyMock.anyInt() ) ).andReturn(segmentIntervals); From f34eef04beba9f809b44de1f0e93b3ba09f5703a Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 9 Apr 2019 20:31:40 +0300 Subject: [PATCH 13/40] Update Javadoc of VersionedIntervalTimeline.iterateAllObjects() --- .../org/apache/druid/timeline/VersionedIntervalTimeline.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java index 8d370019efa4..2b295c4f5108 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -123,8 +123,9 @@ public Map> getAllTimelineEntries( } /** - * Returns a lazy collection with all objects in this VersionedIntervalTimeline to be used for iteration or {@link - * Collection#stream()} transformation. The order of objects in this collection is unspecified. + * Returns a lazy collection with all objects (including overshadowed, see {@link #findOvershadowed}) in this + * VersionedIntervalTimeline to be used for iteration or {@link Collection#stream()} transformation. The order of + * objects in this collection is unspecified. * * Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only From 2ebb23ca0cb6ae57a897b511c43033c76913594c Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 9 Apr 2019 21:21:44 +0300 Subject: [PATCH 14/40] Reorder imports --- .../druid/indexing/common/actions/TaskActionTestKit.java | 4 ++-- .../src/main/java/org/apache/druid/cli/CliCoordinator.java | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java index f3ba60b5913b..f37748ae955a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java @@ -29,10 +29,10 @@ import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; -import org.apache.druid.metadata.SegmentsMetadata; -import org.apache.druid.metadata.SegmentsMetadataConfig; import org.apache.druid.metadata.MetadataStorageConnectorConfig; import org.apache.druid.metadata.MetadataStorageTablesConfig; +import org.apache.druid.metadata.SegmentsMetadata; +import org.apache.druid.metadata.SegmentsMetadataConfig; import org.apache.druid.metadata.SqlSegmentsMetadata; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.server.metrics.NoopServiceEmitter; 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 f637f7046aea..74c79751ece0 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -52,17 +52,17 @@ import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManagerConfig; import org.apache.druid.metadata.MetadataRuleManagerProvider; +import org.apache.druid.metadata.MetadataStorage; +import org.apache.druid.metadata.MetadataStorageProvider; import org.apache.druid.metadata.SegmentsMetadata; import org.apache.druid.metadata.SegmentsMetadataConfig; import org.apache.druid.metadata.SegmentsMetadataProvider; -import org.apache.druid.metadata.MetadataStorage; -import org.apache.druid.metadata.MetadataStorageProvider; import org.apache.druid.server.audit.AuditManagerProvider; import org.apache.druid.server.coordinator.BalancerStrategyFactory; import org.apache.druid.server.coordinator.CachingCostBalancerStrategyConfig; import org.apache.druid.server.coordinator.DruidCoordinator; -import org.apache.druid.server.coordinator.DruidCoordinatorKillStalePendingSegments; import org.apache.druid.server.coordinator.DruidCoordinatorConfig; +import org.apache.druid.server.coordinator.DruidCoordinatorKillStalePendingSegments; import org.apache.druid.server.coordinator.LoadQueueTaskMaster; import org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper; import org.apache.druid.server.coordinator.helper.DruidCoordinatorUnusedSegmentsKiller; From ba8ed6212f248af892e34076d35cd051556a53e6 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Fri, 19 Apr 2019 12:59:21 +0200 Subject: [PATCH 15/40] Rename SegmentsMetadata.tryMark... methods to mark... and make them to return boolean and the numbers of segments changed and relay exceptions to callers --- .idea/misc.xml | 11 +- docs/content/operations/api-reference.md | 16 +- .../MaterializedViewSupervisor.java | 6 +- .../actions/SegmentListActionsTest.java | 2 +- .../org/apache/druid/client/DruidServer.java | 2 - .../druid/metadata/SegmentsMetadata.java | 39 +++- .../druid/metadata/SqlSegmentsMetadata.java | 177 ++++++++---------- .../server/coordinator/DruidCoordinator.java | 6 +- ...natorMarkAsUnusedOvershadowedSegments.java | 2 +- .../server/coordinator/rules/DropRule.java | 2 +- .../server/http/DataSourcesResource.java | 84 +++------ .../metadata/SqlSegmentsMetadataTest.java | 74 ++++---- .../DruidCoordinatorRuleRunnerTest.java | 6 +- ...rMarkAsUnusedOvershadowedSegmentsTest.java | 4 +- .../server/http/DataSourcesResourceTest.java | 2 +- 15 files changed, 204 insertions(+), 229 deletions(-) diff --git a/.idea/misc.xml b/.idea/misc.xml index 9d0b0220e000..ed56a01147cf 100644 --- a/.idea/misc.xml +++ b/.idea/misc.xml @@ -31,6 +31,9 @@ + +