diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml index 4de1e05a5b69..bba3782b8547 100644 --- a/.idea/inspectionProfiles/Druid.xml +++ b/.idea/inspectionProfiles/Druid.xml @@ -311,6 +311,11 @@ + + + + + diff --git a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java index d06125ac35df..dfef1283a2a3 100644 --- a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java @@ -20,6 +20,8 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator; import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy; @@ -42,6 +44,7 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.infra.Blackhole; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -102,14 +105,10 @@ public void setup() ); } - dataSources = new HashMap<>(); + List segments = new ArrayList<>(); for (int i = 0; i < numDataSources; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - final int startYear = ThreadLocalRandom.current().nextInt(2000, 2040); DateTime date = DateTimes.of(startYear, 1, 1, 0, 0); @@ -127,12 +126,11 @@ public void setup() 0, segmentSizeBytes ); - timeline.add(segment.getInterval(), segment.getVersion(), shardSpec.createChunk(segment)); + segments.add(segment); } } - - dataSources.put(dataSource, timeline); } + dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()).getUsedSegmentsTimelinesPerDataSource(); } @Benchmark diff --git a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java index d5aa17ada9c5..569537a18a88 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java +++ b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java @@ -123,6 +123,11 @@ public void error(Throwable t, String message, Object... formatArgs) log.error(StringUtils.nonStrictFormat(message, formatArgs), t); } + public void assertionError(String message, Object... formatArgs) + { + log.error("ASSERTION_ERROR: " + message, formatArgs); + } + public void wtf(String message, Object... formatArgs) { log.error(StringUtils.nonStrictFormat("WTF?!: " + message, formatArgs), new Exception()); diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java index 8e685c696117..ef5d47a1f4b5 100644 --- a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java +++ b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder; import org.apache.druid.java.util.emitter.service.ServiceEmitter; +import javax.annotation.Nullable; import java.io.PrintWriter; import java.io.StringWriter; @@ -61,15 +62,22 @@ public AlertBuilder makeAlert(String message, Object... objects) return makeAlert(null, message, objects); } - public AlertBuilder makeAlert(Throwable t, String message, Object... objects) + public AlertBuilder makeAlert(@Nullable Throwable t, String message, Object... objects) { if (emitter == null) { final String errorMessage = StringUtils.format( - "Emitter not initialized! Cannot alert. Please make sure to call %s.registerEmitter()", this.getClass() + "Emitter not initialized! Cannot alert. Please make sure to call %s.registerEmitter()\n" + + "Message: %s", + this.getClass(), + StringUtils.nonStrictFormat(message, objects) ); error(errorMessage); - throw new ISE(errorMessage); + ISE e = new ISE(errorMessage); + if (t != null) { + e.addSuppressed(t); + } + throw e; } final AlertBuilder retVal = new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter) diff --git a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java index af7544083756..6bdab5c723de 100644 --- a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java +++ b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java @@ -22,6 +22,7 @@ import org.apache.druid.timeline.partition.PartitionHolder; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.List; @@ -50,5 +51,5 @@ public interface TimelineLookup */ List> lookupWithIncompletePartitions(Interval interval); - PartitionHolder findEntry(Interval interval, VersionType version); + @Nullable PartitionHolder findEntry(Interval interval, VersionType version); } 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..8deed36fd7e6 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -23,16 +23,18 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; -import com.google.common.collect.Ordering; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Comparators; 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,7 +46,9 @@ 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. @@ -75,12 +79,11 @@ public class VersionedIntervalTimeline implements Timel Comparators.intervalsByStartThenEnd() ); private final Map> allTimelineEntries = new HashMap<>(); + private final AtomicInteger numObjects = new AtomicInteger(); private final Comparator versionComparator; - public VersionedIntervalTimeline( - Comparator versionComparator - ) + public VersionedIntervalTimeline(Comparator versionComparator) { this.versionComparator = versionComparator; } @@ -92,7 +95,8 @@ public static VersionedIntervalTimeline forSegments(Iterabl public static VersionedIntervalTimeline forSegments(Iterator segments) { - final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); + final VersionedIntervalTimeline timeline = + new VersionedIntervalTimeline<>(Comparator.naturalOrder()); addSegments(timeline, segments); return timeline; } @@ -115,6 +119,28 @@ public Map> getAllTimelineEntries( return allTimelineEntries; } + /** + * 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 + * 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 void add(final Interval interval, VersionType version, PartitionChunk object) { addAll(Iterators.singletonIterator(object), o -> interval, o -> version); @@ -143,15 +169,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 +204,7 @@ private void addAll( } } + @Nullable public PartitionChunk remove(Interval interval, VersionType version, PartitionChunk chunk) { try { @@ -189,7 +220,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 +236,7 @@ public PartitionChunk remove(Interval interval, VersionType version, remove(completePartitionsTimeline, interval, entry, false); - return retVal; + return removedChunk; } finally { lock.writeLock().unlock(); @@ -209,7 +244,7 @@ public PartitionChunk remove(Interval interval, VersionType version, } @Override - public PartitionHolder findEntry(Interval interval, VersionType version) + public @Nullable PartitionHolder findEntry(Interval interval, VersionType version) { try { lock.readLock().lock(); @@ -217,9 +252,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()); } } } @@ -307,6 +340,10 @@ private TimelineObjectHolder timelineEntryToObjectHolde ); } + /** + * This method should be deduplicated with DataSourcesSnapshot.determineOvershadowedSegments(): see + * https://github.com/apache/incubator-druid/issues/8070. + */ public Set> findOvershadowed() { try { @@ -315,8 +352,8 @@ public Set> findOvershadowed() Map> overShadowed = new HashMap<>(); for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) { - Map versionCopy = new HashMap<>(); - versionCopy.putAll(versionEntry.getValue()); + @SuppressWarnings("unchecked") + Map versionCopy = (TreeMap) versionEntry.getValue().clone(); overShadowed.put(versionEntry.getKey(), versionCopy); } 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/main/java/org/apache/druid/utils/CollectionUtils.java b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java index af3cf077f430..deb4eafd0466 100644 --- a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java +++ b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java @@ -19,13 +19,16 @@ package org.apache.druid.utils; +import com.google.common.collect.Iterables; import com.google.common.collect.Maps; import java.util.AbstractCollection; import java.util.Collection; +import java.util.Comparator; import java.util.Iterator; import java.util.Map; import java.util.Spliterator; +import java.util.TreeSet; import java.util.function.Function; import java.util.function.Supplier; import java.util.stream.Stream; @@ -72,6 +75,13 @@ public int size() }; } + public static TreeSet newTreeSet(Comparator comparator, Iterable elements) + { + TreeSet set = new TreeSet<>(comparator); + Iterables.addAll(set, elements); + return set; + } + /** * Returns a transformed map from the given input map where the value is modified based on the given valueMapper * function. diff --git a/core/src/main/java/org/apache/druid/utils/package-info.java b/core/src/main/java/org/apache/druid/utils/package-info.java new file mode 100644 index 000000000000..f41e226c8160 --- /dev/null +++ b/core/src/main/java/org/apache/druid/utils/package-info.java @@ -0,0 +1,23 @@ +/* + * 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. + */ + +@EverythingIsNonnullByDefault +package org.apache.druid.utils; + +import org.apache.druid.annotations.EverythingIsNonnullByDefault; 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..3e66bf53de79 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") ); } @@ -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/ingestion/delete-data.md b/docs/content/ingestion/delete-data.md index 7e21e99bcc95..181fddb0ec63 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 3b49b12c5b1d..716d5e39f74d 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}` @@ -229,11 +231,15 @@ Caution : Avoid using indexing or kill tasks and these API's at the same time fo * `/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. Returns a JSON object of the form +`{"numChangedSegments": }` with the number of segments in the database whose state has been changed (that is, +the segments were marked as used) as the result of this API call. * `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}` -Enables a segment of a datasource. +Marks as used a segment of a data source. Returns a JSON object of the form `{"segmentStateChanged": }` with +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as used) as the +result of this API call. * `/druid/coordinator/v1/datasources/{dataSourceName}/markUsed` @@ -259,7 +265,9 @@ JSON Request Payload: * `/druid/coordinator/v1/datasources/{dataSourceName}` -Disables a datasource. +Marks as unused all segments belonging to a data source. Returns a JSON object of the form +`{"numChangedSegments": }` with the number of segments in the database whose state has been changed (that is, +the segments were marked as unused) as the result of this API call. * `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}` * `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}` @@ -268,7 +276,9 @@ 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. Returns a JSON object of the form `{"segmentStateChanged": }` with +the boolean indicating if the state of the segment has been changed (that is, the segment was marked as unused) as the +result of this API call. #### Retention Rules @@ -595,7 +605,7 @@ Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_` * `/druid/indexer/v1/worker` -Retreives current overlord dynamic configuration. +Retrieves current overlord dynamic configuration. * `/druid/indexer/v1/worker/history?interval={interval}&counter={count}` 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 76883ea3f891..1ed80ca83c9a 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 @@ -390,7 +390,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().toString()); + segmentManager.markSegmentAsUnused(segment.getId().toString()); } } // data of the latest interval will be built firstly. @@ -498,7 +498,7 @@ private void clearSegments() { log.info("Clear all metadata of dataSource %s", dataSource); metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL); - segmentManager.removeDataSource(dataSource); + segmentManager.markAsUnusedAllSegmentsInDataSource(dataSource); metadataStorageCoordinator.deleteDataSourceMetadata(dataSource); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java index e766effb4125..07a507af140d 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java @@ -91,6 +91,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.utils.CircularBuffer; +import org.apache.druid.utils.CollectionUtils; import org.codehaus.plexus.util.FileUtils; import org.joda.time.DateTime; import org.joda.time.Interval; @@ -444,8 +445,7 @@ public TaskStatus run(final TaskToolbox toolbox) toolbox.getTaskActionClient(), intervals ); - versions = locks.entrySet().stream() - .collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion())); + versions = CollectionUtils.mapValues(locks, TaskLock::getVersion); dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( ingestionSchema.getDataSchema() 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/KillTask.java index 44c90bc8a47b..ac15e67e74b1 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/KillTask.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.ClientKillQuery; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; @@ -37,6 +38,9 @@ import java.util.Map; /** + * The client representation of this task is {@link ClientKillQuery}. + * JSON serialization fields of this class must correspond to those of {@link + * ClientKillQuery}, except for "id" and "context" fields. */ public class KillTask extends AbstractFixedIntervalTask { 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 fa30dde802b8..c5f677121edf 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,9 @@ public void setup() throws IOException expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); - expectedUnusedSegments.forEach(s -> actionTestKit.getMetadataSegmentManager().removeSegment(s.getId().toString())); + expectedUnusedSegments.forEach( + s -> actionTestKit.getMetadataSegmentManager().markSegmentAsUnused(s.getId().toString()) + ); } private DataSegment createSegment(Interval interval, String version) diff --git a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java index 84176621e4b7..865b8bd0363a 100644 --- a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java +++ b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java @@ -19,11 +19,13 @@ package org.apache.druid.client; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Ordering; +import com.google.common.collect.Maps; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; +import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.util.ArrayList; @@ -31,40 +33,88 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** - * An immutable snapshot of fields from {@link org.apache.druid.metadata.SQLMetadataSegmentManager} (dataSources and - * overshadowedSegments). Getters of {@link org.apache.druid.metadata.MetadataSegmentManager} should use this snapshot - * to return dataSources and overshadowedSegments. + * An immutable snapshot information about used segments and overshadowed segments for + * {@link org.apache.druid.metadata.SQLMetadataSegmentManager}. */ public class DataSourcesSnapshot { - private final Map dataSources; + public static DataSourcesSnapshot fromUsedSegments( + Iterable segments, + ImmutableMap dataSourceProperties + ) + { + Map dataSources = new HashMap<>(); + segments.forEach(segment -> { + dataSources + .computeIfAbsent(segment.getDataSource(), dsName -> new DruidDataSource(dsName, dataSourceProperties)) + .addSegmentIfAbsent(segment); + }); + return new DataSourcesSnapshot(CollectionUtils.mapValues(dataSources, DruidDataSource::toImmutableDruidDataSource)); + } + + public static DataSourcesSnapshot fromUsedSegmentsTimelines( + Map> usedSegmentsTimelinesPerDataSource, + ImmutableMap dataSourceProperties + ) + { + Map dataSourcesWithAllUsedSegments = + Maps.newHashMapWithExpectedSize(usedSegmentsTimelinesPerDataSource.size()); + usedSegmentsTimelinesPerDataSource.forEach( + (dataSourceName, usedSegmentsTimeline) -> { + DruidDataSource dataSource = new DruidDataSource(dataSourceName, dataSourceProperties); + usedSegmentsTimeline.iterateAllObjects().forEach(dataSource::addSegment); + dataSourcesWithAllUsedSegments.put(dataSourceName, dataSource.toImmutableDruidDataSource()); + } + ); + return new DataSourcesSnapshot(dataSourcesWithAllUsedSegments, usedSegmentsTimelinesPerDataSource); + } + + private final Map dataSourcesWithAllUsedSegments; + private final Map> usedSegmentsTimelinesPerDataSource; private final ImmutableSet overshadowedSegments; - public DataSourcesSnapshot( - Map dataSources + public DataSourcesSnapshot(Map dataSourcesWithAllUsedSegments) + { + this( + dataSourcesWithAllUsedSegments, + CollectionUtils.mapValues( + dataSourcesWithAllUsedSegments, + dataSource -> VersionedIntervalTimeline.forSegments(dataSource.getSegments()) + ) + ); + } + + private DataSourcesSnapshot( + Map dataSourcesWithAllUsedSegments, + Map> usedSegmentsTimelinesPerDataSource ) { - this.dataSources = dataSources; + this.dataSourcesWithAllUsedSegments = dataSourcesWithAllUsedSegments; + this.usedSegmentsTimelinesPerDataSource = usedSegmentsTimelinesPerDataSource; this.overshadowedSegments = ImmutableSet.copyOf(determineOvershadowedSegments()); } - public Collection getDataSources() + public Collection getDataSourcesWithAllUsedSegments() { - return dataSources.values(); + return dataSourcesWithAllUsedSegments.values(); } public Map getDataSourcesMap() { - return dataSources; + return dataSourcesWithAllUsedSegments; } @Nullable public ImmutableDruidDataSource getDataSource(String dataSourceName) { - return dataSources.get(dataSourceName); + return dataSourcesWithAllUsedSegments.get(dataSourceName); + } + + public Map> getUsedSegmentsTimelinesPerDataSource() + { + return usedSegmentsTimelinesPerDataSource; } public ImmutableSet getOvershadowedSegments() @@ -72,40 +122,48 @@ public ImmutableSet getOvershadowedSegments() return overshadowedSegments; } - @Nullable - public Iterable iterateAllSegmentsInSnapshot() + /** + * 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. + * + * This method's name starts with "iterate" because the result is expected to be consumed immediately in a for-each + * statement or a stream pipeline, like + * for (DataSegment segment : snapshot.iterateAllUsedSegmentsInSnapshot()) {...} + */ + public Iterable iterateAllUsedSegmentsInSnapshot() { - if (dataSources == null) { - return null; - } - return () -> dataSources.values().stream() - .flatMap(dataSource -> dataSource.getSegments().stream()) - .iterator(); + return () -> dataSourcesWithAllUsedSegments + .values() + .stream() + .flatMap(dataSource -> dataSource.getSegments().stream()) + .iterator(); } /** - * This method builds timelines from all dataSources and finds the overshadowed segments list + * This method builds timelines from all data sources and finds the overshadowed segments list + * + * This method should be deduplicated with {@link VersionedIntervalTimeline#findOvershadowed()}: see + * https://github.com/apache/incubator-druid/issues/8070. * * @return overshadowed segment Ids list */ private List determineOvershadowedSegments() { - final List segments = dataSources.values().stream() - .flatMap(ds -> ds.getSegments().stream()) - .collect(Collectors.toList()); - final Map> timelines = new HashMap<>(); - segments.forEach(segment -> timelines - .computeIfAbsent(segment.getDataSource(), dataSource -> new VersionedIntervalTimeline<>(Ordering.natural())) - .add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment))); - // It's fine to add all overshadowed segments to a single collection because only // a small fraction of the segments in the cluster are expected to be overshadowed, // so building this collection shouldn't generate a lot of garbage. final List overshadowedSegments = new ArrayList<>(); - for (DataSegment dataSegment : segments) { - final VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { - overshadowedSegments.add(dataSegment.getId()); + for (ImmutableDruidDataSource dataSource : dataSourcesWithAllUsedSegments.values()) { + VersionedIntervalTimeline usedSegmentsTimeline = + usedSegmentsTimelinesPerDataSource.get(dataSource.getName()); + for (DataSegment segment : dataSource.getSegments()) { + if (usedSegmentsTimeline.isOvershadowed(segment.getInterval(), segment.getVersion())) { + overshadowedSegments.add(segment.getId()); + } } } return overshadowedSegments; diff --git a/server/src/main/java/org/apache/druid/client/DruidDataSource.java b/server/src/main/java/org/apache/druid/client/DruidDataSource.java index 7678dad68377..c293523b0797 100644 --- a/server/src/main/java/org/apache/druid/client/DruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/DruidDataSource.java @@ -30,12 +30,13 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.Predicate; /** * A mutable collection of metadata of segments ({@link DataSegment} objects), belonging to a particular data source. * - * Concurrency: could be updated concurrently via {@link #addSegment} and {@link #removeSegment}, and accessed - * concurrently (e. g. via {@link #getSegments}) as well. + * Concurrency: could be updated concurrently via {@link #addSegment}, {@link #removeSegment}, and {@link + * #removeSegmentsIf}, and accessed concurrently (e. g. via {@link #getSegments}) as well. * * @see ImmutableDruidDataSource - an immutable counterpart of this class */ @@ -44,10 +45,7 @@ public class DruidDataSource private final String name; private final Map properties; /** - * This map needs to be concurrent because it should be possible to iterate the segments of the data source - * (indirectly via {@link #getSegments} or in {@link #toString}) concurrently updates via {@link #addSegment} or - * {@link #removeSegment}. Concurrent updates are also supported incidentally, though this is not needed for the use - * cases of DruidDataSource. + * This map needs to be concurrent to support concurrent iteration and updates. */ private final ConcurrentMap idToSegmentMap = new ConcurrentHashMap<>(); @@ -80,6 +78,14 @@ public Collection getSegments() return Collections.unmodifiableCollection(idToSegmentMap.values()); } + /** + * Removes segments for which the given filter returns true. + */ + public void removeSegmentsIf(Predicate filter) + { + idToSegmentMap.values().removeIf(filter); + } + public DruidDataSource addSegment(DataSegment dataSegment) { idToSegmentMap.put(dataSegment.getId(), dataSegment); @@ -99,7 +105,7 @@ public boolean addSegmentIfAbsent(DataSegment dataSegment) * Returns the removed segment, or null if there was no segment with the given {@link SegmentId} in this * DruidDataSource. */ - public DataSegment removeSegment(SegmentId segmentId) + public @Nullable DataSegment removeSegment(SegmentId segmentId) { return idToSegmentMap.remove(segmentId); } @@ -126,7 +132,6 @@ public String toString() @Override public boolean equals(Object o) { - //noinspection Contract throw new UnsupportedOperationException("Use ImmutableDruidDataSource instead"); } diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java index ac73d63febf3..d3bcb0dedcee 100644 --- a/server/src/main/java/org/apache/druid/client/DruidServer.java +++ b/server/src/main/java/org/apache/druid/client/DruidServer.java @@ -232,7 +232,6 @@ public DataSegment removeDataSegment(SegmentId segmentId) segmentId ); // Returning null from the lambda here makes the ConcurrentHashMap to not record any entry. - //noinspection ReturnOfNull return null; } DataSegment segment = dataSource.removeSegment(segmentId); @@ -244,7 +243,6 @@ public DataSegment removeDataSegment(SegmentId segmentId) log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", getName(), segmentId); } // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry. - //noinspection ReturnOfNull return dataSource.isEmpty() ? null : dataSource; } ); 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..7ce3fd6dffb9 100644 --- a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java +++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java @@ -123,6 +123,8 @@ public String toString() @Override public boolean equals(Object o) { + // Note: this method is not well-defined. It should instead just throw UnsupportedOperationsException. + // See https://github.com/apache/incubator-druid/issues/7858. if (this == o) { return true; } @@ -146,6 +148,8 @@ public boolean equals(Object o) @Override public int hashCode() { - return Objects.hash(name, properties, idToSegments); + // Note: this method is not well-defined. It should instead just throw UnsupportedOperationsException. + // See https://github.com/apache/incubator-druid/issues/7858. + 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/indexing/ClientCompactQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java index 695297dd2142..1a09c451e14d 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java @@ -29,6 +29,10 @@ import java.util.Map; import java.util.Objects; +/** + * Client representation of org.apache.druid.indexing.common.task.CompactionTask. JSON serialization fields of + * this class must correspond to those of org.apache.druid.indexing.common.task.CompactionTask. + */ public class ClientCompactQuery implements ClientQuery { private final String dataSource; @@ -136,7 +140,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactQuery{" + + return getClass().getSimpleName() + "{" + "dataSource='" + dataSource + '\'' + ", segments=" + segments + ", interval=" + interval + diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java index 068016952b4d..14cf45b55a8e 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java @@ -43,17 +43,17 @@ public class ClientCompactQueryTuningConfig private final Long pushTimeout; public static ClientCompactQueryTuningConfig from( - @Nullable UserCompactTuningConfig userCompactTuningConfig, + @Nullable UserCompactTuningConfig userCompactionTaskQueryTuningConfig, @Nullable Integer maxRowsPerSegment ) { return new ClientCompactQueryTuningConfig( maxRowsPerSegment, - userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxRowsInMemory(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxTotalRows(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getIndexSpec(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxPendingPersists(), - userCompactTuningConfig == null ? null : userCompactTuningConfig.getPushTimeout() + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxRowsInMemory(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxTotalRows(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getIndexSpec(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxPendingPersists(), + userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getPushTimeout() ); } @@ -150,7 +150,7 @@ public int hashCode() @Override public String toString() { - return "ClientCompactQueryTuningConfig{" + + return getClass().getSimpleName() + "{" + "maxRowsPerSegment=" + maxRowsPerSegment + ", maxRowsInMemory=" + maxRowsInMemory + ", maxTotalRows=" + maxTotalRows + diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java index 06d88f9535a3..583dee506cda 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java @@ -24,6 +24,9 @@ import org.joda.time.Interval; /** + * Client representation of org.apache.druid.indexing.common.task.KillTask. JSON searialization + * fields of this class must correspond to those of + * org.apache.druid.indexing.common.task.KillTask, except for "id" and "context" fields. */ public class ClientKillQuery implements ClientQuery { diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java index aaa8b5c3ce3a..306d6e7c0cb2 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java @@ -24,7 +24,12 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; /** - * org.apache.druid.indexing.common.task.Task representation for clients + * 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 org.apache.druid.indexing.overlord.http.OverlordResource.taskPost() deserializes + * 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 + * org.apache.druid.indexing.common.task.Task objects. */ @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonSubTypes(value = { diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index db1fbef2499a..550dd49b0bf9 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.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; @@ -32,89 +33,105 @@ import java.util.Set; /** + * The difference between this class and org.apache.druid.sql.calcite.schema.MetadataSegmentView is that this + * class resides in Coordinator's memory, while org.apache.druid.sql.calcite.schema.MetadataSegmentView resides + * in Broker's memory. */ public interface MetadataSegmentManager { - void start(); + void startPollingDatabasePeriodically(); - void stop(); + void stopPollingDatabasePeriodically(); + + boolean isPollingDatabasePeriodically(); /** - * Enables all segments for a dataSource which will not be overshadowed. + * Returns the number of segment entries in the database whose state was changed as the result of this call (that is, + * the segments were marked as used). If the call results in a database error, an exception is relayed to the caller. */ - boolean enableDataSource(String dataSource); + int markAsUsedAllNonOvershadowedSegmentsInDataSource(String dataSource); + + int markAsUsedNonOvershadowedSegmentsInInterval(String dataSource, Interval interval); - boolean enableSegment(String segmentId); + int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) + throws UnknownSegmentIdException; /** - * Enables all segments contained in the interval which are not overshadowed by any currently enabled segments. + * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the + * segment was marked as used), false otherwise. If the call results in a database error, an exception is relayed to + * the caller. */ - int enableSegments(String dataSource, Interval interval); + boolean markSegmentAsUsed(String segmentId); /** - * Enables the segments passed which are not overshadowed by any currently enabled segments. + * Returns the number of segment entries in the database whose state was changed as the result of this call (that is, + * the segments were marked as unused). If the call results in a database error, an exception is relayed to the + * caller. */ - int enableSegments(String dataSource, Collection segmentIds); + int markAsUnusedAllSegmentsInDataSource(String dataSource); + + int markAsUnusedSegmentsInInterval(String dataSource, Interval interval); - boolean removeDataSource(String dataSource); + int markSegmentsAsUnused(String dataSource, Set segmentIds); /** - * Removes the given segmentId from metadata store. Returns true if one or more rows were affected. + * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the + * segment was marked as unused), false otherwise. If the call results in a database error, an exception is relayed to + * the caller. */ - boolean removeSegment(String segmentId); + boolean markSegmentAsUnused(String segmentId); - long disableSegments(String dataSource, Collection segmentIds); - - int disableSegments(String dataSource, Interval interval); + /** + * If there are used segments belonging to the given data source this method returns them as an {@link + * ImmutableDruidDataSource} object. If there are no used segments belonging to the given data source this method + * returns null. + */ + @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSource); - boolean isStarted(); + /** + * Returns 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. + */ + Collection getImmutableDataSourcesWithAllUsedSegments(); - @Nullable - ImmutableDruidDataSource getDataSource(String dataSourceName); + /** + * Returns a set of overshadowed segment ids. + */ + Set getOvershadowedSegments(); /** - * Returns a collection of known datasources. - * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) + * Returns a snapshot of DruidDataSources and overshadowed segments */ - @Nullable - Collection getDataSources(); + DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments(); /** * 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. - * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) */ - @Nullable - Iterable iterateAllSegments(); - - Collection getAllDataSourceNames(); + Iterable iterateAllUsedSegments(); /** - * Returns a set of overshadowed segment Ids + * 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. * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) - */ - @Nullable - Set getOvershadowedSegments(); - - /** - * Returns a snapshot of DruidDataSources and overshadowed segments + * 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 + * #getImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there + * are no used segments belonging to it, while {@link #getImmutableDataSourcesWithAllUsedSegments} won't return + * such a data source. */ - @Nullable - DataSourcesSnapshot getDataSourcesSnapshot(); + 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/MetadataSegmentManagerConfig.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java index 3f58cfa70982..d6c881198a89 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java @@ -33,4 +33,9 @@ public Period getPollDuration() { return pollDuration; } + + public void setPollDuration(Period pollDuration) + { + this.pollDuration = pollDuration; + } } 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..1f47e44811c7 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java @@ -200,7 +200,7 @@ public void run() { try { // poll() is synchronized together with start() and stop() to ensure that when stop() exits, poll() - // won't actually run anymore after that (it could only enter the syncrhonized section and exit + // won't actually run anymore after that (it could only enter the synchronized section and exit // immediately because the localStartedOrder doesn't match the new currentStartOrder). It's needed // to avoid flakiness in SQLMetadataRuleManagerTest. // See https://github.com/apache/incubator-druid/issues/6028 diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index bb8514291df1..db1d19e49ae9 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -20,16 +20,20 @@ package org.apache.druid.metadata; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.base.Supplier; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Futures; +import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.MapUtils; import org.apache.druid.java.util.common.Pair; @@ -41,18 +45,18 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.utils.CollectionUtils; +import org.checkerframework.checker.nullness.qual.MonotonicNonNull; +import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Interval; import org.skife.jdbi.v2.BaseResultSetMapper; import org.skife.jdbi.v2.Batch; import org.skife.jdbi.v2.FoldController; -import org.skife.jdbi.v2.Folder3; import org.skife.jdbi.v2.Handle; +import org.skife.jdbi.v2.Query; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.TransactionCallback; import org.skife.jdbi.v2.TransactionStatus; -import org.skife.jdbi.v2.tweak.HandleCallback; import org.skife.jdbi.v2.tweak.ResultSetMapper; import javax.annotation.Nullable; @@ -60,19 +64,19 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; +import java.util.stream.StreamSupport; /** * @@ -83,11 +87,48 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager private static final EmittingLogger log = new EmittingLogger(SQLMetadataSegmentManager.class); /** - * Use to synchronize {@link #start()}, {@link #stop()}, {@link #poll()}, and {@link #isStarted()}. These methods - * should be synchronized to prevent from being called at the same time if two different threads are calling them. - * This might be possible if a druid coordinator gets and drops leadership repeatedly in quick succession. + * Marker interface for objects stored in {@link #latestDatabasePoll}. See the comment for that field for details. */ - private final ReentrantReadWriteLock startStopLock = new ReentrantReadWriteLock(); + private interface DatabasePoll + {} + + /** Represents periodic {@link #poll}s happening from {@link #exec}. */ + private static class PeriodicDatabasePoll implements DatabasePoll + { + /** + * This future allows to wait until {@link #dataSourcesSnapshot} is initialized in the first {@link #poll()} + * happening since {@link #startPollingDatabasePeriodically()} is called for the first time, or since the last + * visible (in happens-before terms) call to {@link #startPollingDatabasePeriodically()} in case of Coordinator's + * leadership changes. + */ + final CompletableFuture firstPollCompletionFuture = new CompletableFuture<>(); + } + + /** + * Represents on-demand {@link #poll} initiated at periods of time when SqlSegmentsMetadata doesn't poll the database + * periodically. + */ + private static class OnDemandDatabasePoll implements DatabasePoll + { + final long initiationTimeNanos = System.nanoTime(); + final CompletableFuture pollCompletionFuture = new CompletableFuture<>(); + + long nanosElapsedFromInitiation() + { + return System.nanoTime() - initiationTimeNanos; + } + } + + /** + * Use to synchronize {@link #startPollingDatabasePeriodically}, {@link #stopPollingDatabasePeriodically}, {@link + * #poll}, and {@link #isPollingDatabasePeriodically}. These methods should be synchronized to prevent from being + * called at the same time if two different threads are calling them. This might be possible if Coordinator gets and + * drops leadership repeatedly in quick succession. + * + * This lock is also used to synchronize {@link #awaitOrPerformDatabasePoll} for times when SqlSegmentsMetadata + * is not polling the database periodically (in other words, when the Coordinator is not the leader). + */ + private final ReentrantReadWriteLock startStopPollLock = new ReentrantReadWriteLock(); /** * Used to ensure that {@link #poll()} is never run concurrently. It should already be so (at least in production @@ -95,41 +136,91 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager * scheduled in a single-threaded {@link #exec}, so this lock is an additional safety net in case there are bugs in * the code, and for tests, where {@link #poll()} is called from the outside code. * - * Not using {@link #startStopLock}.writeLock() in order to still be able to run {@link #poll()} concurrently with - * {@link #isStarted()}. + * Not using {@link #startStopPollLock}.writeLock() in order to still be able to run {@link #poll()} concurrently + * with {@link #isPollingDatabasePeriodically()}. */ private final Object pollLock = new Object(); private final ObjectMapper jsonMapper; - private final Supplier config; + private final Duration periodicPollDelay; private final Supplier dbTables; private final SQLMetadataConnector connector; - // Volatile since this reference is reassigned in "poll" and then read from in other threads. - // Starts null so we can differentiate "never polled" (null) from "polled, but empty" (empty dataSources map and - // empty overshadowedSegments set). - // Note that this is not simply a lazy-initialized variable: it starts off as null, and may transition between - // null and nonnull multiple times as stop() and start() are called. - @Nullable - private volatile DataSourcesSnapshot dataSourcesSnapshot = null; + /** + * This field is made volatile to avoid "ghost secondary reads" that may result in NPE, see + * https://github.com/code-review-checklists/java-concurrency#safe-local-dcl (note that dataSourcesSnapshot resembles + * a lazily initialized field). Alternative is to always read the field in a snapshot local variable, but it's too + * easy to forget to do. + * + * This field may be updated from {@link #exec}, or from whatever thread calling {@link #doOnDemandPoll} via {@link + * #awaitOrPerformDatabasePoll()} via one of the public methods of SqlSegmentsMetadata. + */ + private volatile @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = null; /** - * The number of times this SQLMetadataSegmentManager was started. + * The latest {@link DatabasePoll} represent {@link #poll()} calls which update {@link #dataSourcesSnapshot}, either + * periodically (see {@link PeriodicDatabasePoll}, {@link #startPollingDatabasePeriodically}, {@link + * #stopPollingDatabasePeriodically}) or "on demand" (see {@link OnDemandDatabasePoll}), when one of the methods that + * accesses {@link #dataSourcesSnapshot}'s state (such as {@link #getImmutableDataSourceWithUsedSegments}) is + * called when the Coordinator is not the leader and therefore SqlSegmentsMetadata isn't polling the database + * periodically. + * + * Note that if there is a happens-before relationship between a call to {@link #startPollingDatabasePeriodically()} + * (on Coordinators' leadership change) and one of the methods accessing the {@link #dataSourcesSnapshot}'s state in + * this class the latter is guaranteed to await for the initiated periodic poll. This is because when the latter + * method calls to {@link #awaitLatestDatabasePoll()} via {@link #awaitOrPerformDatabasePoll}, they will + * see the latest {@link PeriodicDatabasePoll} value (stored in this field, latestDatabasePoll, in {@link + * #startPollingDatabasePeriodically()}) and to await on its {@link PeriodicDatabasePoll#firstPollCompletionFuture}. + * + * However, the guarantee explained above doesn't make any actual semantic difference, because on both periodic and + * on-demand database polls the same invariant is maintained that the results not older than {@link + * #periodicPollDelay} are used. The main difference is in performance: since on-demand polls are irregular and happen + * in the context of the thread wanting to access the {@link #dataSourcesSnapshot}, that may cause delays in the + * logic. On the other hand, periodic polls are decoupled into {@link #exec} and {@link + * #dataSourcesSnapshot}-accessing methods should be generally "wait free" for database polls. + * + * The notion and the complexity of "on demand" database polls was introduced to simplify the interface of {@link + * MetadataSegmentManager} and guarantee that it always returns consistent and relatively up-to-date data from methods + * like {@link #getImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part + * is achieved via "hooking on" other polls by awaiting on {@link PeriodicDatabasePoll#firstPollCompletionFuture} or + * {@link OnDemandDatabasePoll#pollCompletionFuture}, see {@link #awaitOrPerformDatabasePoll} method + * implementation for details. + * + * Note: the overall implementation of periodic/on-demand polls is not completely optimal: for example, when the + * Coordinator just stopped leading, the latest periodic {@link #poll} (which is still "fresh") is not considered + * and a new on-demand poll is always initiated. This is done to simplify the implementation, while the efficiency + * during Coordinator leadership switches is not a priority. + * + * This field is {@code volatile} because it's checked and updated in a double-checked locking manner in {@link + * #awaitOrPerformDatabasePoll()}. */ - private long startCount = 0; + private volatile @Nullable DatabasePoll latestDatabasePoll = null; + + /** Used to cancel periodic poll task in {@link #stopPollingDatabasePeriodically}. */ + @GuardedBy("startStopPollLock") + private @Nullable Future periodicPollTaskFuture = null; + + /** The number of times {@link #startPollingDatabasePeriodically} was called. */ + @GuardedBy("startStopPollLock") + private long startPollingCount = 0; + /** - * Equal to the current {@link #startCount} value, if the SQLMetadataSegmentManager is currently started; -1 if + * Equal to the current {@link #startPollingCount} 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 - * the theoretical situation of two or more tasks scheduled in {@link #start()} calling {@link #isStarted()} and - * {@link #poll()} concurrently, if the sequence of {@link #start()} - {@link #stop()} - {@link #start()} actions - * occurs quickly. + * the theoretical situation of two or more tasks scheduled in {@link #startPollingDatabasePeriodically()} calling + * {@link #isPollingDatabasePeriodically()} and {@link #poll()} concurrently, if the sequence of {@link + * #startPollingDatabasePeriodically()} - {@link #stopPollingDatabasePeriodically()} - {@link + * #startPollingDatabasePeriodically()} actions occurs quickly. * - * {@link SQLMetadataRuleManager} also have a similar issue. + * {@link SQLMetadataRuleManager} also has a similar issue. */ - private long currentStartOrder = -1; - private ScheduledExecutorService exec = null; + @GuardedBy("startStopPollLock") + private long currentStartPollingOrder = -1; + + @GuardedBy("startStopPollLock") + private @Nullable ScheduledExecutorService exec = null; @Inject public SQLMetadataSegmentManager( @@ -140,33 +231,73 @@ public SQLMetadataSegmentManager( ) { this.jsonMapper = jsonMapper; - this.config = config; + this.periodicPollDelay = config.get().getPollDuration().toStandardDuration(); this.dbTables = dbTables; this.connector = connector; } - @Override + /** + * Don't confuse this method with {@link #startPollingDatabasePeriodically}. This is a lifecycle starting method to + * be executed just once for an instance of SqlSegmentsMetadata. + */ @LifecycleStart public void start() { - ReentrantReadWriteLock.WriteLock lock = startStopLock.writeLock(); + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); + try { + if (exec != null) { + return; // Already started + } + exec = Execs.scheduledSingleThreaded(getClass().getName() + "-Exec--%d"); + } + finally { + lock.unlock(); + } + } + + /** + * Don't confuse this method with {@link #stopPollingDatabasePeriodically}. This is a lifecycle stopping method to + * be executed just once for an instance of SqlSegmentsMetadata. + */ + @LifecycleStop + public void stop() + { + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); + try { + exec.shutdownNow(); + exec = null; + } + finally { + lock.unlock(); + } + } + + @Override + public void startPollingDatabasePeriodically() + { + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { - if (isStarted()) { + if (exec == null) { + throw new IllegalStateException(getClass().getName() + " is not started"); + } + if (isPollingDatabasePeriodically()) { return; } - startCount++; - currentStartOrder = startCount; - final long localStartOrder = currentStartOrder; + PeriodicDatabasePoll periodicPollUpdate = new PeriodicDatabasePoll(); + latestDatabasePoll = periodicPollUpdate; - exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d"); + startPollingCount++; + currentStartPollingOrder = startPollingCount; + final long localStartOrder = currentStartPollingOrder; - final Duration delay = config.get().getPollDuration().toStandardDuration(); - exec.scheduleWithFixedDelay( - createPollTaskForStartOrder(localStartOrder), + periodicPollTaskFuture = exec.scheduleWithFixedDelay( + createPollTaskForStartOrder(localStartOrder, periodicPollUpdate), 0, - delay.getMillis(), + periodicPollDelay.getMillis(), TimeUnit.MILLISECONDS ); } @@ -175,24 +306,33 @@ public void start() } } - private Runnable createPollTaskForStartOrder(long startOrder) + private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePoll periodicPollUpdate) { return () -> { - // 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 - ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock(); + // poll() is synchronized together with startPollingDatabasePeriodically(), stopPollingDatabasePeriodically() and + // isPollingDatabasePeriodically() to ensure that when stopPollingDatabasePeriodically() exits, poll() won't + // actually run anymore after that (it could only enter the synchronized section and exit immediately because the + // localStartedOrder doesn't match the new currentStartPollingOrder). It's needed to avoid flakiness in + // SqlSegmentsMetadataTest. See https://github.com/apache/incubator-druid/issues/6028 + ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock(); lock.lock(); try { - if (startOrder == currentStartOrder) { + if (startOrder == currentStartPollingOrder) { poll(); + periodicPollUpdate.firstPollCompletionFuture.complete(null); } else { - log.debug("startOrder = currentStartOrder = %d, skipping poll()", startOrder); + log.debug("startOrder = currentStartPollingOrder = %d, skipping poll()", startOrder); } } - catch (Exception e) { - log.makeAlert(e, "uncaught exception in segment manager polling thread").emit(); + catch (Throwable t) { + log.makeAlert(t, "Uncaught exception in %s's polling thread", SQLMetadataSegmentManager.class).emit(); + // Swallow the exception, so that scheduled polling goes on. Leave firstPollFutureSinceLastStart uncompleted + // for now, so that it may be completed during the next poll. + if (!(t instanceof Exception)) { + // Don't try to swallow a Throwable which is not an Exception (that is, a Error). + periodicPollUpdate.firstPollCompletionFuture.completeExceptionally(t); + throw t; + } } finally { lock.unlock(); @@ -201,320 +341,435 @@ private Runnable createPollTaskForStartOrder(long startOrder) } @Override - @LifecycleStop - public void stop() + public boolean isPollingDatabasePeriodically() + { + // isPollingDatabasePeriodically() is synchronized together with startPollingDatabasePeriodically(), + // stopPollingDatabasePeriodically() and poll() to ensure that the latest currentStartPollingOrder is always + // visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator. + ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock(); + lock.lock(); + try { + return currentStartPollingOrder >= 0; + } + finally { + lock.unlock(); + } + } + + @Override + public void stopPollingDatabasePeriodically() { - ReentrantReadWriteLock.WriteLock lock = startStopLock.writeLock(); + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); lock.lock(); try { - if (!isStarted()) { + if (!isPollingDatabasePeriodically()) { return; } - dataSourcesSnapshot = null; - currentStartOrder = -1; - exec.shutdownNow(); - exec = null; + + periodicPollTaskFuture.cancel(false); + latestDatabasePoll = null; + + // NOT nulling dataSourcesSnapshot, allowing to query the latest polled data even when this SegmentsMetadata + // object is stopped. + + currentStartPollingOrder = -1; } finally { lock.unlock(); } } - private Pair usedPayloadMapper( - final int index, - final ResultSet resultSet, - final StatementContext context - ) throws SQLException + private void awaitOrPerformDatabasePoll() { + // Double-checked locking with awaitLatestDatabasePoll() call playing the role of the "check". + if (awaitLatestDatabasePoll()) { + return; + } + ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock(); + lock.lock(); try { - return new Pair<>( - jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class), - resultSet.getBoolean("used") - ); + if (awaitLatestDatabasePoll()) { + return; + } + OnDemandDatabasePoll newOnDemandUpdate = new OnDemandDatabasePoll(); + this.latestDatabasePoll = newOnDemandUpdate; + doOnDemandPoll(newOnDemandUpdate); } - catch (IOException e) { - throw new RuntimeException(e); + finally { + lock.unlock(); } } /** - * Gets a list of all datasegments that overlap the provided interval along with thier used status. + * If the latest {@link DatabasePoll} is a {@link PeriodicDatabasePoll}, or an {@link OnDemandDatabasePoll} that is + * made not longer than {@link #periodicPollDelay} from now, awaits for it and returns true; returns false otherwise, + * meaning that a new on-demand database poll should be initiated. */ - private List> getDataSegmentsOverlappingInterval( - final String dataSource, - final Interval interval - ) - { - return connector.inReadOnlyTransaction( - (handle, status) -> handle.createQuery( - StringUtils.format( - "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start", - getSegmentsTable(), - connector.getQuoteString() - ) - ) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .map(this::usedPayloadMapper) - .list() - ); - } - - private List> getDataSegments( - final String dataSource, - final Collection segmentIds, - final Handle handle - ) + private boolean awaitLatestDatabasePoll() { - return segmentIds.stream().map( - segmentId -> Optional.ofNullable( - handle.createQuery( - StringUtils.format( - "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND id = :id", - getSegmentsTable() - ) - ) - .bind("dataSource", dataSource) - .bind("id", segmentId) - .map(this::usedPayloadMapper) - .first() - ) - .orElseThrow(() -> new UnknownSegmentIdException(StringUtils.format("Cannot find segment id [%s]", segmentId))) - ) - .collect(Collectors.toList()); + DatabasePoll latestDatabasePoll = this.latestDatabasePoll; + if (latestDatabasePoll instanceof PeriodicDatabasePoll) { + Futures.getUnchecked(((PeriodicDatabasePoll) latestDatabasePoll).firstPollCompletionFuture); + return true; + } + if (latestDatabasePoll instanceof OnDemandDatabasePoll) { + long periodicPollDelayNanos = TimeUnit.MILLISECONDS.toNanos(periodicPollDelay.getMillis()); + OnDemandDatabasePoll latestOnDemandPoll = (OnDemandDatabasePoll) latestDatabasePoll; + boolean latestUpdateIsFresh = latestOnDemandPoll.nanosElapsedFromInitiation() < periodicPollDelayNanos; + if (latestUpdateIsFresh) { + Futures.getUnchecked(latestOnDemandPoll.pollCompletionFuture); + return true; + } + // Latest on-demand update is not fresh. Fall through to return false from this method. + } else { + assert latestDatabasePoll == null; + // No periodic updates and no on-demand database poll have been done yet, nothing to await for. + } + return false; } - /** - * Builds a VersionedIntervalTimeline containing used segments that overlap the intervals passed. - */ - private VersionedIntervalTimeline buildVersionedIntervalTimeline( - final String dataSource, - final Collection intervals, - final Handle handle - ) + private void doOnDemandPoll(OnDemandDatabasePoll onDemandPoll) { - return VersionedIntervalTimeline.forSegments(intervals - .stream() - .flatMap(interval -> handle.createQuery( - StringUtils.format( - "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start AND used = true", - getSegmentsTable(), - connector.getQuoteString() - ) - ) - .setFetchSize(connector.getStreamingFetchSize()) - .bind("dataSource", dataSource) - .bind("start", interval.getStart().toString()) - .bind("end", interval.getEnd().toString()) - .map((i, resultSet, context) -> { - try { - return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }) - .list() - .stream() - ) - .iterator() - ); + try { + poll(); + onDemandPoll.pollCompletionFuture.complete(null); + } + catch (Throwable t) { + onDemandPoll.pollCompletionFuture.completeExceptionally(t); + throw t; + } } @Override - public boolean enableDataSource(final String dataSource) + public boolean markSegmentAsUsed(final String segmentId) { try { - return enableSegments(dataSource, Intervals.ETERNITY) != 0; + int numUpdatedDatabaseEntries = connector.getDBI().withHandle( + (Handle handle) -> handle + .createStatement(StringUtils.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable())) + .bind("id", segmentId) + .execute() + ); + // Unlike bulk markAsUsed methods: markAsUsedAllNonOvershadowedSegmentsInDataSource(), + // markAsUsedNonOvershadowedSegmentsInInterval(), and markAsUsedNonOvershadowedSegments() we don't put the marked + // segment into the respective data source, because we don't have it fetched from the database. It's probably not + // worth complicating the implementation and making two database queries just to add the segment because it will + // be anyway fetched during the next poll(). Segment putting that is done in the bulk markAsUsed methods is a nice + // to have thing, but doesn't formally affects the external guarantees of SegmentsMetadata class. + return numUpdatedDatabaseEntries > 0; } - catch (Exception e) { - log.error(e, "Exception enabling datasource %s", dataSource); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking segment %s as used", segmentId); + throw e; } } @Override - public int enableSegments(final String dataSource, final Interval interval) + public int markAsUsedAllNonOvershadowedSegmentsInDataSource(final String dataSource) { - List> segments = getDataSegmentsOverlappingInterval(dataSource, interval); - List segmentsToEnable = segments.stream() - .filter(segment -> !segment.rhs && interval.contains(segment.lhs.getInterval())) - .map(segment -> segment.lhs) - .collect(Collectors.toList()); - - VersionedIntervalTimeline versionedIntervalTimeline = VersionedIntervalTimeline.forSegments( - segments.stream().filter(segment -> segment.rhs).map(segment -> segment.lhs).iterator() - ); - VersionedIntervalTimeline.addSegments(versionedIntervalTimeline, segmentsToEnable.iterator()); - - return enableSegments( - segmentsToEnable, - versionedIntervalTimeline - ); + return doMarkAsUsedNonOvershadowedSegments(dataSource, null); } @Override - public int enableSegments(final String dataSource, final Collection segmentIds) + public int markAsUsedNonOvershadowedSegmentsInInterval(final String dataSource, final Interval interval) { - Pair, VersionedIntervalTimeline> data = connector.inReadOnlyTransaction( - (handle, status) -> { - List segments = getDataSegments(dataSource, segmentIds, handle) - .stream() - .filter(pair -> !pair.rhs) - .map(pair -> pair.lhs) - .collect(Collectors.toList()); - - VersionedIntervalTimeline versionedIntervalTimeline = buildVersionedIntervalTimeline( - dataSource, - JodaUtils.condenseIntervals(segments.stream().map(segment -> segment.getInterval()).collect(Collectors.toList())), - handle - ); - VersionedIntervalTimeline.addSegments(versionedIntervalTimeline, segments.iterator()); + Preconditions.checkNotNull(interval); + return doMarkAsUsedNonOvershadowedSegments(dataSource, interval); + } - return new Pair<>( - segments, - versionedIntervalTimeline - ); + /** + * Implementation for both {@link #markAsUsedAllNonOvershadowedSegmentsInDataSource} (if the given interval is null) + * and {@link #markAsUsedNonOvershadowedSegmentsInInterval}. + */ + private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable Interval interval) + { + List usedSegmentsOverlappingInterval = new ArrayList<>(); + List unusedSegmentsInInterval = new ArrayList<>(); + connector.inReadOnlyTransaction( + (handle, status) -> { + String queryString = + StringUtils.format("SELECT used, payload FROM %1$s WHERE dataSource = :dataSource", getSegmentsTable()); + if (interval != null) { + queryString += StringUtils.format(" AND start < :end AND %1$send%1$s > :start", connector.getQuoteString()); + } + Query query = handle + .createQuery(queryString) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("dataSource", dataSourceName); + if (interval != null) { + query = query + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()); + } + query = query + .map((int index, ResultSet resultSet, StatementContext context) -> { + try { + DataSegment segment = jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); + if (resultSet.getBoolean("used")) { + usedSegmentsOverlappingInterval.add(segment); + } else { + if (interval == null || interval.contains(segment.getInterval())) { + unusedSegmentsInInterval.add(segment); + } + } + return null; + } + catch (IOException e) { + throw new RuntimeException(e); + } + }); + // Consume the query results to ensure usedSegmentsOverlappingInterval and unusedSegmentsInInterval are + // populated. + consume(query.iterator()); + return null; } ); - return enableSegments( - data.lhs, - data.rhs + VersionedIntervalTimeline versionedIntervalTimeline = VersionedIntervalTimeline.forSegments( + Iterators.concat(usedSegmentsOverlappingInterval.iterator(), unusedSegmentsInInterval.iterator()) ); + + return markNonOvershadowedSegmentsAsUsed(unusedSegmentsInInterval, versionedIntervalTimeline); } - private int enableSegments( - final Collection segments, - final VersionedIntervalTimeline versionedIntervalTimeline + private static void consume(Iterator iterator) + { + while (iterator.hasNext()) { + iterator.next(); + } + } + + private int markNonOvershadowedSegmentsAsUsed( + List unusedSegments, + VersionedIntervalTimeline timeline ) { - if (segments.isEmpty()) { - log.warn("No segments found to update!"); - return 0; + List segmentIdsToMarkAsUsed = new ArrayList<>(); + for (DataSegment segment : unusedSegments) { + if (timeline.isOvershadowed(segment.getInterval(), segment.getVersion())) { + continue; + } + segmentIdsToMarkAsUsed.add(segment.getId().toString()); } - return connector.getDBI().withHandle(handle -> { - Batch batch = handle.createBatch(); - segments - .stream() - .map(segment -> segment.getId()) - .filter(segmentId -> !versionedIntervalTimeline.isOvershadowed( - segmentId.getInterval(), - segmentId.getVersion() - )) - .forEach(segmentId -> batch.add( - StringUtils.format( - "UPDATE %s SET used=true WHERE id = '%s'", - getSegmentsTable(), - segmentId - ) - )); - return batch.execute().length; - }); + return markSegmentsAsUsed(segmentIdsToMarkAsUsed); } @Override - public boolean enableSegment(final String segmentId) + public int markAsUsedNonOvershadowedSegments(final String dataSource, final Set segmentIds) + throws UnknownSegmentIdException { try { - connector.getDBI().withHandle( - new HandleCallback() - { - @Override - public Void withHandle(Handle handle) - { - handle.createStatement(StringUtils.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable())) - .bind("id", segmentId) - .execute(); - return null; - } - } - ); + Pair, VersionedIntervalTimeline> unusedSegmentsAndTimeline = connector + .inReadOnlyTransaction( + (handle, status) -> { + List unusedSegments = retrieveUnusedSegments(dataSource, segmentIds, handle); + List unusedSegmentsIntervals = JodaUtils.condenseIntervals( + unusedSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList()) + ); + Iterator usedSegmentsOverlappingUnusedSegmentsIntervals = + retrieveUsedSegmentsOverlappingIntervals(dataSource, unusedSegmentsIntervals, handle); + VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments( + Iterators.concat(usedSegmentsOverlappingUnusedSegmentsIntervals, unusedSegments.iterator()) + ); + return new Pair<>(unusedSegments, timeline); + } + ); + + List unusedSegments = unusedSegmentsAndTimeline.lhs; + VersionedIntervalTimeline timeline = unusedSegmentsAndTimeline.rhs; + return markNonOvershadowedSegmentsAsUsed(unusedSegments, timeline); } catch (Exception e) { - log.error(e, "Exception enabling segment %s", segmentId); - return false; + Throwable rootCause = Throwables.getRootCause(e); + if (rootCause instanceof UnknownSegmentIdException) { + throw (UnknownSegmentIdException) rootCause; + } else { + throw e; + } + } + } + + private List retrieveUnusedSegments( + final String dataSource, + final Set segmentIds, + final Handle handle + ) throws UnknownSegmentIdException + { + List unknownSegmentIds = new ArrayList<>(); + List segments = segmentIds + .stream() + .map( + segmentId -> { + Iterator segmentResultIterator = handle + .createQuery( + StringUtils.format( + "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND id = :id", + getSegmentsTable() + ) + ) + .bind("dataSource", dataSource) + .bind("id", segmentId) + .map((int index, ResultSet resultSet, StatementContext context) -> { + try { + if (!resultSet.getBoolean("used")) { + return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); + } else { + // We emit nulls for used segments. They are filtered out below in this method. + return null; + } + } + catch (IOException e) { + throw new RuntimeException(e); + } + }) + .iterator(); + if (!segmentResultIterator.hasNext()) { + unknownSegmentIds.add(segmentId); + return null; + } else { + @Nullable DataSegment segment = segmentResultIterator.next(); + if (segmentResultIterator.hasNext()) { + log.error( + "There is more than one row corresponding to segment id [%s] in data source [%s] in the database", + segmentId, + dataSource + ); + } + return segment; + } + } + ) + .filter(Objects::nonNull) // Filter nulls corresponding to used segments. + .collect(Collectors.toList()); + if (!unknownSegmentIds.isEmpty()) { + throw new UnknownSegmentIdException(unknownSegmentIds); + } + return segments; + } + + private Iterator retrieveUsedSegmentsOverlappingIntervals( + final String dataSource, + final Collection intervals, + final Handle handle + ) + { + return intervals + .stream() + .flatMap(interval -> { + Iterable segmentResultIterable = () -> handle + .createQuery( + StringUtils.format( + "SELECT payload FROM %1$s " + + "WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start AND used = true", + getSegmentsTable(), + connector.getQuoteString() + ) + ) + .setFetchSize(connector.getStreamingFetchSize()) + .bind("dataSource", dataSource) + .bind("start", interval.getStart().toString()) + .bind("end", interval.getEnd().toString()) + .map((int index, ResultSet resultSet, StatementContext context) -> { + try { + return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } + }) + .iterator(); + return StreamSupport.stream(segmentResultIterable.spliterator(), false); + }) + .iterator(); + } + + private int markSegmentsAsUsed(final List segmentIds) + { + if (segmentIds.isEmpty()) { + log.info("No segments found to update!"); + return 0; } - return true; + return connector.getDBI().withHandle(handle -> { + Batch batch = handle.createBatch(); + segmentIds.forEach(segmentId -> batch.add( + StringUtils.format("UPDATE %s SET used=true WHERE id = '%s'", getSegmentsTable(), segmentId) + )); + int[] segmentChanges = batch.execute(); + return computeNumChangedSegments(segmentIds, segmentChanges); + }); } @Override - public boolean removeDataSource(final String dataSource) + public int markAsUnusedAllSegmentsInDataSource(final String dataSource) { try { - final int removed = connector.getDBI().withHandle( - handle -> handle.createStatement( - StringUtils.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable()) - ).bind("dataSource", dataSource).execute() + final int numUpdatedDatabaseEntries = connector.getDBI().withHandle( + (Handle handle) -> handle + .createStatement( + StringUtils.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable()) + ) + .bind("dataSource", dataSource) + .execute() ); - if (removed == 0) { - return false; - } + return numUpdatedDatabaseEntries; } - catch (Exception e) { - log.error(e, "Error removing datasource %s", dataSource); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking all segments as unused in data source [%s]", dataSource); + throw e; } - - return true; } /** - * This method does not update {@code dataSourcesSnapshot}, see the comments in {@code doPoll()} about - * snapshot update. The segment removal will be reflected after next poll cyccle runs. + * This method does not update {@link #dataSourcesSnapshot}, see the comments in {@link #doPoll()} about + * snapshot update. The update of the segment's state will be reflected after the next {@link DatabasePoll}. */ @Override - public boolean removeSegment(String segmentId) + public boolean markSegmentAsUnused(final String segmentId) { try { - return removeSegmentFromTable(segmentId); + return markSegmentAsUnusedInDatabase(segmentId); } - catch (Exception e) { - log.error(e, e.toString()); - return false; + catch (RuntimeException e) { + log.error(e, "Exception marking segment [%s] as unused", segmentId); + throw e; } } @Override - public long disableSegments(String dataSource, Collection segmentIds) + public int markSegmentsAsUnused(String dataSourceName, Set segmentIds) { if (segmentIds.isEmpty()) { return 0; } - final long[] result = new long[1]; + final List segmentIdList = new ArrayList<>(segmentIds); try { - connector.getDBI().withHandle(handle -> { + return connector.getDBI().withHandle(handle -> { Batch batch = handle.createBatch(); - segmentIds - .forEach(segmentId -> batch.add( - StringUtils.format( - "UPDATE %s SET used=false WHERE datasource = '%s' AND id = '%s' ", - getSegmentsTable(), - dataSource, - segmentId - ) - )); - final int[] resultArr = batch.execute(); - result[0] = Arrays.stream(resultArr).filter(x -> x > 0).count(); - return result[0]; + segmentIdList.forEach(segmentId -> batch.add( + StringUtils.format( + "UPDATE %s SET used=false WHERE datasource = '%s' AND id = '%s'", + getSegmentsTable(), + dataSourceName, + segmentId + ) + )); + final int[] segmentChanges = batch.execute(); + return computeNumChangedSegments(segmentIdList, segmentChanges); }); } catch (Exception e) { throw new RuntimeException(e); } - return result[0]; } @Override - public int disableSegments(String dataSource, Interval interval) + public int markAsUnusedSegmentsInInterval(String dataSourceName, Interval interval) { try { - return connector.getDBI().withHandle( + Integer numUpdatedDatabaseEntries = connector.getDBI().withHandle( handle -> handle .createStatement( StringUtils @@ -524,116 +779,121 @@ public int disableSegments(String dataSource, Interval interval) getSegmentsTable(), connector.getQuoteString() )) - .bind("datasource", dataSource) + .bind("datasource", dataSourceName) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) .execute() ); + return numUpdatedDatabaseEntries; } catch (Exception e) { throw new RuntimeException(e); } } - private boolean removeSegmentFromTable(String segmentId) + private boolean markSegmentAsUnusedInDatabase(String segmentId) { - final int removed = connector.getDBI().withHandle( + final int numUpdatedRows = connector.getDBI().withHandle( handle -> handle .createStatement(StringUtils.format("UPDATE %s SET used=false WHERE id = :segmentID", getSegmentsTable())) .bind("segmentID", segmentId) .execute() ); - return removed > 0; + if (numUpdatedRows < 0) { + log.assertionError( + "Negative number of rows updated for segment id [%s]: %d", + segmentId, + numUpdatedRows + ); + } else if (numUpdatedRows > 1) { + log.error( + "More than one row updated for segment id [%s]: %d, " + + "there may be more than one row for the segment id in the database", + segmentId, + numUpdatedRows + ); + } + return numUpdatedRows > 0; } - @Override - public boolean isStarted() + private static int computeNumChangedSegments(List segmentIds, int[] segmentChanges) { - // isStarted() is synchronized together with start(), stop() and poll() to ensure that the latest currentStartOrder - // is always visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator. - ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock(); - lock.lock(); - try { - return currentStartOrder >= 0; - } - finally { - lock.unlock(); + int numChangedSegments = 0; + for (int i = 0; i < segmentChanges.length; i++) { + int numUpdatedRows = segmentChanges[i]; + if (numUpdatedRows < 0) { + log.assertionError( + "Negative number of rows updated for segment id [%s]: %d", + segmentIds.get(i), + numUpdatedRows + ); + } else if (numUpdatedRows > 1) { + log.error( + "More than one row updated for segment id [%s]: %d, " + + "there may be more than one row for the segment id in the database", + segmentIds.get(i), + numUpdatedRows + ); + } + if (numUpdatedRows > 0) { + numChangedSegments += 1; + } } + return numChangedSegments; } @Override - @Nullable - public ImmutableDruidDataSource getDataSource(String dataSourceName) + public @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSourceName) { - final ImmutableDruidDataSource dataSource = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.getDataSourcesMap().get(dataSourceName)) - .orElse(null); - return dataSource == null ? null : dataSource; + return getSnapshotOfDataSourcesWithAllUsedSegments().getDataSource(dataSourceName); } @Override - @Nullable - public Collection getDataSources() + public Collection getImmutableDataSourcesWithAllUsedSegments() { - return Optional.ofNullable(dataSourcesSnapshot).map(m -> m.getDataSources()).orElse(null); + return getSnapshotOfDataSourcesWithAllUsedSegments().getDataSourcesWithAllUsedSegments(); } @Override - @Nullable - public Iterable iterateAllSegments() + public Set getOvershadowedSegments() { - final Collection dataSources = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.getDataSources()) - .orElse(null); - if (dataSources == null) { - return null; - } - - return () -> dataSources.stream() - .flatMap(dataSource -> dataSource.getSegments().stream()) - .iterator(); + return getSnapshotOfDataSourcesWithAllUsedSegments().getOvershadowedSegments(); } @Override - @Nullable - public Set getOvershadowedSegments() + public DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments() { - return Optional.ofNullable(dataSourcesSnapshot).map(m -> m.getOvershadowedSegments()).orElse(null); + awaitOrPerformDatabasePoll(); + return dataSourcesSnapshot; } - @Nullable @Override - public DataSourcesSnapshot getDataSourcesSnapshot() + public Iterable iterateAllUsedSegments() { - return dataSourcesSnapshot; + awaitOrPerformDatabasePoll(); + return () -> dataSourcesSnapshot + .getDataSourcesWithAllUsedSegments() + .stream() + .flatMap(dataSource -> dataSource.getSegments().stream()) + .iterator(); } @Override - public Collection getAllDataSourceNames() + public Collection retrieveAllDataSourceNames() { return connector.getDBI().withHandle( - handle -> handle.createQuery( - StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable()) - ) - .fold( - new ArrayList<>(), - new Folder3, Map>() - { - @Override - public List fold( - List druidDataSources, - Map stringObjectMap, - FoldController foldController, - StatementContext statementContext - ) - { - druidDataSources.add( - MapUtils.getString(stringObjectMap, "datasource") - ); - return druidDataSources; - } - } - ) + handle -> handle + .createQuery(StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable())) + .fold( + new ArrayList<>(), + (List druidDataSources, + Map stringObjectMap, + FoldController foldController, + StatementContext statementContext) -> { + druidDataSources.add(MapUtils.getString(stringObjectMap, "datasource")); + return druidDataSources; + } + ) ); } @@ -642,15 +902,12 @@ public void poll() { // See the comment to the pollLock field, explaining this synchronized block synchronized (pollLock) { - try { - doPoll(); - } - catch (Exception e) { - log.makeAlert(e, "Problem polling DB.").emit(); - } + doPoll(); } } + /** This method is extracted from {@link #poll()} solely to reduce code nesting. */ + @GuardedBy("pollLock") private void doPoll() { log.debug("Starting polling of segment table"); @@ -681,6 +938,8 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE } catch (IOException e) { log.makeAlert(e, "Failed to read segment from db.").emit(); + // If one entry in database is corrupted doPoll() should continue to work overall. See + // filter by `Objects::nonNull` below in this method. return null; } } @@ -692,54 +951,49 @@ public DataSegment map(int index, ResultSet r, StatementContext ctx) throws SQLE ); if (segments == null || segments.isEmpty()) { - log.warn("No segments found in the database!"); + log.info("No segments found in the database!"); return; } log.info("Polled and found %,d segments in the database", segments.size()); - ConcurrentHashMap newDataSources = new ConcurrentHashMap<>(); + ImmutableMap dataSourceProperties = createDefaultDataSourceProperties(); - ImmutableMap dataSourceProperties = ImmutableMap.of("created", DateTimes.nowUtc().toString()); - segments - .stream() - .filter(Objects::nonNull) - .forEach(segment -> { - newDataSources - .computeIfAbsent(segment.getDataSource(), dsName -> new DruidDataSource(dsName, dataSourceProperties)) - .addSegmentIfAbsent(segment); - }); - - // dataSourcesSnapshot is updated only here, please note that if datasources or segments are enabled or disabled - // outside of poll, the dataSourcesSnapshot can become invalid until the next poll cycle. + // dataSourcesSnapshot is updated only here and the DataSourcesSnapshot object is immutable. If data sources or + // segments are marked as used or unused directly (via markAs...() methods in MetadataSegmentManager), the + // dataSourcesSnapshot can become invalid until the next database poll. // DataSourcesSnapshot computes the overshadowed segments, which makes it an expensive operation if the - // snapshot is invalidated on each segment removal, especially if a user issues a lot of single segment remove - // calls in rapid succession. So the snapshot update is not done outside of poll at this time. - // Updates outside of poll(), were primarily for the user experience, so users would immediately see the effect of - // a segment remove call reflected in MetadataResource API calls. These updates outside of scheduled poll may be - // added back in removeDataSource and removeSegment methods after the on-demand polling changes from - // https://github.com/apache/incubator-druid/pull/7653 are in. - final Map updatedDataSources = CollectionUtils.mapValues( - newDataSources, - v -> v.toImmutableDruidDataSource() + // snapshot was invalidated on each segment mark as unused or used, especially if a user issues a lot of single + // segment mark calls in rapid succession. So the snapshot update is not done outside of database poll at this time. + // Updates outside of database polls were primarily for the user experience, so users would immediately see the + // effect of a segment mark call reflected in MetadataResource API calls. + dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments( + Iterables.filter(segments, Objects::nonNull), // Filter corrupted entries (see above in this method). + dataSourceProperties ); - dataSourcesSnapshot = new DataSourcesSnapshot(updatedDataSources); + } + + private static ImmutableMap createDefaultDataSourceProperties() + { + return ImmutableMap.of("created", DateTimes.nowUtc().toString()); } /** * For the garbage collector in Java, it's better to keep new objects short-living, but once they are old enough * (i. e. promoted to old generation), try to keep them alive. In {@link #poll()}, we fetch and deserialize all - * existing segments each time, and then replace them in {@link #dataSourcesSnapshot}. This method allows to use already - * existing (old) segments when possible, effectively interning them a-la {@link String#intern} or {@link + * existing segments each time, and then replace them in {@link #dataSourcesSnapshot}. This method allows to use + * already existing (old) segments when possible, effectively interning them a-la {@link String#intern} or {@link * com.google.common.collect.Interner}, aiming to make the majority of {@link DataSegment} objects garbage soon after * they are deserialized and to die in young generation. It allows to avoid fragmentation of the old generation and * full GCs. */ private DataSegment replaceWithExistingSegmentIfPresent(DataSegment segment) { - ImmutableDruidDataSource dataSource = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.getDataSourcesMap().get(segment.getDataSource())) - .orElse(null); + @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = this.dataSourcesSnapshot; + if (dataSourcesSnapshot == null) { + return segment; + } + @Nullable ImmutableDruidDataSource dataSource = dataSourcesSnapshot.getDataSource(segment.getDataSource()); if (dataSource == null) { return segment; } @@ -753,11 +1007,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>() @@ -768,7 +1018,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() ) @@ -776,8 +1027,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/metadata/UnknownSegmentIdException.java b/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java index cca37b9318fa..6362077a2010 100644 --- a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java +++ b/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java @@ -19,13 +19,23 @@ package org.apache.druid.metadata; +import java.util.Collection; + /** - * Exception thrown by MetadataSegmentManager when an segment id is unknown. + * Exception thrown by {@link MetadataSegmentManager} when a segment id is unknown. */ -public class UnknownSegmentIdException extends RuntimeException +public class UnknownSegmentIdException extends Exception { - public UnknownSegmentIdException(String message) + private final Collection unknownSegmentIds; + + UnknownSegmentIdException(Collection segmentIds) + { + super("Cannot find segment ids " + segmentIds); + this.unknownSegmentIds = segmentIds; + } + + public Collection getUnknownSegmentIds() { - super(message); + return unknownSegmentIds; } } 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..ada41d22bb88 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 @@ -48,7 +48,7 @@ public class CoordinatorDynamicConfig { public static final String CONFIG_KEY = "coordinator.config"; - private final long millisToWaitBeforeDeleting; + private final long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; private final long mergeBytesLimit; private final int mergeSegmentsLimit; private final int maxSegmentsToMove; @@ -56,13 +56,29 @@ public class CoordinatorDynamicConfig private final int replicationThrottleLimit; private final int balancerComputeThreads; private final boolean emitBalancingStats; - private final boolean killAllDataSources; - private final Set killableDataSources; + + /** + * If true, {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller} 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 + * org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. + */ + 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 by {@link + * DruidCoordinatorCleanupPendingSegments}. In other words, segments in these data sources are "protected". + * + * Pending segments are considered "stale" when their created_time is older than {@link + * DruidCoordinatorCleanupPendingSegments#KEEP_PENDING_SEGMENTS_OFFSET} from now. + */ + private final Set dataSourcesToNotKillStalePendingSegmentsIn; /** * The maximum number of segments that could be queued for loading to any given server. @@ -74,7 +90,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 leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, @JsonProperty("mergeBytesLimit") long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") int maxSegmentsToMove, @@ -82,19 +101,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 killableDataSources, - @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.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -102,9 +128,10 @@ public CoordinatorDynamicConfig( this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.emitBalancingStats = emitBalancingStats; - this.killAllDataSources = killAllDataSources; - this.killableDataSources = parseJsonStringOrArray(killableDataSources); - this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources); + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; + this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn); + this.dataSourcesToNotKillStalePendingSegmentsIn = + parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn); this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes); Preconditions.checkArgument( @@ -113,8 +140,10 @@ public CoordinatorDynamicConfig( ); this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; - if (this.killAllDataSources && !this.killableDataSources.isEmpty()) { - throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist"); + if (this.killUnusedSegmentsInAllDataSources && !this.specificDataSourcesToKillUnusedSegmentsIn.isEmpty()) { + throw new IAE( + "can't have killUnusedSegmentsInAllDataSources and non-empty specificDataSourcesToKillUnusedSegmentsIn" + ); } } @@ -152,10 +181,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 getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() { - return millisToWaitBeforeDeleting; + return leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; } @JsonProperty @@ -200,30 +229,22 @@ public int getBalancerComputeThreads() return balancerComputeThreads; } - /** - * List of dataSources for which kill tasks are sent in - * {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}. - */ @JsonProperty("killDataSourceWhitelist") - public Set getKillableDataSources() + public Set getSpecificDataSourcesToKillUnusedSegmentsIn() { - return killableDataSources; + return specificDataSourcesToKillUnusedSegmentsIn; } - @JsonProperty - public boolean isKillAllDataSources() + @JsonProperty("killAllDataSources") + public boolean isKillUnusedSegmentsInAllDataSources() { - return killAllDataSources; + return killUnusedSegmentsInAllDataSources; } - /** - * List of dataSources for which pendingSegments are NOT cleaned up - * in {@link DruidCoordinatorCleanupPendingSegments}. - */ - @JsonProperty - public Set getProtectedPendingSegmentDatasources() + @JsonProperty("killPendingSegmentsSkipList") + public Set getDataSourcesToNotKillStalePendingSegmentsIn() { - return protectedPendingSegmentDatasources; + return dataSourcesToNotKillStalePendingSegmentsIn; } @JsonProperty @@ -233,9 +254,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 */ @@ -270,7 +291,8 @@ public int getDecommissioningMaxPercentOfMaxSegmentsToMove() public String toString() { return "CoordinatorDynamicConfig{" + - "millisToWaitBeforeDeleting=" + millisToWaitBeforeDeleting + + "leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments=" + + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments + ", mergeBytesLimit=" + mergeBytesLimit + ", mergeSegmentsLimit=" + mergeSegmentsLimit + ", maxSegmentsToMove=" + maxSegmentsToMove + @@ -278,9 +300,9 @@ public String toString() ", replicationThrottleLimit=" + replicationThrottleLimit + ", balancerComputeThreads=" + balancerComputeThreads + ", emitBalancingStats=" + emitBalancingStats + - ", killAllDataSources=" + killAllDataSources + - ", killDataSourceWhitelist=" + killableDataSources + - ", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + + ", killUnusedSegmentsInAllDataSources=" + killUnusedSegmentsInAllDataSources + + ", specificDataSourcesToKillUnusedSegmentsIn=" + specificDataSourcesToKillUnusedSegmentsIn + + ", dataSourcesToNotKillStalePendingSegmentsIn=" + dataSourcesToNotKillStalePendingSegmentsIn + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + ", decommissioningNodes=" + decommissioningNodes + ", decommissioningMaxPercentOfMaxSegmentsToMove=" + decommissioningMaxPercentOfMaxSegmentsToMove + @@ -299,7 +321,8 @@ public boolean equals(Object o) CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o; - if (millisToWaitBeforeDeleting != that.millisToWaitBeforeDeleting) { + if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments != + that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) { return false; } if (mergeBytesLimit != that.mergeBytesLimit) { @@ -323,16 +346,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(killableDataSources, that.killableDataSources)) { + 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)) { @@ -345,7 +368,7 @@ public boolean equals(Object o) public int hashCode() { return Objects.hash( - millisToWaitBeforeDeleting, + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit, mergeSegmentsLimit, maxSegmentsToMove, @@ -353,10 +376,10 @@ public int hashCode() replicationThrottleLimit, balancerComputeThreads, emitBalancingStats, - killAllDataSources, + killUnusedSegmentsInAllDataSources, maxSegmentsInNodeLoadingQueue, - killableDataSources, - protectedPendingSegmentDatasources, + specificDataSourcesToKillUnusedSegmentsIn, + dataSourcesToNotKillStalePendingSegmentsIn, decommissioningNodes, decommissioningMaxPercentOfMaxSegmentsToMove ); @@ -369,19 +392,20 @@ 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_LEADING_TIME_MILLIS_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 leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; private Long mergeBytesLimit; private Integer mergeSegmentsLimit; private Integer maxSegmentsToMove; @@ -389,9 +413,9 @@ public static class Builder private Integer replicationThrottleLimit; private Boolean emitBalancingStats; private Integer balancerComputeThreads; - private Object killableDataSources; - private Boolean killAllDataSources; - private Object killPendingSegmentsSkipList; + private Object specificDataSourcesToKillUnusedSegmentsIn; + private Boolean killUnusedSegmentsInAllDataSources; + private Object dataSourcesToNotKillStalePendingSegmentsIn; private Integer maxSegmentsInNodeLoadingQueue; private Object decommissioningNodes; private Integer decommissioningMaxPercentOfMaxSegmentsToMove; @@ -402,7 +426,8 @@ public Builder() @JsonCreator public Builder( - @JsonProperty("millisToWaitBeforeDeleting") @Nullable Long millisToWaitBeforeDeleting, + @JsonProperty("millisToWaitBeforeDeleting") + @Nullable Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, @JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit, @JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit, @JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove, @@ -410,15 +435,17 @@ public Builder( @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, - @JsonProperty("killDataSourceWhitelist") @Nullable Object killableDataSources, - @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.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments; this.mergeBytesLimit = mergeBytesLimit; this.mergeSegmentsLimit = mergeSegmentsLimit; this.maxSegmentsToMove = maxSegmentsToMove; @@ -426,17 +453,17 @@ public Builder( this.replicationThrottleLimit = replicationThrottleLimit; this.balancerComputeThreads = balancerComputeThreads; this.emitBalancingStats = emitBalancingStats; - this.killAllDataSources = killAllDataSources; - this.killableDataSources = killableDataSources; - 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 withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(long leadingTimeMillis) { - this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; + this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = leadingTimeMillis; return this; } @@ -482,15 +509,15 @@ public Builder withEmitBalancingStats(boolean emitBalancingStats) return this; } - public Builder withKillDataSourceWhitelist(Set killDataSourceWhitelist) + public Builder withSpecificDataSourcesToKillUnusedSegmentsIn(Set dataSources) { - this.killableDataSources = killDataSourceWhitelist; + this.specificDataSourcesToKillUnusedSegmentsIn = dataSources; return this; } - public Builder withKillAllDataSources(boolean killAllDataSources) + public Builder withKillUnusedSegmentsInAllDataSources(boolean killUnusedSegmentsInAllDataSources) { - this.killAllDataSources = killAllDataSources; + this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources; return this; } @@ -515,7 +542,9 @@ public Builder withDecommissioningMaxPercentOfMaxSegmentsToMove(Integer percent) public CoordinatorDynamicConfig build() { return new CoordinatorDynamicConfig( - millisToWaitBeforeDeleting == null ? DEFAULT_MILLIS_TO_WAIT_BEFORE_DELETING : millisToWaitBeforeDeleting, + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null + ? DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS + : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit, mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit, maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove, @@ -523,9 +552,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, - killableDataSources, - 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, @@ -539,7 +570,9 @@ public CoordinatorDynamicConfig build() public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) { return new CoordinatorDynamicConfig( - millisToWaitBeforeDeleting == null ? defaults.getMillisToWaitBeforeDeleting() : millisToWaitBeforeDeleting, + leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null + ? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + : leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit, mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit, maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove, @@ -547,11 +580,15 @@ 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, - 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/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/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java index 586f92b2dd79..2001b3fa4e7f 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java @@ -22,11 +22,13 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.utils.CollectionUtils; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -35,7 +37,6 @@ import java.util.Set; import java.util.TreeSet; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; /** * Contains a representation of the current state of the cluster by tier. @@ -43,6 +44,16 @@ */ public class DruidCluster { + /** This static factory method must be called only from inside DruidClusterBuilder in tests. */ + @VisibleForTesting + static DruidCluster createDruidClusterFromBuilderInTest( + @Nullable Set realtimes, + Map> historicals + ) + { + return new DruidCluster(realtimes, historicals); + } + private final Set realtimes; private final Map> historicals; @@ -52,8 +63,7 @@ public DruidCluster() this.historicals = new HashMap<>(); } - @VisibleForTesting - public DruidCluster( + private DruidCluster( @Nullable Set realtimes, Map> historicals ) @@ -62,12 +72,13 @@ public DruidCluster( this.historicals = historicals .entrySet() .stream() - .collect(Collectors.toMap( - Map.Entry::getKey, - e -> StreamSupport - .stream(e.getValue().spliterator(), false) - .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - )); + .collect( + Collectors.toMap( + Map.Entry::getKey, + (Map.Entry> e) -> + CollectionUtils.newTreeSet(Comparator.reverseOrder(), e.getValue()) + ) + ); } public void add(ServerHolder serverHolder) 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 ba4296892a0b..431cb81ca5a7 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 @@ -19,13 +19,15 @@ package org.apache.druid.server.coordinator; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Ordering; import com.google.common.collect.Sets; 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; @@ -76,7 +78,6 @@ import org.joda.time.DateTime; import org.joda.time.Duration; -import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -84,7 +85,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.Optional; import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.ConcurrentHashMap; @@ -104,7 +104,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 @@ -128,7 +128,7 @@ public class DruidCoordinator private final DruidCoordinatorConfig config; private final ZkPathsConfig zkPaths; private final JacksonConfigManager configManager; - private final MetadataSegmentManager metadataSegmentManager; + private final MetadataSegmentManager segmentsMetadata; private final ServerInventoryView serverInventoryView; private final MetadataRuleManager metadataRuleManager; private final CuratorFramework curator; @@ -148,17 +148,13 @@ public class DruidCoordinator private volatile boolean started = false; private volatile SegmentReplicantLookup segmentReplicantLookup = null; - /** - * set in {@link CoordinatorRunnable#run()} at start of every coordinator run - */ - private volatile DataSourcesSnapshot dataSourcesSnapshot = null; @Inject public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegmentManager metadataSegmentManager, + MetadataSegmentManager segmentsMetadata, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -178,7 +174,7 @@ public DruidCoordinator( config, zkPaths, configManager, - metadataSegmentManager, + segmentsMetadata, serverInventoryView, metadataRuleManager, curator, @@ -200,7 +196,7 @@ public DruidCoordinator( DruidCoordinatorConfig config, ZkPathsConfig zkPaths, JacksonConfigManager configManager, - MetadataSegmentManager metadataSegmentManager, + MetadataSegmentManager segmentsMetadata, ServerInventoryView serverInventoryView, MetadataRuleManager metadataRuleManager, CuratorFramework curator, @@ -221,7 +217,7 @@ public DruidCoordinator( this.zkPaths = zkPaths; this.configManager = configManager; - this.metadataSegmentManager = metadataSegmentManager; + this.segmentsMetadata = segmentsMetadata; this.serverInventoryView = serverInventoryView; this.metadataRuleManager = metadataRuleManager; this.curator = curator; @@ -263,11 +259,7 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - final Iterable dataSegments = iterateAvailableDataSegments(); - - if (dataSegments == null) { - return underReplicationCountsPerDataSourcePerTier; - } + final Iterable dataSegments = segmentsMetadata.iterateAllUsedSegments(); final DateTime now = DateTimes.nowUtc(); @@ -295,45 +287,36 @@ public Map> computeUnderReplicationCountsPerDataS return underReplicationCountsPerDataSourcePerTier; } - public Object2LongMap getSegmentAvailability() + public Object2IntMap computeNumsUnavailableUsedSegmentsPerDataSource() { - final Object2LongOpenHashMap retVal = new Object2LongOpenHashMap<>(); - if (segmentReplicantLookup == null) { - return retVal; + return Object2IntMaps.emptyMap(); } - final Iterable dataSegments = iterateAvailableDataSegments(); + final Object2IntOpenHashMap numsUnavailableUsedSegmentsPerDataSource = new Object2IntOpenHashMap<>(); - if (dataSegments == null) { - return retVal; - } + final Iterable dataSegments = segmentsMetadata.iterateAllUsedSegments(); for (DataSegment segment : dataSegments) { 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() { final Map loadStatus = new HashMap<>(); - final Collection dataSources = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.getDataSources()) - .orElse(null); - - if (dataSources == null) { - return loadStatus; - } + final Collection dataSources = + segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments(); for (ImmutableDruidDataSource dataSource : dataSources) { 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()) { @@ -346,10 +329,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) ); } @@ -371,10 +354,10 @@ public CoordinatorCompactionConfig getCompactionConfig() return CoordinatorCompactionConfig.current(configManager); } - public void removeSegment(DataSegment segment) + public void markSegmentAsUnused(DataSegment segment) { - log.info("Removing Segment[%s]", segment.getId()); - metadataSegmentManager.removeSegment(segment.getId().toString()); + log.info("Marking segment[%s] as unused", segment.getId()); + segmentsMetadata.markSegmentAsUnused(segment.getId().toString()); } public String getCurrentLeader() @@ -382,13 +365,8 @@ public String getCurrentLeader() return coordLeaderSelector.getCurrentLeader(); } - @VisibleForTesting - void setDataSourcesSnapshotForTest(DataSourcesSnapshot snapshot) - { - dataSourcesSnapshot = snapshot; - } - public void moveSegment( + DruidCoordinatorRuntimeParams params, ImmutableDruidServer fromServer, ImmutableDruidServer toServer, DataSegment segment, @@ -408,9 +386,7 @@ public void moveSegment( throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); } - ImmutableDruidDataSource dataSource = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.getDataSource(segment.getDataSource())) - .orElse(null); + ImmutableDruidDataSource dataSource = params.getDataSourcesSnapshot().getDataSource(segment.getDataSource()); if (dataSource == null) { throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); } @@ -488,24 +464,6 @@ 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. - * - * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has - * not yet been polled.) - */ - @Nullable - public Iterable iterateAvailableDataSegments() - { - final Iterable dataSources = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.iterateAllSegmentsInSnapshot()) - .orElse(null); - return dataSources == null ? null : dataSources; - } - @LifecycleStart public void start() { @@ -560,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(); + segmentsMetadata.startPollingDatabasePeriodically(); metadataRuleManager.start(); lookupCoordinatorManager.start(); serviceAnnouncer.announce(self); @@ -628,7 +586,7 @@ private void stopBeingLeader() serviceAnnouncer.unannounce(self); lookupCoordinatorManager.stop(); metadataRuleManager.stop(); - metadataSegmentManager.stop(); + segmentsMetadata.stopPollingDatabasePeriodically(); } } @@ -648,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; @@ -672,7 +630,7 @@ public void run() } List allStarted = Arrays.asList( - metadataSegmentManager.isStarted(), + segmentsMetadata.isPollingDatabasePeriodically(), serverInventoryView.isStarted() ); for (Boolean aBoolean : allStarted) { @@ -690,26 +648,19 @@ public void run() BalancerStrategy balancerStrategy = factory.createBalancerStrategy(balancerExec); // Do coordinator stuff. - dataSourcesSnapshot = metadataSegmentManager.getDataSourcesSnapshot(); - final Collection dataSources = Optional.ofNullable(dataSourcesSnapshot) - .map(m -> m.getDataSources()) - .orElse(null); - - if (dataSources == null) { - log.info("Metadata store not polled yet, skipping this run."); - return; - } + DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments(); DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withStartTime(startTime) - .withDataSources(dataSources) - .withDynamicConfigs(getDynamicConfigs()) - .withCompactionConfig(getCompactionConfig()) - .withEmitter(emitter) - .withBalancerStrategy(balancerStrategy) - .withDataSourcesSnapshot(dataSourcesSnapshot) - .build(); + DruidCoordinatorRuntimeParams + .newBuilder() + .withStartTimeNanos(startTimeNanos) + .withSnapshotOfDataSourcesWithAllUsedSegments(dataSourcesSnapshot) + .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()) { 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 bf4ffdee80e5..07cb037f6f0c 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 @@ -69,15 +69,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); - for (String dataSource : params.getDataSources().keySet()) { - if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) { + final DateTime stalePendingSegmentsCutoffCreationTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); + for (String dataSource : params.getUsedSegmentsTimelinesPerDataSource().keySet()) { + 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 de75bce9d9c2..d96a7a9ad974 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 @@ -21,8 +21,8 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DataSourcesSnapshot; -import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.metadata.MetadataRuleManager; @@ -36,6 +36,7 @@ import java.util.HashMap; import java.util.Map; import java.util.TreeSet; +import java.util.concurrent.TimeUnit; /** */ @@ -46,19 +47,19 @@ public class DruidCoordinatorRuntimeParams * 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) + private static TreeSet createUsedSegmentsSet(Iterable usedSegments) { TreeSet segmentsSet = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); - availableSegments.forEach(segmentsSet::add); + usedSegments.forEach(segmentsSet::add); return segmentsSet; } - private final long startTime; + private final long startTimeNanos; private final DruidCluster druidCluster; private final MetadataRuleManager databaseRuleManager; private final SegmentReplicantLookup segmentReplicantLookup; - private final Map> dataSources; - private final @Nullable TreeSet availableSegments; + private final @Nullable TreeSet usedSegments; + private final @Nullable DataSourcesSnapshot dataSourcesSnapshot; private final Map loadManagementPeons; private final ReplicationThrottler replicationManager; private final ServiceEmitter emitter; @@ -67,15 +68,14 @@ public static TreeSet createAvailableSegmentsSet(Iterable> dataSources, - @Nullable TreeSet availableSegments, + @Nullable TreeSet usedSegments, + @Nullable DataSourcesSnapshot dataSourcesSnapshot, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -83,16 +83,15 @@ private DruidCoordinatorRuntimeParams( CoordinatorCompactionConfig coordinatorCompactionConfig, CoordinatorStats stats, DateTime balancerReferenceTimestamp, - BalancerStrategy balancerStrategy, - DataSourcesSnapshot dataSourcesSnapshot + BalancerStrategy balancerStrategy ) { - this.startTime = startTime; + this.startTimeNanos = startTimeNanos; this.druidCluster = druidCluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSources = dataSources; - this.availableSegments = availableSegments; + this.usedSegments = usedSegments; + this.dataSourcesSnapshot = dataSourcesSnapshot; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -101,12 +100,11 @@ private DruidCoordinatorRuntimeParams( this.stats = stats; this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.balancerStrategy = balancerStrategy; - this.dataSourcesSnapshot = dataSourcesSnapshot; } - public long getStartTime() + public long getStartTimeNanos() { - return startTime; + return startTimeNanos; } public DruidCluster getDruidCluster() @@ -124,15 +122,20 @@ public SegmentReplicantLookup getSegmentReplicantLookup() return segmentReplicantLookup; } - public Map> getDataSources() + /** + * Creates and returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used" + * segments. + */ + public Map> getUsedSegmentsTimelinesPerDataSource() { - return dataSources; + Preconditions.checkState(dataSourcesSnapshot != null, "dataSourcesSnapshot or usedSegments must be set"); + return dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource(); } - public TreeSet getAvailableSegments() + public TreeSet getUsedSegments() { - Preconditions.checkState(availableSegments != null, "availableSegments must be set"); - return availableSegments; + Preconditions.checkState(usedSegments != null, "usedSegments or dataSourcesSnapshot must be set"); + return usedSegments; } public Map getLoadManagementPeons() @@ -175,14 +178,18 @@ public BalancerStrategy getBalancerStrategy() return balancerStrategy; } - public DataSourcesSnapshot getDataSourcesSnapshot() + public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements() { - return dataSourcesSnapshot; + long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos(); + long lagNanos = TimeUnit.MILLISECONDS.toNanos( + coordinatorDynamicConfig.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + ); + return nanosElapsedSinceCoordinatorStart > lagNanos; } - public boolean hasDeletionWaitTimeElapsed() + public DataSourcesSnapshot getDataSourcesSnapshot() { - return (System.currentTimeMillis() - getStartTime() > coordinatorDynamicConfig.getMillisToWaitBeforeDeleting()); + return dataSourcesSnapshot; } public static Builder newBuilder() @@ -193,12 +200,12 @@ public static Builder newBuilder() public Builder buildFromExisting() { return new Builder( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - availableSegments, + usedSegments, + dataSourcesSnapshot, loadManagementPeons, replicationManager, emitter, @@ -210,15 +217,15 @@ public Builder buildFromExisting() ); } - public Builder buildFromExistingWithoutAvailableSegments() + public Builder buildFromExistingWithoutSegmentsMetadata() { return new Builder( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - null, // availableSegments + null, // usedSegments + null, // dataSourcesSnapshot loadManagementPeons, replicationManager, emitter, @@ -232,12 +239,12 @@ public Builder buildFromExistingWithoutAvailableSegments() public static class Builder { - private long startTime; + private @Nullable Long startTimeNanos; private DruidCluster druidCluster; private MetadataRuleManager databaseRuleManager; private SegmentReplicantLookup segmentReplicantLookup; - private Map> dataSources; - private @Nullable TreeSet availableSegments; + private @Nullable TreeSet usedSegments; + private @Nullable DataSourcesSnapshot dataSourcesSnapshot; private final Map loadManagementPeons; private ReplicationThrottler replicationManager; private ServiceEmitter emitter; @@ -246,16 +253,15 @@ public static class Builder private CoordinatorStats stats; private DateTime balancerReferenceTimestamp; private BalancerStrategy balancerStrategy; - private DataSourcesSnapshot dataSourcesSnapshot; - Builder() + private Builder() { - this.startTime = 0; + this.startTimeNanos = null; this.druidCluster = null; this.databaseRuleManager = null; this.segmentReplicantLookup = null; - this.dataSources = new HashMap<>(); - this.availableSegments = null; + this.usedSegments = null; + this.dataSourcesSnapshot = null; this.loadManagementPeons = new HashMap<>(); this.replicationManager = null; this.emitter = null; @@ -263,16 +269,15 @@ public static class Builder this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); this.balancerReferenceTimestamp = DateTimes.nowUtc(); - this.dataSourcesSnapshot = null; } Builder( - long startTime, + long startTimeNanos, DruidCluster cluster, MetadataRuleManager databaseRuleManager, SegmentReplicantLookup segmentReplicantLookup, - Map> dataSources, - @Nullable TreeSet availableSegments, + @Nullable TreeSet usedSegments, + @Nullable DataSourcesSnapshot dataSourcesSnapshot, Map loadManagementPeons, ReplicationThrottler replicationManager, ServiceEmitter emitter, @@ -283,12 +288,12 @@ public static class Builder BalancerStrategy balancerStrategy ) { - this.startTime = startTime; + this.startTimeNanos = startTimeNanos; this.druidCluster = cluster; this.databaseRuleManager = databaseRuleManager; this.segmentReplicantLookup = segmentReplicantLookup; - this.dataSources = dataSources; - this.availableSegments = availableSegments; + this.usedSegments = usedSegments; + this.dataSourcesSnapshot = dataSourcesSnapshot; this.loadManagementPeons = loadManagementPeons; this.replicationManager = replicationManager; this.emitter = emitter; @@ -301,13 +306,14 @@ public static class Builder public DruidCoordinatorRuntimeParams build() { + Preconditions.checkNotNull(startTimeNanos, "startTime must be set"); return new DruidCoordinatorRuntimeParams( - startTime, + startTimeNanos, druidCluster, databaseRuleManager, segmentReplicantLookup, - dataSources, - availableSegments, + usedSegments, + dataSourcesSnapshot, loadManagementPeons, replicationManager, emitter, @@ -315,14 +321,13 @@ public DruidCoordinatorRuntimeParams build() coordinatorCompactionConfig, stats, balancerReferenceTimestamp, - balancerStrategy, - dataSourcesSnapshot + balancerStrategy ); } - public Builder withStartTime(long time) + public Builder withStartTimeNanos(long startTimeNanos) { - startTime = time; + this.startTimeNanos = startTimeNanos; return this; } @@ -344,58 +349,40 @@ public Builder withSegmentReplicantLookup(SegmentReplicantLookup lookup) return this; } - public Builder withDataSources(Map> dataSources) + public Builder withSnapshotOfDataSourcesWithAllUsedSegments(DataSourcesSnapshot snapshot) { - this.dataSources = dataSources; - return this; - } - - public Builder withDataSources(Collection dataSourcesCollection) - { - dataSourcesCollection.forEach( - dataSource -> { - VersionedIntervalTimeline timeline = dataSources.computeIfAbsent( - dataSource.getName(), - k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER) - ); - - dataSource.getSegments().forEach( - segment -> timeline.add( - segment.getInterval(), - segment.getVersion(), - segment.getShardSpec().createChunk(segment) - ) - ); - } - ); + this.usedSegments = createUsedSegmentsSet(snapshot.iterateAllUsedSegmentsInSnapshot()); + this.dataSourcesSnapshot = snapshot; return this; } /** 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)); + this.usedSegments = createUsedSegmentsSet(usedSegments); + this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments(usedSegments, ImmutableMap.of()); + return this; } - /** - * Note: unlike {@link #withAvailableSegmentsInTest(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) + /** This method must be used in test code only. */ + @VisibleForTesting + public Builder withUsedSegmentsTimelinesPerDataSourceInTest( + Map> usedSegmentsTimelinesPerDataSource + ) { - //noinspection ObjectEquality - if (availableSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) { - throw new IllegalArgumentException("Expected DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST"); - } - this.availableSegments = availableSegments; + this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegmentsTimelines( + usedSegmentsTimelinesPerDataSource, + ImmutableMap.of() + ); + usedSegments = createUsedSegmentsSet(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()); return this; } @@ -446,11 +433,5 @@ public Builder withBalancerStrategy(BalancerStrategy balancerStrategy) this.balancerStrategy = balancerStrategy; return this; } - - public Builder withDataSourcesSnapshot(DataSourcesSnapshot snapshot) - { - this.dataSourcesSnapshot = snapshot; - 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 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 + // 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(); @@ -269,6 +268,7 @@ protected boolean moveSegment( movingSegments.put(segmentId, segment); callback = () -> movingSegments.remove(segmentId); coordinator.moveSegment( + params, fromServer, toServer, segmentToMove, 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..bde7c15810d2 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 @@ -46,41 +46,40 @@ 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.coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()) { + 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 + .computeIfAbsent( + dataSource.getName(), + dsName -> new VersionedIntervalTimeline<>(Comparator.naturalOrder()) + ); + VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator()); } } + } - //Remove all segments in db that are overshadowed by served segments - for (DataSegment dataSegment : params.getAvailableSegments()) { - VersionedIntervalTimeline timeline = timelines.get(dataSegment.getDataSource()); - if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { - coordinator.removeSegment(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.markSegmentAsUnused(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/DruidCoordinatorCleanupUnneeded.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java index 2e77577b69b0..af7ccc1cd0a1 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/DruidCoordinatorCleanupUnneeded.java @@ -43,22 +43,19 @@ public class DruidCoordinatorCleanupUnneeded implements DruidCoordinatorHelper public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { CoordinatorStats stats = new CoordinatorStats(); - Set availableSegments = params.getAvailableSegments(); + Set usedSegments = params.getUsedSegments(); DruidCluster cluster = params.getDruidCluster(); - // Drop segments that no longer exist in the available segments configuration, *if* it has been populated. (It's - // also filled atomically, so if there are any segments at all, we should have all of them.) - // - // Note that if the metadata store has not been polled yet, "getAvailableSegments" would throw an error since - // "availableSegments" is null. But this won't happen, since the earlier helper "DruidCoordinatorSegmentInfoLoader" - // would have canceled the run. + // Unload segments that are no longer marked as used from historical servers, *if* the usedSegments collection has + // been populated. Used segments must be already populated because otherwise the earlier helper + // DruidCoordinatorUsedSegmentsLoader would have canceled the Coordinator's run. 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..69363639f221 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; @@ -214,14 +215,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 ) ); } @@ -247,7 +248,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) emitter.emit( new ServiceMetricEvent.Builder().build( "compact/task/count", - stats.getGlobalStat("compactTaskCount") + stats.getGlobalStat(DruidCoordinatorSegmentCompactor.COMPACT_TASK_COUNT) ) ); @@ -264,7 +265,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) // Emit segment metrics final Stream allSegments = params - .getDataSources() + .getUsedSegmentsTimelinesPerDataSource() .values() .stream() .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) 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 7570c81cbe13..c7b3b7a7bd9e 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 @@ -19,9 +19,7 @@ package org.apache.druid.server.coordinator.helper; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; -import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.metadata.MetadataRuleManager; @@ -36,7 +34,6 @@ import org.joda.time.DateTime; import java.util.List; -import java.util.Optional; import java.util.Set; /** @@ -44,7 +41,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; @@ -83,34 +80,28 @@ 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 - // If metadata store hasn't been polled yet, use empty overshadowed list - final DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot(); - Set overshadowed = ImmutableSet.of(); - if (dataSourcesSnapshot != null) { - overshadowed = Optional - .ofNullable(dataSourcesSnapshot.getOvershadowedSegments()) - .orElse(ImmutableSet.of()); - } + // Get 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 = params.getDataSourcesSnapshot().getOvershadowedSegments(); for (String tier : cluster.getTierNames()) { replicatorThrottler.updateReplicationState(tier); } DruidCoordinatorRuntimeParams paramsWithReplicationManager = params - .buildFromExistingWithoutAvailableSegments() + .buildFromExistingWithoutSegmentsMetadata() .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.getId())) { // Skipping overshadowed segments continue; 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 f023135b623a..f2a1a9fa51fd 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 @@ -76,7 +76,8 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) final CoordinatorStats stats = new CoordinatorStats(); if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { - Map> dataSources = params.getDataSources(); + Map> dataSources = + params.getUsedSegmentsTimelinesPerDataSource(); List compactionConfigList = dynamicConfig.getCompactionConfigs(); if (compactionConfigList != null && !compactionConfigList.isEmpty()) { 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/DruidCoordinatorSegmentInfoLoader.java index 2353247e3f30..1b7beab2a960 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/DruidCoordinatorSegmentInfoLoader.java @@ -19,14 +19,12 @@ package org.apache.druid.server.coordinator.helper; -import com.google.common.collect.Iterables; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.timeline.DataSegment; -import java.util.TreeSet; - public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper { private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class); @@ -41,50 +39,27 @@ public DruidCoordinatorSegmentInfoLoader(DruidCoordinator coordinator) @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - log.info("Starting coordination. Getting available segments."); + log.info("Starting coordination. Getting used segments."); - final Iterable dataSegments = coordinator.iterateAvailableDataSegments(); - if (dataSegments == null) { - log.info("Metadata store not polled yet, canceling this run."); - return null; + DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot(); + for (DataSegment segment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) { + if (segment.getSize() < 0) { + log.makeAlert("No size on a segment") + .addData("segment", segment) + .emit(); + } } - // 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 - // 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 - // 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( - dataSegments, - segment -> { - if (segment.getSize() < 0) { - log.makeAlert("No size on a segment") - .addData("segment", segment) - .emit(); - } - return segment; - } - ); - final TreeSet availableSegments = - DruidCoordinatorRuntimeParams.createAvailableSegmentsSet(availableSegmentsWithSizeChecking); - - // 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 : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) { log.debug(" %s", dataSegment); } } - log.info("Found [%,d] available segments.", availableSegments.size()); + log.info("Found [%,d] used segments.", params.getUsedSegments().size()); - return params.buildFromExisting() - .setAvailableSegments(availableSegments) - .build(); + return params; } } 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..302a91d3f08b 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 @@ -31,10 +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; /** + * 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.KillTask */ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper { @@ -46,12 +51,12 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper private long lastKillTime = 0; - private final MetadataSegmentManager segmentManager; + private final MetadataSegmentManager segmentsMetadata; private final IndexingServiceClient indexingServiceClient; @Inject public DruidCoordinatorSegmentKiller( - MetadataSegmentManager segmentManager, + MetadataSegmentManager segmentsMetadata, IndexingServiceClient indexingServiceClient, DruidCoordinatorConfig config ) @@ -75,30 +80,36 @@ public DruidCoordinatorSegmentKiller( this.maxSegmentsToKill ); - this.segmentManager = segmentManager; + this.segmentsMetadata = segmentsMetadata; this.indexingServiceClient = indexingServiceClient; } @Override public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) { - boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources(); - Collection whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources(); - - if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) { - log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled."); + boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources(); + Collection specificDataSourcesToKill = + params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn(); + + 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 = segmentsMetadata.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) { - final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill); + for (String dataSource : dataSourcesToKill) { + final Interval intervalToKill = findIntervalForKill(dataSource, maxSegmentsToKill); if (intervalToKill != null) { try { indexingServiceClient.killSegments(dataSource, intervalToKill); @@ -117,13 +128,11 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) } @VisibleForTesting - Interval findIntervalForKillTask(String dataSource, int limit) + @Nullable + Interval findIntervalForKill(String dataSource, int limit) { - List unusedSegmentIntervals = segmentManager.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/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..c565df9b58be 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.markSegmentAsUnused(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 a2f04fffa68c..7d4976711c35 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 @@ -22,10 +22,12 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Throwables; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.inject.Inject; import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.commons.lang.StringUtils; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidServer; @@ -50,7 +52,6 @@ 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; @@ -97,27 +98,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 MetadataSegmentManager segmentsMetadata; + 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, + MetadataSegmentManager segmentsMetadata, + MetadataRuleManager metadataRuleManager, @Nullable IndexingServiceClient indexingServiceClient, - AuthConfig authConfig, AuthorizerMapper authorizerMapper ) { this.serverInventoryView = serverInventoryView; - this.databaseSegmentManager = databaseSegmentManager; - this.databaseRuleManager = databaseRuleManager; + this.segmentsMetadata = segmentsMetadata; + this.metadataRuleManager = metadataRuleManager; this.indexingServiceClient = indexingServiceClient; - this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; } @@ -130,24 +128,17 @@ public Response getQueryableDataSources( ) { Response.ResponseBuilder builder = Response.ok(); - final Set datasources = InventoryViewUtils.getSecuredDataSources( - req, - serverInventoryView, - authorizerMapper - ); + final Set datasources = + InventoryViewUtils.getSecuredDataSources(req, serverInventoryView, authorizerMapper); final Object entity; if (full != null) { entity = datasources; } else if (simple != null) { - entity = datasources.stream() - .map(this::makeSimpleDatasource) - .collect(Collectors.toList()); + entity = datasources.stream().map(this::makeSimpleDatasource).collect(Collectors.toList()); } else { - entity = datasources.stream() - .map(ImmutableDruidDataSource::getName) - .collect(Collectors.toList()); + entity = datasources.stream().map(ImmutableDruidDataSource::getName).collect(Collectors.toList()); } return builder.entity(entity).build(); @@ -157,7 +148,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 ) @@ -165,7 +156,7 @@ public Response getTheDataSource( final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } if (full != null) { @@ -175,33 +166,128 @@ public Response getTheDataSource( return Response.ok(getSimpleDatasource(dataSourceName)).build(); } + private interface MarkSegments + { + int markSegments() throws UnknownSegmentIdException; + } + @POST @Path("/{dataSourceName}") @Consumes(MediaType.APPLICATION_JSON) @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDataSource( - @PathParam("dataSourceName") final String dataSourceName + public Response markAsUsedAllNonOvershadowedSegments(@PathParam("dataSourceName") final String dataSourceName) + { + MarkSegments markSegments = () -> segmentsMetadata.markAsUsedAllNonOvershadowedSegmentsInDataSource(dataSourceName); + return doMarkSegments("markAsUsedAllNonOvershadowedSegments", dataSourceName, markSegments); + } + + @POST + @Path("/{dataSourceName}/markUsed") + @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(DatasourceResourceFilter.class) + public Response markAsUsedNonOvershadowedSegments( + @PathParam("dataSourceName") String dataSourceName, + MarkDataSourceSegmentsPayload payload + ) + { + MarkSegments markSegments = () -> { + final Interval interval = payload.getInterval(); + if (interval != null) { + return segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName, interval); + } else { + final Set segmentIds = payload.getSegmentIds(); + return segmentsMetadata.markAsUsedNonOvershadowedSegments(dataSourceName, segmentIds); + } + }; + return doMarkSegmentsWithPayload("markAsUsedNonOvershadowedSegments", dataSourceName, payload, markSegments); + } + + @POST + @Path("/{dataSourceName}/markUnused") + @ResourceFilters(DatasourceResourceFilter.class) + @Produces(MediaType.APPLICATION_JSON) + @Consumes(MediaType.APPLICATION_JSON) + public Response markSegmentsAsUnused( + @PathParam("dataSourceName") final String dataSourceName, + final MarkDataSourceSegmentsPayload payload + ) + { + MarkSegments markSegments = () -> { + final Interval interval = payload.getInterval(); + if (interval != null) { + return segmentsMetadata.markAsUnusedSegmentsInInterval(dataSourceName, interval); + } else { + final Set segmentIds = payload.getSegmentIds(); + return segmentsMetadata.markSegmentsAsUnused(dataSourceName, segmentIds); + } + }; + return doMarkSegmentsWithPayload("markSegmentsAsUnused", dataSourceName, payload, markSegments); + } + + private Response doMarkSegmentsWithPayload( + String method, + String dataSourceName, + MarkDataSourceSegmentsPayload payload, + MarkSegments markSegments ) { - if (!databaseSegmentManager.enableDataSource(dataSourceName)) { - return Response.noContent().build(); + if (payload == null || !payload.isValid()) { + log.warn("Invalid request payload: [%s]", payload); + return Response + .status(Response.Status.BAD_REQUEST) + .entity("Invalid request payload, either interval or segmentIds array must be specified") + .build(); + } + + final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); + if (dataSource == null) { + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } - return Response.ok().build(); + return doMarkSegments(method, dataSourceName, markSegments); + } + + private static Response logAndCreateDataSourceNotFoundResponse(String dataSourceName) + { + log.warn("datasource not found [%s]", dataSourceName); + return Response.noContent().build(); + } + + private static Response doMarkSegments(String method, String dataSourceName, MarkSegments markSegments) + { + try { + int numChangedSegments = markSegments.markSegments(); + return Response.ok(ImmutableMap.of("numChangedSegments", numChangedSegments)).build(); + } + catch (UnknownSegmentIdException e) { + log.warn("Segment ids %s are not found", e.getUnknownSegmentIds()); + return Response + .status(Response.Status.NOT_FOUND) + .entity(ImmutableMap.of("message", e.getMessage())) + .build(); + } + catch (Exception e) { + log.error(e, "Error occurred during [%s] call, data source: [%s]", method, dataSourceName); + return Response + .serverError() + .entity(ImmutableMap.of("error", "Exception occurred.", "message", Throwables.getRootCause(e).toString())) + .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 markAsUnusedAllSegmentsOrKillSegmentsInInterval( @PathParam("dataSourceName") final String dataSourceName, @QueryParam("kill") final String kill, @QueryParam("interval") final String interval @@ -211,47 +297,20 @@ public Response deleteDataSource( return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } - if (kill != null && Boolean.valueOf(kill)) { - try { - indexingServiceClient.killSegments(dataSourceName, Intervals.of(interval)); - } - catch (IllegalArgumentException e) { - return Response.status(Response.Status.BAD_REQUEST) - .entity( - ImmutableMap.of( - "error", - "Exception occurred. Probably the interval is invalid", - "message", - e.toString() - ) - ) - .build(); - } - catch (Exception e) { - return Response.serverError().entity( - ImmutableMap.of( - "error", - "Exception occurred. Are you sure you have an indexing service?", - "message", - e.toString() - ) - ) - .build(); - } + boolean killSegments = kill != null && Boolean.valueOf(kill); + if (killSegments) { + return killSegmentsInInterval(dataSourceName, interval); } else { - if (!databaseSegmentManager.removeDataSource(dataSourceName)) { - return Response.noContent().build(); - } + MarkSegments markSegments = () -> segmentsMetadata.markAsUnusedAllSegmentsInDataSource(dataSourceName); + return doMarkSegments("markAsUnusedAllSegments", dataSourceName, markSegments); } - - return Response.ok().build(); } @DELETE @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 ) @@ -259,28 +318,32 @@ public Response deleteDataSourceSpecificInterval( if (indexingServiceClient == null) { return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); } + if (StringUtils.contains(interval, '_')) { + log.warn("Use interval with '/', not '_': [%s] given", interval); + } final Interval theInterval = Intervals.of(interval.replace('_', '/')); try { indexingServiceClient.killSegments(dataSourceName, theInterval); + return Response.ok().build(); } catch (Exception e) { - return Response.serverError() - .entity(ImmutableMap.of( - "error", - "Exception occurred. Are you sure you have an indexing service?", - "message", - e.toString() - )) - .build(); - } - return Response.ok().build(); + return Response + .serverError() + .entity( + ImmutableMap.of( + "error", "Exception occurred. Are you sure you have an indexing service?", + "message", e.toString() + ) + ) + .build(); + } } @GET @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 @@ -289,21 +352,22 @@ public Response getSegmentDataSourceIntervals( if (simple == null && full == null) { final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); Set intervals = new TreeSet<>(comparator); dataSource.getSegments().forEach(segment -> intervals.add(segment.getInterval())); return Response.ok(intervals).build(); + } else { + return getServedSegmentsInInterval(dataSourceName, full != null, interval -> true); } - return getSegmentDataSourceIntervals(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, @@ -314,7 +378,7 @@ public Response getSegmentDataSourceSpecificInterval( if (simple == null && full == null) { final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } final Set segmentIds = new TreeSet<>(); for (DataSegment dataSegment : dataSource.getSegments()) { @@ -324,7 +388,7 @@ public Response getSegmentDataSourceSpecificInterval( } return Response.ok(segmentIds).build(); } - return getSegmentDataSourceIntervals(dataSourceName, full != null, theInterval::contains); + return getServedSegmentsInInterval(dataSourceName, full != null, theInterval::contains); } /** @@ -337,7 +401,7 @@ enum SimpleProperties count } - private Response getSegmentDataSourceIntervals( + private Response getServedSegmentsInInterval( String dataSourceName, boolean full, Predicate intervalFilter @@ -346,7 +410,7 @@ private Response getSegmentDataSourceIntervals( final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } final Comparator comparator = Comparators.intervalsByStartThenEnd().reversed(); @@ -357,10 +421,13 @@ private Response getSegmentDataSourceIntervals( if (intervalFilter.test(dataSegment.getInterval())) { Map segments = retVal.computeIfAbsent(dataSegment.getInterval(), i -> new HashMap<>()); - Pair> val = getServersWhereSegmentIsServed(dataSegment.getId()); + Pair> segmentAndServers = getServersWhereSegmentIsServed(dataSegment.getId()); - if (val != null) { - segments.put(dataSegment.getId(), ImmutableMap.of("metadata", val.lhs, "servers", val.rhs)); + if (segmentAndServers != null) { + segments.put( + dataSegment.getId(), + ImmutableMap.of("metadata", segmentAndServers.lhs, "servers", segmentAndServers.rhs) + ); } } } @@ -385,14 +452,14 @@ 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 ) { ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } Response.ResponseBuilder builder = Response.ok(); @@ -407,14 +474,14 @@ 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 ) { ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); if (dataSource == null) { - return Response.noContent().build(); + return logAndCreateDataSourceNotFoundResponse(dataSourceName); } for (SegmentId possibleSegmentId : SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId)) { @@ -423,44 +490,40 @@ public Response getSegmentDataSourceSegment( return Response.ok(ImmutableMap.of("metadata", retVal.lhs, "servers", retVal.rhs)).build(); } } + log.warn("Segment id [%s] is unknown", segmentId); return Response.noContent().build(); } @DELETE @Path("/{dataSourceName}/segments/{segmentId}") @ResourceFilters(DatasourceResourceFilter.class) - public Response deleteDatasourceSegment( + public Response markSegmentAsUnused( @PathParam("dataSourceName") String dataSourceName, @PathParam("segmentId") String segmentId ) { - if (databaseSegmentManager.removeSegment(segmentId)) { - return Response.ok().build(); - } - return Response.noContent().build(); + boolean segmentStateChanged = segmentsMetadata.markSegmentAsUnused(segmentId); + return Response.ok(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build(); } @POST @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(); - } - - return Response.ok().build(); + boolean segmentStateChanged = segmentsMetadata.markSegmentAsUsed(segmentId); + return Response.ok().entity(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build(); } @GET @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()) { @@ -475,20 +538,22 @@ public Response getSegmentDataSourceTiers(@PathParam("dataSourceName") String da @Nullable private ImmutableDruidDataSource getDataSource(final String dataSourceName) { - List dataSources = serverInventoryView + List dataSources = serverInventoryView .getInventory() .stream() .map(server -> server.getDataSource(dataSourceName)) .filter(Objects::nonNull) - .map(DruidDataSource::toImmutableDruidDataSource) .collect(Collectors.toList()); if (dataSources.isEmpty()) { return null; } + // Note: this logic doesn't guarantee that the result is a snapshot that ever existed in the cluster because all + // DruidDataSource objects (belonging to different servers) are independently, concurrently mutable objects. + // But this is OK because a "snapshot" hardly even makes sense in a distributed system anyway. final SortedMap segmentMap = new TreeMap<>(); - for (ImmutableDruidDataSource dataSource : dataSources) { + for (DruidDataSource dataSource : dataSources) { Iterable segments = dataSource.getSegments(); for (DataSegment segment : segments) { segmentMap.put(segment.getId(), segment); @@ -551,7 +616,7 @@ private Map> getSimpleDatasource(String dataSourceNa continue; } - tierDistinctSegments.computeIfAbsent(tier, k -> new HashSet<>()); + tierDistinctSegments.computeIfAbsent(tier, t -> new HashSet<>()); long dataSourceSegmentSize = 0; long replicatedSegmentSize = 0; @@ -597,14 +662,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 @@ -619,17 +684,25 @@ public Response getSegmentDataSourceSpecificInterval( return Response.ok(new ArrayList()).build(); } - Iterable> lookup = timeline.lookupWithIncompletePartitions(theInterval); - FunctionalIterable retval = FunctionalIterable - .create(lookup).transformCat( + return Response.ok(prepareServedSegmentsInInterval(timeline, theInterval)).build(); + } + + private Iterable prepareServedSegmentsInInterval( + TimelineLookup dataSourceServingTimeline, + Interval interval + ) + { + Iterable> lookup = + dataSourceServingTimeline.lookupWithIncompletePartitions(interval); + return FunctionalIterable + .create(lookup) + .transformCat( (TimelineObjectHolder input) -> Iterables.transform( input.getObject(), - (PartitionChunk chunk) -> - chunk.getObject().toImmutableSegmentLoadInfo() + (PartitionChunk chunk) -> chunk.getObject().toImmutableSegmentLoadInfo() ) ); - return Response.ok(retval).build(); } /** @@ -648,7 +721,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(); @@ -675,18 +748,9 @@ public Response isHandOffComplete( return Response.ok(false).build(); } - Iterable> lookup = timeline.lookupWithIncompletePartitions( - theInterval); - FunctionalIterable loadInfoIterable = FunctionalIterable - .create(lookup).transformCat( - (TimelineObjectHolder input) -> - Iterables.transform( - input.getObject(), - (PartitionChunk chunk) -> - chunk.getObject().toImmutableSegmentLoadInfo() - ) - ); - if (isSegmentLoaded(loadInfoIterable, descriptor)) { + Iterable servedSegmentsInInterval = + prepareServedSegmentsInInterval(timeline, theInterval); + if (isSegmentLoaded(servedSegmentsInInterval, descriptor)) { return Response.ok(true).build(); } @@ -698,58 +762,9 @@ public Response isHandOffComplete( } } - @POST - @Path("/{dataSourceName}/markUnused") - @ResourceFilters(DatasourceResourceFilter.class) - @Produces(MediaType.APPLICATION_JSON) - @Consumes(MediaType.APPLICATION_JSON) - public Response markDatasourceUnused( - @PathParam("dataSourceName") final String dataSourceName, - final MarkDatasourceSegmentsPayload payload - ) + static boolean isSegmentLoaded(Iterable servedSegments, SegmentDescriptor descriptor) { - if (payload == null || !payload.isValid()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Invalid request payload, either interval or segmentIds array must be specified") - .build(); - } - - final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); - if (dataSource == null) { - log.warn("datasource not found [%s]", dataSourceName); - return Response.noContent().build(); - } - - long markedSegmentCount = 0; - try { - final Interval interval = payload.getInterval(); - final Set segmentIds = payload.getSegmentIds(); - if (interval != null) { - markedSegmentCount = databaseSegmentManager.disableSegments(dataSourceName, interval); - } else if (segmentIds != null) { - markedSegmentCount = databaseSegmentManager.disableSegments(dataSourceName, segmentIds); - } - } - catch (Exception e) { - return Response.serverError().entity( - ImmutableMap.of( - "error", - "Exception occurred.", - "message", - e.toString() - ) - ).build(); - - } - if (markedSegmentCount == 0) { - return Response.noContent().build(); - } - return Response.ok().build(); - } - - static boolean isSegmentLoaded(Iterable serverView, SegmentDescriptor descriptor) - { - for (ImmutableSegmentLoadInfo segmentLoadInfo : serverView) { + for (ImmutableSegmentLoadInfo segmentLoadInfo : servedSegments) { if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 @@ -762,68 +777,14 @@ static boolean isSegmentLoaded(Iterable serverView, Se return false; } - @POST - @Path("/{dataSourceName}/markUsed") - @Produces(MediaType.APPLICATION_JSON) - @ResourceFilters(DatasourceResourceFilter.class) - public Response enableDatasourceSegments( - @PathParam("dataSourceName") String dataSourceName, - MarkDatasourceSegmentsPayload payload - ) - { - if (payload == null || !payload.isValid()) { - return Response.status(Response.Status.BAD_REQUEST) - .entity("Invalid request payload, either interval or segmentIds array must be specified") - .build(); - } - - final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); - if (dataSource == null) { - return Response.noContent().build(); - } - - int modified; - try { - if (payload.getInterval() != null) { - modified = databaseSegmentManager.enableSegments(dataSource.getName(), payload.getInterval()); - } else { - modified = databaseSegmentManager.enableSegments(dataSource.getName(), payload.getSegmentIds()); - } - } - catch (Exception e) { - if (e.getCause() instanceof UnknownSegmentIdException) { - return Response.status(Response.Status.NOT_FOUND).entity( - ImmutableMap.of( - "message", - e.getCause().getMessage() - ) - ).build(); - } - return Response.serverError().entity( - ImmutableMap.of( - "error", - "Exception occurred.", - "message", - e.getMessage() - ) - ).build(); - } - - if (modified == 0) { - return Response.noContent().build(); - } - - return Response.ok().build(); - } - @VisibleForTesting - protected static class MarkDatasourceSegmentsPayload + protected static class MarkDataSourceSegmentsPayload { private final Interval interval; private final Set segmentIds; @JsonCreator - public MarkDatasourceSegmentsPayload( + public MarkDataSourceSegmentsPayload( @JsonProperty("interval") Interval interval, @JsonProperty("segmentIds") Set segmentIds ) 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 556ed3d8dd49..f4cbcfa3088d 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 @@ -23,15 +23,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.DataSourcesSnapshot; 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.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; @@ -40,6 +40,7 @@ import org.apache.druid.timeline.SegmentWithOvershadowedStatus; import org.joda.time.Interval; +import javax.annotation.Nullable; import javax.servlet.http.HttpServletRequest; import javax.ws.rs.GET; import javax.ws.rs.POST; @@ -50,12 +51,13 @@ import javax.ws.rs.core.Context; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriInfo; import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.TreeSet; +import java.util.stream.Collectors; import java.util.stream.Stream; /** @@ -63,20 +65,19 @@ @Path("/druid/coordinator/v1/metadata") public class MetadataResource { - private final MetadataSegmentManager metadataSegmentManager; + private final MetadataSegmentManager segmentsMetadata; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final AuthorizerMapper authorizerMapper; @Inject public MetadataResource( - MetadataSegmentManager metadataSegmentManager, + MetadataSegmentManager segmentsMetadata, IndexerMetadataStorageCoordinator metadataStorageCoordinator, - AuthConfig authConfig, AuthorizerMapper authorizerMapper, @Json ObjectMapper jsonMapper ) { - this.metadataSegmentManager = metadataSegmentManager; + this.segmentsMetadata = segmentsMetadata; this.metadataStorageCoordinator = metadataStorageCoordinator; this.authorizerMapper = authorizerMapper; } @@ -84,29 +85,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 ) { - // If we haven't polled the metadata store yet, use an empty list of datasources. - final Collection druidDataSources = Optional.ofNullable(metadataSegmentManager.getDataSources()) - .orElse(Collections.emptyList()); - + final boolean includeUnused = JettyUtils.getQueryParam(uriInfo, "includeUnused", "includeDisabled") != null; + Collection druidDataSources = null; final Set dataSourceNamesPreAuth; - if (includeDisabled != null) { - dataSourceNamesPreAuth = new TreeSet<>(metadataSegmentManager.getAllDataSourceNames()); + if (includeUnused) { + dataSourceNamesPreAuth = new TreeSet<>(segmentsMetadata.retrieveAllDataSourceNames()); } else { - dataSourceNamesPreAuth = Sets.newTreeSet( - Iterables.transform(druidDataSources, ImmutableDruidDataSource::getName) - ); + druidDataSources = segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments(); + 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, @@ -118,9 +118,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) { return Response.ok().entity( Collections2.filter(druidDataSources, dataSource -> dataSourceNamesPostAuth.contains(dataSource.getName())) ).build(); @@ -129,74 +129,61 @@ 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 @Nullable Set dataSources, @QueryParam("includeOvershadowedStatus") final String includeOvershadowedStatus ) { - // If we haven't polled the metadata store yet, use an empty list of datasources. - Collection druidDataSources = Optional.ofNullable(metadataSegmentManager.getDataSources()) - .orElse(Collections.emptyList()); - Stream dataSourceStream = druidDataSources.stream(); - if (datasources != null && !datasources.isEmpty()) { - dataSourceStream = dataSourceStream.filter(src -> datasources.contains(src.getName())); + if (includeOvershadowedStatus != null) { + return getAllUsedSegmentsWithOvershadowedStatus(req, dataSources); } - final Stream metadataSegments = dataSourceStream.flatMap(t -> t.getSegments().stream()); - if (includeOvershadowedStatus != null) { - final Iterable authorizedSegments = - findAuthorizedSegmentWithOvershadowedStatus( - req, - metadataSegments - ); - Response.ResponseBuilder builder = Response.status(Response.Status.OK); - return builder.entity(authorizedSegments).build(); - } else { + Collection dataSourcesWithUsedSegments = + segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments(); + if (dataSources != null && !dataSources.isEmpty()) { + dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + .stream() + .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) + .collect(Collectors.toList()); + } + final Stream usedSegments = dataSourcesWithUsedSegments + .stream() + .flatMap(t -> t.getSegments().stream()); - final Function> raGenerator = segment -> Collections.singletonList( - AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); + final Function> raGenerator = segment -> Collections.singletonList( + AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource())); - final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( - req, - metadataSegments::iterator, - raGenerator, - authorizerMapper - ); + final Iterable authorizedSegments = + AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper); - Response.ResponseBuilder builder = Response.status(Response.Status.OK); - return builder.entity(authorizedSegments).build(); - } + Response.ResponseBuilder builder = Response.status(Response.Status.OK); + return builder.entity(authorizedSegments).build(); } - private Iterable findAuthorizedSegmentWithOvershadowedStatus( + private Response getAllUsedSegmentsWithOvershadowedStatus( HttpServletRequest req, - Stream metadataSegments + @Nullable Set dataSources ) { - // If metadata store hasn't been polled yet, use empty overshadowed list - final Set overshadowedSegments = Optional - .ofNullable(metadataSegmentManager.getOvershadowedSegments()) - .orElse(Collections.emptySet()); + DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments(); + Collection dataSourcesWithUsedSegments = + dataSourcesSnapshot.getDataSourcesWithAllUsedSegments(); + if (dataSources != null && !dataSources.isEmpty()) { + dataSourcesWithUsedSegments = dataSourcesWithUsedSegments + .stream() + .filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName())) + .collect(Collectors.toList()); + } + final Stream usedSegments = dataSourcesWithUsedSegments + .stream() + .flatMap(t -> t.getSegments().stream()); + final Set overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments(); - final Stream segmentsWithOvershadowedStatus = metadataSegments + final Stream usedSegmentsWithOvershadowedStatus = usedSegments .map(segment -> new SegmentWithOvershadowedStatus( segment, overshadowedSegments.contains(segment.getId()) @@ -207,23 +194,45 @@ private Iterable findAuthorizedSegmentWithOversha final Iterable authorizedSegments = AuthorizationUtils.filterAuthorizedResources( req, - segmentsWithOvershadowedStatus::iterator, + usedSegmentsWithOvershadowedStatus::iterator, raGenerator, authorizerMapper ); - return authorizedSegments; + + Response.ResponseBuilder builder = Response.status(Response.Status.OK); + return builder.entity(authorizedSegments).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 = + segmentsMetadata.getImmutableDataSourceWithUsedSegments(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 = + segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } @@ -236,11 +245,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 @@ -260,12 +273,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); + ImmutableDruidDataSource dataSource = segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName); if (dataSource == null) { return Response.status(Response.Status.NOT_FOUND).build(); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 8170014c3e6a..2a8e755cdad1 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2198,7 +2198,8 @@ private List> populateTimeline( serverExpectationList.add(serverExpectations); for (int j = 0; j < numChunks; ++j) { DruidServer lastServer = servers[random.nextInt(servers.length)]; - serverExpectations.computeIfAbsent(lastServer, server -> new ServerExpectations(lastServer, makeMock(mocks, QueryRunner.class))); + serverExpectations + .computeIfAbsent(lastServer, server -> new ServerExpectations(server, makeMock(mocks, QueryRunner.class))); DataSegment mockSegment = makeMock(mocks, DataSegment.class); ServerExpectation expectation = new ServerExpectation<>( 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..fea252f146f5 --- /dev/null +++ b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java @@ -0,0 +1,39 @@ +/* + * 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/client/indexing/ClientKillQueryTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java index ebd3b6f4438e..0b6c235b6a7d 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java @@ -32,35 +32,35 @@ public class ClientKillQueryTest 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; + ClientKillQuery clientKillUnusedSegmentsQuery; @Before public void setUp() { - clientKillQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL); + clientKillUnusedSegmentsQuery = new ClientKillQuery(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/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index a5d436b226e3..a1852be765e3 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.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.client.ImmutableDruidDataSource; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; @@ -33,14 +34,13 @@ import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; -import org.hamcrest.core.IsInstanceOf; import org.joda.time.Interval; +import org.joda.time.Period; import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; -import org.junit.rules.ExpectedException; import java.io.IOException; import java.util.stream.Collectors; @@ -48,57 +48,64 @@ public class SQLMetadataSegmentManagerTest { - @Rule - public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); + private static DataSegment createSegment( + String dataSource, + String interval, + String version, + String bucketKey, + int binaryVersion + ) + { + return new DataSegment( + dataSource, + Intervals.of(interval), + version, + ImmutableMap.of( + "type", "s3_zip", + "bucket", "test", + "key", dataSource + "/" + bucketKey + ), + ImmutableList.of("dim1", "dim2", "dim3"), + ImmutableList.of("count", "value"), + NoneShardSpec.instance(), + binaryVersion, + 1234L + ); + } @Rule - public ExpectedException thrown = ExpectedException.none(); + public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule(); - private SQLMetadataSegmentManager manager; + private SQLMetadataSegmentManager sqlSegmentsMetadata; private SQLMetadataSegmentPublisher publisher; private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); - private final DataSegment segment1 = new DataSegment( + private final DataSegment segment1 = createSegment( "wikipedia", - Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000"), + "2012-03-15T00:00:00.000/2012-03-16T00:00:00.000", "2012-03-16T00:36:30.848Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia/index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip", + 0 ); - private final DataSegment segment2 = new DataSegment( + private final DataSegment segment2 = createSegment( "wikipedia", - Intervals.of("2012-01-05T00:00:00.000/2012-01-06T00:00:00.000"), + "2012-01-05T00:00:00.000/2012-01-06T00:00:00.000", "2012-01-06T22:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip", + 0 ); private void publish(DataSegment segment, boolean used) throws IOException { + boolean partitioned = !(segment.getShardSpec() instanceof NoneShardSpec); publisher.publishSegment( segment.getId().toString(), segment.getDataSource(), DateTimes.nowUtc().toString(), segment.getInterval().getStart().toString(), segment.getInterval().getEnd().toString(), - (segment.getShardSpec() instanceof NoneShardSpec) ? false : true, + partitioned, segment.getVersion(), used, jsonMapper.writeValueAsBytes(segment) @@ -109,12 +116,15 @@ private void publish(DataSegment segment, boolean used) throws IOException public void setUp() throws Exception { TestDerbyConnector connector = derbyConnectorRule.getConnector(); - manager = new SQLMetadataSegmentManager( + MetadataSegmentManagerConfig config = new MetadataSegmentManagerConfig(); + config.setPollDuration(Period.seconds(1)); + sqlSegmentsMetadata = new SQLMetadataSegmentManager( jsonMapper, - Suppliers.ofInstance(new MetadataSegmentManagerConfig()), + Suppliers.ofInstance(config), derbyConnectorRule.metadataTablesConfigSupplier(), connector ); + sqlSegmentsMetadata.start(); publisher = new SQLMetadataSegmentPublisher( jsonMapper, @@ -131,67 +141,104 @@ public void setUp() throws Exception @After public void teardown() { - if (manager.isStarted()) { - manager.stop(); + if (sqlSegmentsMetadata.isPollingDatabasePeriodically()) { + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); } + sqlSegmentsMetadata.stop(); } @Test public void testPoll() { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + sqlSegmentsMetadata.retrieveAllDataSourceNames() ); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getDataSources().stream().map(d -> d.getName()).collect(Collectors.toList()) + sqlSegmentsMetadata + .getImmutableDataSourcesWithAllUsedSegments() + .stream() + .map(ImmutableDruidDataSource::getName) + .collect(Collectors.toList()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.getDataSource("wikipedia").getSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia").getSegments()) ); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testNoPoll() + public void testPrepareImmutableDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOException { - manager.start(); - Assert.assertTrue(manager.isStarted()); + DataSegment newSegment = pollThenStopThenStartIntro(); Assert.assertEquals( - ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + ImmutableSet.of(newSegment), + ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia2").getSegments()) + ); + } + + @Test + public void testGetDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOException + { + DataSegment newSegment = pollThenStopThenStartIntro(); + Assert.assertEquals( + ImmutableSet.of(newSegment), + ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia2").getSegments()) + ); + } + + @Test + public void testPrepareImmutableDataSourcesWithAllUsedSegmentsAwaitsPollOnRestart() throws IOException + { + DataSegment newSegment = pollThenStopThenStartIntro(); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2, newSegment), + ImmutableSet.copyOf( + sqlSegmentsMetadata + .getImmutableDataSourcesWithAllUsedSegments() + .stream() + .flatMap((ImmutableDruidDataSource dataSource) -> dataSource.getSegments().stream()) + .iterator() + ) ); - Assert.assertNull(manager.getDataSources()); - Assert.assertNull(manager.getDataSource("wikipedia")); - Assert.assertNull(manager.iterateAllSegments()); } @Test - public void testPollThenStop() + public void testIterateAllUsedSegmentsAwaitsPollOnRestart() throws IOException + { + DataSegment newSegment = pollThenStopThenStartIntro(); + Assert.assertEquals( + ImmutableSet.of(segment1, segment2, newSegment), + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) + ); + } + + private DataSegment pollThenStopThenStartIntro() throws IOException { - manager.start(); - manager.poll(); - manager.stop(); - Assert.assertFalse(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); + Assert.assertFalse(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( ImmutableList.of("wikipedia"), - manager.getAllDataSourceNames() + sqlSegmentsMetadata.retrieveAllDataSourceNames() ); - Assert.assertNull(manager.getDataSources()); - Assert.assertNull(manager.getDataSource("wikipedia")); - Assert.assertNull(manager.iterateAllSegments()); + DataSegment newSegment = createNewSegment1("wikipedia2"); + publisher.publishSegment(newSegment); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + return newSegment; } @Test - public void testPollWithCurroptedSegment() + public void testPollWithCorruptedSegment() { //create a corrupted segment entry in segments table, which tests //that overall loading of segments from database continues to work @@ -209,355 +256,246 @@ public void testPollWithCurroptedSegment() ); EmittingLogger.registerEmitter(new NoopServiceEmitter()); - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( - "wikipedia", Iterables.getOnlyElement(manager.getDataSources()).getName() + "wikipedia", + Iterables.getOnlyElement(sqlSegmentsMetadata.getImmutableDataSourcesWithAllUsedSegments()).getName() ); } @Test - public void testGetUnusedSegmentsForInterval() + public void testGetUnusedSegmentIntervals() { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - Assert.assertTrue(manager.removeDataSource("wikipedia")); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource("wikipedia"); + Assert.assertEquals(2, numChangedSegments); Assert.assertEquals( ImmutableList.of(segment2.getInterval()), - manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 1) + sqlSegmentsMetadata.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 1) ); Assert.assertEquals( ImmutableList.of(segment2.getInterval(), segment1.getInterval()), - manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 5) + sqlSegmentsMetadata.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 5) ); } - @Test - public void testRemoveDataSource() throws IOException + @Test(timeout = 60_000) + public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException, InterruptedException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); final String newDataSource = "wikipedia2"; - final DataSegment newSegment = new DataSegment( + final DataSegment newSegment = createNewSegment1(newDataSource); + + publisher.publishSegment(newSegment); + + awaitDataSourceAppeared(newDataSource); + int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource(newDataSource); + Assert.assertEquals(1, numChangedSegments); + awaitDataSourceDisappeared(newDataSource); + Assert.assertNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource)); + } + + private static DataSegment createNewSegment1(String newDataSource) + { + return createSegment( newDataSource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), + "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); + } - publisher.publishSegment(newSegment); - - Assert.assertNull(manager.getDataSource(newDataSource)); - Assert.assertTrue(manager.removeDataSource(newDataSource)); + private static DataSegment createNewSegment2(String newDataSource) + { + return createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", + "2017-10-15T20:19:12.565Z", + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 + ); } - @Test - public void testRemoveDataSegment() throws IOException + @Test(timeout = 60_000) + public void testMarkSegmentAsUnused() throws IOException, InterruptedException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); final String newDataSource = "wikipedia2"; - final DataSegment newSegment = new DataSegment( + final DataSegment newSegment = createSegment( newDataSource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), + "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); publisher.publishSegment(newSegment); + awaitDataSourceAppeared(newDataSource); + Assert.assertNotNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource)); + + Assert.assertTrue(sqlSegmentsMetadata.markSegmentAsUnused(newSegment.getId().toString())); + awaitDataSourceDisappeared(newDataSource); + Assert.assertNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource)); + } - Assert.assertNull(manager.getDataSource(newDataSource)); - Assert.assertTrue(manager.removeSegment(newSegment.getId().toString())); + private void awaitDataSourceAppeared(String newDataSource) throws InterruptedException + { + while (sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource) == null) { + Thread.sleep(1000); + } + } + + private void awaitDataSourceDisappeared(String dataSource) throws InterruptedException + { + while (sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSource) != null) { + Thread.sleep(1000); + } } @Test - public void testEnableSegmentsWithSegmentIds() throws IOException + public void testMarkAsUsedNonOvershadowedSegments() throws Exception { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"), + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createSegment( + newDataSource, + "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment2 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); // Overshadowed by newSegment2 - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); publish(newSegment1, false); publish(newSegment2, false); publish(newSegment3, false); - final ImmutableList segmentIds = ImmutableList.of( + final ImmutableSet segmentIds = ImmutableSet.of( newSegment1.getId().toString(), newSegment2.getId().toString(), newSegment3.getId().toString() ); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - Assert.assertEquals(2, manager.enableSegments(datasource, segmentIds)); - manager.poll(); + Assert.assertEquals(2, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments(newDataSource, segmentIds)); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } - @Test - public void testEnableSegmentsWithSegmentIdsInvalidDatasource() throws IOException + @Test(expected = UnknownSegmentIdException.class) + public void testMarkAsUsedNonOvershadowedSegmentsInvalidDataSource() throws Exception { - thrown.expectCause(IsInstanceOf.instanceOf(UnknownSegmentIdException.class)); - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment1(newDataSource); publish(newSegment1, false); publish(newSegment2, false); - final ImmutableList segmentIds = ImmutableList.of( - newSegment1.getId().toString(), - newSegment2.getId().toString() - ); - manager.poll(); + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString()); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - // none of the segments are in datasource - Assert.assertEquals(0, manager.enableSegments("wrongDataSource", segmentIds)); + // none of the segments are in data source + Assert.assertEquals(0, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments("wrongDataSource", segmentIds)); } - @Test - public void testEnableSegmentsWithInvalidSegmentIds() + @Test(expected = UnknownSegmentIdException.class) + public void testMarkAsUsedNonOvershadowedSegmentsWithInvalidSegmentIds() throws UnknownSegmentIdException { - thrown.expectCause(IsInstanceOf.instanceOf(UnknownSegmentIdException.class)); - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); - final ImmutableList segmentIds = ImmutableList.of( - newSegment1.getId().toString(), - newSegment2.getId().toString() - ); - manager.poll(); + final DataSegment newSegment2 = createNewSegment1(newDataSource); + + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString()); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - // none of the segments are in datasource - Assert.assertEquals(0, manager.enableSegments(datasource, segmentIds)); + // none of the segments are in data source + Assert.assertEquals(0, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments(newDataSource, segmentIds)); } @Test - public void testEnableSegmentsWithInterval() throws IOException + public void testMarkAsUsedNonOvershadowedSegmentsInInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment2 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-20T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); // Overshadowed by newSegment2 - final DataSegment newSegment4 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final DataSegment newSegment4 = createNewSegment2(newDataSource); publish(newSegment1, false); publish(newSegment2, false); @@ -565,141 +503,75 @@ public void testEnableSegmentsWithInterval() throws IOException publish(newSegment4, false); final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); // 2 out of 3 segments match the interval - Assert.assertEquals(2, manager.enableSegments(datasource, theInterval)); + Assert.assertEquals(2, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test(expected = IllegalArgumentException.class) - public void testEnableSegmentsWithInvalidInterval() throws IOException + public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithInvalidInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment2(newDataSource); publish(newSegment1, false); publish(newSegment2, false); // invalid interval start > end final Interval theInterval = Intervals.of("2017-10-22T00:00:00.000/2017-10-02T00:00:00.000"); - manager.enableSegments(datasource, theInterval); + sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval); } @Test - public void testEnableSegmentsWithOverlappingInterval() throws IOException + public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithOverlappingInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"), + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); + + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createSegment( + newDataSource, + "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final DataSegment newSegment2 = createSegment( + newDataSource, + "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000", "2017-10-16T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 1, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 1 ); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-22T00:00:00.000"), + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); // Overshadowed by newSegment2 - final DataSegment newSegment4 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final DataSegment newSegment4 = createNewSegment2(newDataSource); publish(newSegment1, false); publish(newSegment2, false); @@ -707,183 +579,90 @@ public void testEnableSegmentsWithOverlappingInterval() throws IOException publish(newSegment4, false); final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); - // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be disabled - Assert.assertEquals(1, manager.enableSegments(datasource, theInterval)); + // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused + Assert.assertEquals(1, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testDisableSegmentsWithSegmentIds() throws IOException + public void testMarkSegmentsAsUnused() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment1(newDataSource); publisher.publishSegment(newSegment1); publisher.publishSegment(newSegment2); - final ImmutableList segmentIds = ImmutableList.of(newSegment1.getId().toString(), newSegment1.getId().toString()); + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment1.getId().toString()); - Assert.assertEquals(segmentIds.size(), manager.disableSegments(datasource, segmentIds)); - manager.poll(); + Assert.assertEquals(segmentIds.size(), sqlSegmentsMetadata.markSegmentsAsUnused(newDataSource, segmentIds)); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testDisableSegmentsWithSegmentIdsInvalidDatasource() throws IOException + public void testMarkSegmentsAsUnusedInvalidDataSource() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment1(newDataSource); publisher.publishSegment(newSegment1); publisher.publishSegment(newSegment2); - final ImmutableList segmentIds = ImmutableList.of( - newSegment1.getId().toString(), - newSegment2.getId().toString() - ); - // none of the segments are in datasource - Assert.assertEquals(0, manager.disableSegments("wrongDataSource", segmentIds)); - manager.poll(); + final ImmutableSet segmentIds = + ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString()); + // none of the segments are in data source + Assert.assertEquals(0, sqlSegmentsMetadata.markSegmentsAsUnused("wrongDataSource", segmentIds)); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment2), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test - public void testDisableSegmentsWithInterval() throws IOException + public void testMarkAsUnusedSegmentsInInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-20T00:00:00.000"), + final DataSegment newSegment2 = createNewSegment2(newDataSource); + + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); publisher.publishSegment(newSegment1); @@ -892,116 +671,58 @@ public void testDisableSegmentsWithInterval() throws IOException final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000"); // 2 out of 3 segments match the interval - Assert.assertEquals(2, manager.disableSegments(datasource, theInterval)); + Assert.assertEquals(2, sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment3), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @Test(expected = IllegalArgumentException.class) - public void testDisableSegmentsWithInvalidInterval() throws IOException + public void testMarkAsUnusedSegmentsInIntervalWithInvalidInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createNewSegment1(newDataSource); + + final DataSegment newSegment2 = createNewSegment2(newDataSource); publisher.publishSegment(newSegment1); publisher.publishSegment(newSegment2); // invalid interval start > end final Interval theInterval = Intervals.of("2017-10-22T00:00:00.000/2017-10-02T00:00:00.000"); - manager.disableSegments(datasource, theInterval); + sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval); } @Test - public void testDisableSegmentsWithOverlappingInterval() throws IOException + public void testMarkAsUnusedSegmentsInIntervalWithOverlappingInterval() throws IOException { - manager.start(); - manager.poll(); - Assert.assertTrue(manager.isStarted()); - - final String datasource = "wikipedia2"; - final DataSegment newSegment1 = new DataSegment( - datasource, - Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"), - "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L - ); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.poll(); + Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); - final DataSegment newSegment2 = new DataSegment( - datasource, - Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"), + final String newDataSource = "wikipedia2"; + final DataSegment newSegment1 = createSegment( + newDataSource, + "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); - final DataSegment newSegment3 = new DataSegment( - datasource, - Intervals.of("2017-10-19T00:00:00.000/2017-10-22T00:00:00.000"), + final DataSegment newSegment2 = createNewSegment2(newDataSource); + + final DataSegment newSegment3 = createSegment( + newDataSource, + "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000", "2017-10-15T20:19:12.565Z", - ImmutableMap.of( - "type", "s3_zip", - "bucket", "test", - "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip" - ), - ImmutableList.of("dim1", "dim2", "dim3"), - ImmutableList.of("count", "value"), - NoneShardSpec.instance(), - 0, - 1234L + "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip", + 0 ); publisher.publishSegment(newSegment1); @@ -1009,13 +730,13 @@ public void testDisableSegmentsWithOverlappingInterval() throws IOException publisher.publishSegment(newSegment3); final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000"); - // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be disabled - Assert.assertEquals(1, manager.disableSegments(datasource, theInterval)); + // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused + Assert.assertEquals(1, sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval)); - manager.poll(); + sqlSegmentsMetadata.poll(); Assert.assertEquals( ImmutableSet.of(segment1, segment2, newSegment1, newSegment3), - ImmutableSet.copyOf(manager.iterateAllSegments()) + ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments()) ); } @@ -1023,9 +744,9 @@ public void testDisableSegmentsWithOverlappingInterval() throws IOException public void testStopAndStart() { // Simulate successive losing and getting the coordinator leadership - manager.start(); - manager.stop(); - manager.start(); - manager.stop(); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); + sqlSegmentsMetadata.startPollingDatabasePeriodically(); + sqlSegmentsMetadata.stopPollingDatabasePeriodically(); } } diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java index 4d7d1108dd91..9db997be895c 100644 --- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java +++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java @@ -41,13 +41,7 @@ public TestUsedSegmentChecker(AppenderatorTester appenderatorTester) public Set findUsedSegments(Set identifiers) { final VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>(Ordering.natural()); - for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { - timeline.add( - dataSegment.getInterval(), - dataSegment.getVersion(), - dataSegment.getShardSpec().createChunk(dataSegment) - ); - } + VersionedIntervalTimeline.addSegments(timeline, appenderatorTester.getPushedSegments().iterator()); final Set retVal = new HashSet<>(); for (SegmentIdWithShardSpec identifier : identifiers) { diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java new file mode 100644 index 000000000000..dfc98e5cc3ca --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import org.apache.druid.java.util.common.DateTimes; + +public class CoordinatorRuntimeParamsTestHelpers +{ + public static DruidCoordinatorRuntimeParams.Builder newBuilder() + { + return DruidCoordinatorRuntimeParams + .newBuilder() + .withStartTimeNanos(System.nanoTime()) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")); + } + + public static DruidCoordinatorRuntimeParams.Builder newBuilder(DruidCluster druidCluster) + { + return newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)); + } + + private CoordinatorRuntimeParamsTestHelpers() + { + } +} 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..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.getLazyAllSegments()).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/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java index a8faeacc8807..eb3b25b6899f 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 @@ -83,7 +83,10 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase { private DruidCoordinator coordinator; - private MetadataSegmentManager databaseSegmentManager; + private MetadataSegmentManager segmentsMetadata; + private DataSourcesSnapshot dataSourcesSnapshot; + private DruidCoordinatorRuntimeParams coordinatorRuntimeParams; + private ScheduledExecutorFactory scheduledExecutorFactory; private ConcurrentMap loadManagementPeons; private LoadQueuePeon sourceLoadQueuePeon; @@ -97,7 +100,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase private ObjectMapper objectMapper; private JacksonConfigManager configManager; private DruidNode druidNode; - private DataSourcesSnapshot dataSourcesSnapshot; private static final String SEGPATH = "/druid/segments"; private static final String SOURCE_LOAD_PATH = "/druid/loadQueue/localhost:1"; private static final String DESTINATION_LOAD_PATH = "/druid/loadQueue/localhost:2"; @@ -126,17 +128,19 @@ public CuratorDruidCoordinatorTest() @Before public void setUp() throws Exception { - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); + segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class); + dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); + coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class); + metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); configManager = EasyMock.createNiceMock(JacksonConfigManager.class); - dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); EasyMock.expect( configManager.watch( EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), EasyMock.anyObject(Class.class), EasyMock.anyObject() ) - ).andReturn(new AtomicReference(CoordinatorDynamicConfig.builder().build())).anyTimes(); + ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes(); EasyMock.expect( configManager.watch( EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), @@ -212,7 +216,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + segmentsMetadata, baseView, metadataRuleManager, curator, @@ -365,13 +369,15 @@ 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(segmentsMetadata.getImmutableDataSourceWithUsedSegments(EasyMock.anyString())) + .andReturn(druidDataSource); + EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(dataSourcesSnapshot).anyTimes(); + EasyMock.replay(segmentsMetadata, coordinatorRuntimeParams); - coordinator.setDataSourcesSnapshotForTest(dataSourcesSnapshot); EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes(); EasyMock.replay(dataSourcesSnapshot); coordinator.moveSegment( + coordinatorRuntimeParams, source.toImmutableDruidServer(), dest.toImmutableDruidServer(), sourceSegments.get(2), @@ -498,7 +504,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + segmentsMetadata, baseView, metadataRuleManager, curator, @@ -535,14 +541,7 @@ private DataSegment dataSegmentWithIntervalAndVersion(String intervalStr, String return DataSegment.builder() .dataSource("test_curator_druid_coordinator") .interval(Intervals.of(intervalStr)) - .loadSpec( - ImmutableMap.of( - "type", - "local", - "path", - "somewhere" - ) - ) + .loadSpec(ImmutableMap.of("type", "local", "path", "somewhere")) .version(version) .dimensions(ImmutableList.of()) .metrics(ImmutableList.of()) 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..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.getLazyAllSegments()).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/DruidClusterBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java new file mode 100644 index 000000000000..772b7aec1401 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.coordinator; + +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +public final class DruidClusterBuilder +{ + public static DruidClusterBuilder newBuilder() + { + return new DruidClusterBuilder(); + } + + private @Nullable Set realtimes = null; + private final Map> historicals = new HashMap<>(); + + private DruidClusterBuilder() + { + } + + public DruidClusterBuilder withRealtimes(ServerHolder... realtimes) + { + this.realtimes = new HashSet<>(Arrays.asList(realtimes)); + return this; + } + + public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals) + { + if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) { + throw new IllegalArgumentException("Duplicate tier: " + tierName); + } + return this; + } + + public DruidCluster build() + { + return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals); + } +} diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java index c07830e1320c..798c89107b2e 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java @@ -21,7 +21,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.Intervals; @@ -39,9 +38,7 @@ import java.util.Map; import java.util.NavigableSet; import java.util.Set; -import java.util.TreeSet; import java.util.stream.Collectors; -import java.util.stream.Stream; public class DruidClusterTest { @@ -100,8 +97,9 @@ public class DruidClusterTest @Before public void setup() { - cluster = new DruidCluster( - ImmutableSet.of( + cluster = DruidClusterBuilder + .newBuilder() + .withRealtimes( new ServerHolder( new ImmutableDruidServer( new DruidServerMetadata("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0), @@ -111,22 +109,20 @@ public void setup() ), new LoadQueuePeonTester() ) - ), - ImmutableMap.of( + ) + .addTier( "tier1", - Stream.of( - new ServerHolder( - new ImmutableDruidServer( - new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0), - 0L, - ImmutableMap.of("src1", dataSources.get("src1")), - 1 - ), - new LoadQueuePeonTester() - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new ImmutableDruidServer( + new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0), + 0L, + ImmutableMap.of("src1", dataSources.get("src1")), + 1 + ), + new LoadQueuePeonTester() + ) ) - ); + .build(); } @Test 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..fa0954fac0dc 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; @@ -42,10 +43,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TreeSet; -import java.util.stream.Collectors; -import java.util.stream.Stream; +/** + * TODO convert benchmarks to JMH + */ public class DruidCoordinatorBalancerProfiler { private static final int MAX_SEGMENTS_TO_MOVE = 5; @@ -83,6 +84,7 @@ public void bigProfiler() EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); @@ -116,9 +118,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(); + ImmutableDruidServerTests.expectSegments(server, segments); } else { - EasyMock.expect(server.getLazyAllSegments()).andReturn(Collections.emptyList()).anyTimes(); + ImmutableDruidServerTests.expectSegments(server, Collections.emptyList()); } EasyMock.expect(server.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(server); @@ -128,56 +130,26 @@ public void bigProfiler() serverHolderList.add(new ServerHolder(server, peon)); } - DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster( - new DruidCluster( - null, - ImmutableMap.of( - "normal", - serverHolderList.stream().collect( - Collectors.toCollection( - () -> new TreeSet<>( - DruidCoordinatorBalancerTester.percentUsedComparator - ) - ) - ) - ) - ) - ) - .withLoadManagementPeons( - peonMap - ) - .withAvailableSegmentsInTest(segments) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - MAX_SEGMENTS_TO_MOVE - ).withReplicantLifetime(500) - .withReplicationThrottleLimit(5) - .build() - ) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withEmitter(emitter) - .withDatabaseRuleManager(manager) - .withReplicationManager(new ReplicationThrottler(2, 500)) - .withSegmentReplicantLookup( - SegmentReplicantLookup.make( - new DruidCluster( - null, - ImmutableMap.of( - "normal", - serverHolderList.stream().collect( - Collectors.toCollection( - () -> new TreeSet<>( - DruidCoordinatorBalancerTester.percentUsedComparator - ) - ) - ) - ) - ) - ) - ) - .build(); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("normal", serverHolderList.toArray(new ServerHolder[0])) + .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder(druidCluster) + .withLoadManagementPeons(peonMap) + .withUsedSegmentsInTest(segments) + .withDynamicConfigs( + CoordinatorDynamicConfig + .builder() + .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE) + .withReplicantLifetime(500) + .withReplicationThrottleLimit(5) + .build() + ) + .withEmitter(emitter) + .withDatabaseRuleManager(manager) + .withReplicationManager(new ReplicationThrottler(2, 500)) + .build(); DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(coordinator); @@ -197,7 +169,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(); + ImmutableDruidServerTests.expectSegments(druidServer1, segments); EasyMock.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer1); @@ -205,7 +177,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(); + ImmutableDruidServerTests.expectSegments(druidServer2, Collections.emptyList()); EasyMock.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.replay(druidServer2); @@ -213,47 +185,28 @@ public void profileRun() EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); - DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster( - new DruidCluster( - null, - ImmutableMap.of( - "normal", - Stream.of( - new ServerHolder(druidServer1, fromPeon), - new ServerHolder(druidServer2, toPeon) - ).collect( - Collectors.toCollection( - () -> new TreeSet<>( - DruidCoordinatorBalancerTester.percentUsedComparator - ) - ) - ) - ) - ) - ) - .withLoadManagementPeons( - ImmutableMap.of( - "from", - fromPeon, - "to", - toPeon - ) - ) - .withAvailableSegmentsInTest(segments) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - MAX_SEGMENTS_TO_MOVE - ).build() - ) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster( + DruidClusterBuilder + .newBuilder() + .addTier( + "normal", + new ServerHolder(druidServer1, fromPeon), + new ServerHolder(druidServer2, toPeon) + ) + .build() + ) + .withLoadManagementPeons(ImmutableMap.of("from", fromPeon, "to", toPeon)) + .withUsedSegmentsInTest(segments) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .build(); DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); watch.start(); DruidCoordinatorRuntimeParams balanceParams = tester.run(params); 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 e094c3a3494c..585a06164e93 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 @@ -20,11 +20,11 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; 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; @@ -520,19 +520,19 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( List decommissioning ) { - return DruidCoordinatorRuntimeParams + return CoordinatorRuntimeParamsTestHelpers .newBuilder() .withDruidCluster( - new DruidCluster( - null, - ImmutableMap.of( + DruidClusterBuilder + .newBuilder() + .addTier( "normal", IntStream .range(0, druidServers.size()) .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), decommissioning.get(i))) - .collect(Collectors.toSet()) + .toArray(ServerHolder[]::new) ) - ) + .build() ) .withLoadManagementPeons( IntStream @@ -540,14 +540,9 @@ private DruidCoordinatorRuntimeParams.Builder defaultRuntimeParamsBuilder( .boxed() .collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get)) ) - .withAvailableSegmentsInTest(segments) - .withDynamicConfigs( - CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( - MAX_SEGMENTS_TO_MOVE - ).build() - ) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")); + .withUsedSegmentsInTest(segments) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build()) + .withBalancerStrategy(balancerStrategy); } private static void mockDruidServer( @@ -563,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(); + ImmutableDruidServerTests.expectSegments(druidServer, segments); EasyMock.expect(druidServer.getHost()).andReturn(name).anyTimes(); EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); if (!segments.isEmpty()) { @@ -581,6 +576,7 @@ private static void mockCoordinator(DruidCoordinator coordinator) EasyMock.anyObject(), EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.anyObject(), EasyMock.anyObject() ); EasyMock.expectLastCall().anyTimes(); diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java index fad85d0effc9..24b8de39de9a 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java @@ -25,17 +25,8 @@ import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; -import java.util.Comparator; - public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer { - public static final Comparator percentUsedComparator = (ServerHolder a, ServerHolder b) -> { - int c = Double.compare(a.getPercentUsed(), b.getPercentUsed()); - if (c == 0) { - return a.getServer().getName().compareTo(b.getServer().getName()); - } - return c; - }; public DruidCoordinatorBalancerTester(DruidCoordinator coordinator) { 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 7d38fa09b8f2..2b7975380f7e 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 @@ -20,11 +20,9 @@ package org.apache.druid.server.coordinator; import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; -import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -55,23 +53,22 @@ import java.util.HashSet; import java.util.List; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Executors; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** */ public class DruidCoordinatorRuleRunnerTest { + public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS = + CoordinatorDynamicConfig.builder().withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L).build(); + private DruidCoordinator coordinator; private LoadQueuePeon mockPeon; - private List availableSegments; + private List usedSegments; private DruidCoordinatorRuleRunner ruleRunner; private ServiceEmitter emitter; private MetadataRuleManager databaseRuleManager; - private MetadataSegmentManager databaseSegmentManager; - private DataSourcesSnapshot dataSourcesSnapshot; + private MetadataSegmentManager segmentsMetadata; @Before public void setUp() @@ -81,13 +78,12 @@ public void setUp() emitter = EasyMock.createMock(ServiceEmitter.class); EmittingLogger.registerEmitter(emitter); databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); - dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); + segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.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)), @@ -118,8 +114,6 @@ public void tearDown() * hot - 1 replicant * normal - 1 replicant * cold - 1 replicant - * - * @throws Exception */ @Test public void testRunThreeTiersOneReplicant() @@ -146,72 +140,40 @@ public void testRunThreeTiersOneReplicant() )).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( "normal", - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( "cold", - Stream.of( - new ServerHolder( - new DruidServer( - "serverCold", - "hostCold", - null, - 1000, - ServerType.HISTORICAL, - "cold", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -226,12 +188,29 @@ public void testRunThreeTiersOneReplicant() EasyMock.verify(mockPeon); } + private DruidCoordinatorRuntimeParams.Builder makeCoordinatorRuntimeParams( + DruidCluster druidCluster, + BalancerStrategy balancerStrategy + ) + { + return createCoordinatorRuntimeParams(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerStrategy(balancerStrategy); + } + + private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams(DruidCluster druidCluster) + { + return CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withDatabaseRuleManager(databaseRuleManager); + } + /** * Nodes: * hot - 2 replicants * cold - 1 replicant - * - * @throws Exception */ @Test public void testRunTwoTiersTwoReplicants() @@ -255,68 +234,35 @@ public void testRunTwoTiersTwoReplicants() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ), + new ServerHolder( + new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( "cold", - Stream.of( - new ServerHolder( - new DruidServer( - "serverCold", - "hostCold", - null, - 1000, - ServerType.HISTORICAL, - "cold", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -334,8 +280,6 @@ public void testRunTwoTiersTwoReplicants() * Nodes: * hot - 1 replicant * normal - 1 replicant - * - * @throws Exception */ @Test public void testRunTwoTiersWithExistingSegments() @@ -359,63 +303,33 @@ public void testRunTwoTiersWithExistingSegments() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer normServer = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment availableSegment : availableSegments) { - normServer.addDataSegment(availableSegment); + DruidServer normServer = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { + normServer.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), - "normal", - Stream.of( - new ServerHolder( - normServer.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .addTier("normal", new ServerHolder(normServer.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(segmentReplicantLookup) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withSegmentReplicantLookup(segmentReplicantLookup) + .withBalancerStrategy(balancerStrategy) + .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -455,42 +369,24 @@ public void testRunTwoTiersTierDoesNotExist() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withEmitter(emitter) - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy) + .withEmitter(emitter) + .build(); ruleRunner.run(params); @@ -522,35 +418,22 @@ public void testRunRuleDoesNotExist() EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).anyTimes(); EasyMock.replay(databaseRuleManager, mockPeon); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withEmitter(emitter) - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .build(); + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withEmitter(emitter) + .build(); ruleRunner.run(params); @@ -565,7 +448,7 @@ public void testDropRemove() mockEmptyPeon(); EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); EasyMock.replay(coordinator); @@ -580,47 +463,25 @@ public void testDropRemove() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "normal", - Stream.of( - new ServerHolder( - server.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("normal", new ServerHolder(server.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -651,62 +512,37 @@ public void testDropTooManyInSameTier() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server1.addDataSegment(availableSegments.get(0)); - - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server1 = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0); + server1.addDataSegment(usedSegments.get(0)); + + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) - ); + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withDynamicConfigs( + CoordinatorDynamicConfig + .builder() + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L) + .build() + ) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -740,64 +576,28 @@ public void testDropTooManyInDifferentTiers() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); - server1.addDataSegment(availableSegments.get(0)); - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0); + server1.addDataSegment(usedSegments.get(0)); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), - "normal", - Stream.of( - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -829,54 +629,27 @@ public void testDontDropInDifferentTiers() ).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment segment : availableSegments) { + DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : usedSegments) { server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of(new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) - .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), - "normal", - Stream.of(new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) - .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon)) + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) + DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -906,37 +679,13 @@ public void testDropServerActuallyServesSegment() .atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidServer server1 = new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server1.addDataSegment(availableSegments.get(0)); - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server2.addDataSegment(availableSegments.get(1)); - DruidServer server3 = new DruidServer( - "serverNorm3", - "hostNorm3", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - server3.addDataSegment(availableSegments.get(1)); - server3.addDataSegment(availableSegments.get(2)); + DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "normal", 0); + server1.addDataSegment(usedSegments.get(0)); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + server2.addDataSegment(usedSegments.get(1)); + DruidServer server3 = new DruidServer("serverNorm3", "hostNorm3", null, 1000, ServerType.HISTORICAL, "normal", 0); + server3.addDataSegment(usedSegments.get(1)); + server3.addDataSegment(usedSegments.get(2)); mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); @@ -948,33 +697,29 @@ public void testDropServerActuallyServesSegment() EasyMock.replay(anotherMockPeon); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), - new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), - new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false), + new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false), + new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false) ) - ); + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(availableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(usedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -990,15 +735,11 @@ public void testDropServerActuallyServesSegment() /** * Nodes: * hot - 2 replicants - * - * @throws Exception */ @Test public void testReplicantThrottle() { - EasyMock.expect(dataSourcesSnapshot.getOvershadowedSegments()).andReturn(ImmutableSet.of()).anyTimes(); - EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - EasyMock.replay(coordinator, databaseSegmentManager, dataSourcesSnapshot); + mockCoordinator(); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); EasyMock.expectLastCall().atLeastOnce(); mockEmptyPeon(); @@ -1006,7 +747,8 @@ public void testReplicantThrottle() EasyMock .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject())) .andReturn( - Collections.singletonList(new IntervalLoadRule( + Collections.singletonList( + new IntervalLoadRule( Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"), ImmutableMap.of("hot", 2) ) @@ -1015,53 +757,27 @@ public void testReplicantThrottle() .atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ), + new ServerHolder( + new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -1083,13 +799,13 @@ public void testReplicantThrottle() ); afterParams = ruleRunner.run( - new DruidCoordinatorRuntimeParams.Builder() + CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) .withEmitter(emitter) - .withAvailableSegmentsInTest(Collections.singletonList(overFlowSegment)) + .withUsedSegmentsInTest(overFlowSegment) .withDatabaseRuleManager(databaseRuleManager) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .build() ); @@ -1107,8 +823,6 @@ public void testReplicantThrottle() * Nodes: * hot - nothing loaded * _default_tier - 1 segment loaded - * - * @throws Exception */ @Test public void testReplicantThrottleAcrossTiers() @@ -1123,9 +837,7 @@ public void testReplicantThrottleAcrossTiers() .build() ) .atLeastOnce(); - EasyMock.expect(dataSourcesSnapshot.getOvershadowedSegments()).andReturn(ImmutableSet.of()).anyTimes(); - EasyMock.replay(dataSourcesSnapshot); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); EasyMock.replay(coordinator); mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject()); @@ -1147,56 +859,37 @@ public void testReplicantThrottleAcrossTiers() .atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer( + "serverNorm", + "hostNorm", + null, + 1000, + ServerType.HISTORICAL, + DruidServer.DEFAULT_TIER, + 0 + ).toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .build(); + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build(); DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(new ReplicationThrottler(7, 1), coordinator); DruidCoordinatorRuntimeParams afterParams = runner.run(params); @@ -1242,66 +935,40 @@ public void testDropReplicantThrottle() 1, 0 ); - List longerAvailableSegments = Lists.newArrayList(availableSegments); - longerAvailableSegments.add(overFlowSegment); - - DruidServer server1 = new DruidServer( - "serverNorm1", - "hostNorm1", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment availableSegment : longerAvailableSegments) { - server1.addDataSegment(availableSegment); + List longerUsedSegments = Lists.newArrayList(usedSegments); + longerUsedSegments.add(overFlowSegment); + + DruidServer server1 = new DruidServer("serverNorm1", "hostNorm1", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : longerUsedSegments) { + server1.addDataSegment(segment); } - DruidServer server2 = new DruidServer( - "serverNorm2", - "hostNorm2", - null, - 1000, - ServerType.HISTORICAL, - "normal", - 0 - ); - for (DataSegment availableSegment : longerAvailableSegments) { - server2.addDataSegment(availableSegment); + DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0); + for (DataSegment segment : longerUsedSegments) { + server2.addDataSegment(segment); } - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "normal", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) - ); + .build(); SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder() + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() .withDruidCluster(druidCluster) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build()) - .withAvailableSegmentsInTest(longerAvailableSegments) + .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS) + .withUsedSegmentsInTest(longerUsedSegments) .withDatabaseRuleManager(databaseRuleManager) .withSegmentReplicantLookup(segmentReplicantLookup) .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); @@ -1316,7 +983,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"), @@ -1339,11 +1006,10 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() IndexIO.CURRENT_VERSION_ID, 1 ); - availableSegments.add(v1); - availableSegments.add(v2); - EasyMock.expect(dataSourcesSnapshot.getOvershadowedSegments()).andReturn(ImmutableSet.of(v1.getId())).anyTimes(); - EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - EasyMock.replay(coordinator, dataSourcesSnapshot); + usedSegments.add(v1); + usedSegments.add(v2); + + mockCoordinator(); mockPeon.loadSegment(EasyMock.eq(v2), EasyMock.anyObject()); EasyMock.expectLastCall().once(); mockEmptyPeon(); @@ -1352,43 +1018,38 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce(); EasyMock.replay(databaseRuleManager); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer( + "serverHot", + "hostHot", + null, + 1000, + ServerType.HISTORICAL, + DruidServer.DEFAULT_TIER, + 0 + ).toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - ListeningExecutorService exec = MoreExecutors.listeningDecorator( - Executors.newFixedThreadPool(1)); - BalancerStrategy balancerStrategy = - new CostBalancerStrategyFactory().createBalancerStrategy(exec); + ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1)); + BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec); - DruidCoordinatorRuntimeParams params = - new DruidCoordinatorRuntimeParams.Builder() - .withDruidCluster(druidCluster) - .withAvailableSegmentsInTest(availableSegments) - .withDatabaseRuleManager(databaseRuleManager) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withBalancerStrategy(balancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) - .withDataSourcesSnapshot(dataSourcesSnapshot) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withUsedSegmentsInTest(usedSegments) + .withDatabaseRuleManager(databaseRuleManager) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withBalancerStrategy(balancerStrategy) + .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build()) + .build(); DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params); CoordinatorStats stats = afterParams.getCoordinatorStats(); @@ -1398,9 +1059,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(); @@ -1409,9 +1070,9 @@ public void testRulesRunOnNonOvershadowedSegmentsOnly() private void mockCoordinator() { EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes(); - coordinator.removeSegment(EasyMock.anyObject()); + coordinator.markSegmentAsUnused(EasyMock.anyObject()); EasyMock.expectLastCall().anyTimes(); - EasyMock.replay(coordinator); + EasyMock.replay(coordinator, segmentsMetadata); } private void mockEmptyPeon() @@ -1426,7 +1087,7 @@ private void mockEmptyPeon() private CoordinatorDynamicConfig createCoordinatorDynamicConfig() { return CoordinatorDynamicConfig.builder() - .withMillisToWaitBeforeDeleting(0) + .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0) .withMergeBytesLimit(0) .withMergeSegmentsLimit(0) .withMaxSegmentsToMove(0) @@ -1434,8 +1095,8 @@ private CoordinatorDynamicConfig createCoordinatorDynamicConfig() .withReplicationThrottleLimit(24) .withBalancerComputeThreads(0) .withEmitBalancingStats(false) - .withKillDataSourceWhitelist(null) - .withKillAllDataSources(false) + .withSpecificDataSourcesToKillUnusedSegmentsIn(null) + .withKillUnusedSegmentsInAllDataSources(false) .withMaxSegmentsInNodeLoadingQueue(1000) .build(); } 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 3df7fc655bc5..b587842d797d 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; @@ -86,7 +87,10 @@ public class DruidCoordinatorTest extends CuratorTestBase private static final long COORDINATOR_PERIOD = 100; private DruidCoordinator coordinator; - private MetadataSegmentManager databaseSegmentManager; + private MetadataSegmentManager segmentsMetadata; + private DataSourcesSnapshot dataSourcesSnapshot; + private DruidCoordinatorRuntimeParams coordinatorRuntimeParams; + private SingleServerInventoryView serverInventoryView; private ScheduledExecutorFactory scheduledExecutorFactory; private DruidServer druidServer; @@ -100,20 +104,17 @@ public class DruidCoordinatorTest extends CuratorTestBase private ObjectMapper objectMapper; private DruidNode druidNode; private LatchableServiceEmitter serviceEmitter = new LatchableServiceEmitter(); - private DataSourcesSnapshot dataSourcesSnapshot; @Before public void setUp() throws Exception { druidServer = EasyMock.createMock(DruidServer.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); - databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); + segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class); dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); + coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.class); - EasyMock.expect(databaseSegmentManager.getDataSourcesSnapshot()).andReturn(dataSourcesSnapshot).anyTimes(); - EasyMock.expect(databaseSegmentManager.isStarted()).andReturn(true).anyTimes(); - EasyMock.replay(databaseSegmentManager); EasyMock.expect( configManager.watch( EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), @@ -185,7 +186,7 @@ public String getBase() } }, configManager, - databaseSegmentManager, + segmentsMetadata, serverInventoryView, metadataRuleManager, curator, @@ -251,7 +252,10 @@ public void testMoveSegment() ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); EasyMock.replay(druidDataSource); - coordinator.setDataSourcesSnapshotForTest(dataSourcesSnapshot); + EasyMock + .expect(segmentsMetadata.getImmutableDataSourceWithUsedSegments(EasyMock.anyString())) + .andReturn(druidDataSource); + EasyMock.replay(segmentsMetadata); EasyMock.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes(); EasyMock.replay(dataSourcesSnapshot); scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); @@ -288,7 +292,10 @@ public void testMoveSegment() EasyMock.expect(serverInventoryView.isSegmentLoadedByServer("to", segment)).andReturn(true).once(); EasyMock.replay(serverInventoryView); + mockCoordinatorRuntimeParams(); + coordinator.moveSegment( + coordinatorRuntimeParams, druidServer.toImmutableDruidServer(), druidServer2.toImmutableDruidServer(), segment, @@ -301,11 +308,14 @@ public void testMoveSegment() LoadPeonCallback dropCallback = dropCallbackCapture.getValue(); dropCallback.execute(); - EasyMock.verify(druidServer); - EasyMock.verify(druidServer2); - EasyMock.verify(loadQueuePeon); - EasyMock.verify(serverInventoryView); - EasyMock.verify(metadataRuleManager); + EasyMock.verify(druidServer, druidServer2, loadQueuePeon, serverInventoryView, metadataRuleManager); + EasyMock.verify(coordinatorRuntimeParams); + } + + private void mockCoordinatorRuntimeParams() + { + EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(this.dataSourcesSnapshot).anyTimes(); + EasyMock.replay(coordinatorRuntimeParams); } @Test(timeout = 60_000L) @@ -395,9 +405,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(); @@ -531,21 +542,42 @@ public void testCoordinatorTieredRun() throws Exception leaderUnannouncerLatch.await(); EasyMock.verify(serverInventoryView); - EasyMock.verify(databaseSegmentManager); + EasyMock.verify(segmentsMetadata); EasyMock.verify(metadataRuleManager); } private void setupMetadataSegmentManagerMock(DruidDataSource dataSource) { + EasyMock.expect(segmentsMetadata.isPollingDatabasePeriodically()).andReturn(true).anyTimes(); EasyMock - .expect(dataSourcesSnapshot.iterateAllSegmentsInSnapshot()) + .expect(segmentsMetadata.iterateAllUsedSegments()) .andReturn(dataSource.getSegments()) .anyTimes(); EasyMock - .expect(dataSourcesSnapshot.getDataSources()) + .expect(segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments()) .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) .anyTimes(); - EasyMock.replay(dataSourcesSnapshot); + DataSourcesSnapshot dataSourcesSnapshot = + new DataSourcesSnapshot(ImmutableMap.of(dataSource.getName(), dataSource.toImmutableDruidDataSource())); + EasyMock + .expect(segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments()) + .andReturn(dataSourcesSnapshot) + .anyTimes(); + EasyMock + .expect(segmentsMetadata.retrieveAllDataSourceNames()) + .andReturn(Collections.singleton(dataSource.getName())) + .anyTimes(); + EasyMock.replay(segmentsMetadata); + + EasyMock + .expect(this.dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) + .andReturn(dataSource.getSegments()) + .anyTimes(); + EasyMock + .expect(this.dataSourcesSnapshot.getDataSourcesWithAllUsedSegments()) + .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) + .anyTimes(); + EasyMock.replay(this.dataSourcesSnapshot); } @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..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.getLazyAllSegments()).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.getLazyAllSegments()).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.getLazyAllSegments()).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.getLazyAllSegments()).andReturn(segments4).anyTimes(); + ImmutableDruidServerTests.expectSegments(druidServer4, segments4); 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..f450bf3b051c 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 @@ -20,15 +20,17 @@ package org.apache.druid.server.coordinator.helper; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinator; +import org.apache.druid.server.coordinator.DruidCoordinatorRuleRunnerTest; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.ServerHolder; @@ -38,17 +40,13 @@ import org.joda.time.Interval; import org.junit.Test; -import java.util.Collections; import java.util.List; -import java.util.TreeSet; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class DruidCoordinatorCleanupOvershadowedTest { - DruidCoordinatorCleanupOvershadowed druidCoordinatorCleanupOvershadowed; + DruidCoordinatorCleanupOvershadowed druidCoordinatorMarkAsUnusedOvershadowedSegments; 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); @@ -70,8 +68,8 @@ public class DruidCoordinatorCleanupOvershadowedTest @Test public void testRun() { - druidCoordinatorCleanupOvershadowed = new DruidCoordinatorCleanupOvershadowed(coordinator); - availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); + druidCoordinatorMarkAsUnusedOvershadowedSegments = new DruidCoordinatorCleanupOvershadowed(coordinator); + usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); // Dummy values for comparisons in TreeSet EasyMock.expect(mockPeon.getLoadQueueSize()) @@ -103,27 +101,26 @@ public void testRun() .andReturn(ImmutableSet.of(segmentV1, segmentV2)) .anyTimes(); EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); - coordinator.removeSegment(segmentV1); - coordinator.removeSegment(segmentV0); + coordinator.markSegmentAsUnused(segmentV1); + coordinator.markSegmentAsUnused(segmentV0); EasyMock.expectLastCall(); EasyMock.replay(mockPeon, coordinator, druidServer, druidDataSource); - druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "normal", - Stream.of( - new ServerHolder(druidServer, mockPeon) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - )); + druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("normal", new ServerHolder(druidServer, mockPeon)) + .build(); - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() - .withAvailableSegmentsInTest(availableSegments) + .withUsedSegmentsInTest(usedSegments) .withCoordinatorStats(new CoordinatorStats()) .withDruidCluster(druidCluster) + .withDynamicConfigs( + DruidCoordinatorRuleRunnerTest.COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS + ) .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/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java index bdf711fa6153..0eb8b39ccc0b 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 @@ -21,7 +21,9 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import org.apache.druid.client.DataSourcesSnapshot; import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient; @@ -29,6 +31,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; @@ -48,7 +51,6 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.Supplier; @@ -123,34 +125,19 @@ public int getTotalWorkerCapacity() @Before public void setup() { - dataSources = new HashMap<>(); + List segments = new ArrayList<>(); for (int i = 0; i < 3; i++) { final String dataSource = DATA_SOURCE_PREFIX + i; - - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - - for (int j = 0; j < 4; j++) { - for (int k = 0; k < 2; k++) { - DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - } - } - - for (int j = 7; j < 9; j++) { + for (int j : new int[] {0, 1, 2, 3, 7, 8}) { for (int k = 0; k < 2; k++) { - DataSegment segment = createSegment(dataSource, j, true, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); - segment = createSegment(dataSource, j, false, k); - timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); + segments.add(createSegment(dataSource, j, true, k)); + segments.add(createSegment(dataSource, j, false, k)); } } - - dataSources.put(dataSource, timeline); } + dataSources = DataSourcesSnapshot + .fromUsedSegments(segments, ImmutableMap.of()) + .getUsedSegmentsTimelinesPerDataSource(); } private static DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition) @@ -261,9 +248,9 @@ public String get() private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor) { - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers .newBuilder() - .withDataSources(dataSources) + .withUsedSegmentsTimelinesPerDataSourceInTest(dataSources) .withCompactionConfig(CoordinatorCompactionConfig.from(createCompactionConfigs())) .build(); return compactor.run(params).getCoordinatorStats(); @@ -291,10 +278,7 @@ private void assertCompactSegments( long numDataSourceOfExpectedRemainingSegments = stats .getDataSources(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT) .stream() - .mapToLong(dataSource -> stats.getDataSourceStat( - DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT, - dataSource) - ) + .mapToLong(ds -> stats.getDataSourceStat(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT, ds)) .filter(stat -> stat == expectedRemainingSegments) .count(); Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments); 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 0f0bc033cb93..6882c71174cc 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 @@ -25,6 +25,7 @@ import org.apache.druid.metadata.MetadataSegmentManager; 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; @@ -37,34 +38,34 @@ public class DruidCoordinatorSegmentKillerTest { @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 +74,7 @@ public void testFindIntervalForKillTask() Intervals.of("2014/2020") ); - testFindIntervalForKillTask( + testFindIntervalForKill( ImmutableList.of( Intervals.of("2015/2019"), Intervals.of("2014/2016"), @@ -84,23 +85,21 @@ public void testFindIntervalForKillTask() ); } - private void testFindIntervalForKillTask(List segmentManagerResult, Interval expected) + private void testFindIntervalForKill(List segmentIntervals, Interval expected) { - MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect( - segmentManager.getUnusedSegmentIntervals( + segmentsMetadata.getUnusedSegmentIntervals( EasyMock.anyString(), - EasyMock.anyObject(Interval.class), + EasyMock.anyObject(DateTime.class), EasyMock.anyInt() ) - ).andReturn( - segmentManagerResult - ); - EasyMock.replay(segmentManager); + ).andReturn(segmentIntervals); + EasyMock.replay(segmentsMetadata); IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class); DruidCoordinatorSegmentKiller coordinatorSegmentKiller = new DruidCoordinatorSegmentKiller( - segmentManager, + segmentsMetadata, indexingServiceClient, new TestDruidCoordinatorConfig( null, @@ -117,7 +116,7 @@ private void testFindIntervalForKillTask(List segmentManagerResult, In Assert.assertEquals( expected, - coordinatorSegmentKiller.findIntervalForKillTask("test", 10000) + coordinatorSegmentKiller.findIntervalForKill("test", 10000) ); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java index 9c5b9b4e2759..b28bd7faf98f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java @@ -625,10 +625,7 @@ private static VersionedIntervalTimeline createTimeline( SegmentGenerateSpec... specs ) { - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline<>( - String.CASE_INSENSITIVE_ORDER - ); - + List segments = new ArrayList<>(); final String version = DateTimes.nowUtc().toString(); final List orderedSpecs = Arrays.asList(specs); @@ -659,18 +656,14 @@ private static VersionedIntervalTimeline createTimeline( 0, spec.segmentSize ); - timeline.add( - segmentInterval, - version, - shardSpec.createChunk(segment) - ); + segments.add(segment); } remaininInterval = SegmentCompactorUtil.removeIntervalFromEnd(remaininInterval, segmentInterval); } } - return timeline; + return VersionedIntervalTimeline.forSegments(segments); } private DataSourceCompactionConfig createCompactionConfig( 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 431f6de19858..cdb5e416da82 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 @@ -20,13 +20,14 @@ package org.apache.druid.server.coordinator.rules; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import org.apache.druid.client.DruidServer; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeonTester; import org.apache.druid.server.coordinator.SegmentReplicantLookup; @@ -38,12 +39,8 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.TreeSet; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class BroadcastDistributionRuleTest { @@ -241,56 +238,50 @@ public void setUp() true ); - druidCluster = new DruidCluster( - null, - ImmutableMap.of( + druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - holdersOfLargeSegments.get(0), - holderOfSmallSegment, - holdersOfLargeSegments2.get(0) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + holdersOfLargeSegments.get(0), + holderOfSmallSegment, + holdersOfLargeSegments2.get(0) + ) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - holdersOfLargeSegments.get(1), - holdersOfLargeSegments.get(2), - holdersOfLargeSegments2.get(1) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + holdersOfLargeSegments.get(1), + holdersOfLargeSegments.get(2), + holdersOfLargeSegments2.get(1) ) - ); + .build(); - secondCluster = new DruidCluster( - null, - ImmutableMap.of( + secondCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Stream.of( - activeServer, - decommissioningServer1, - decommissioningServer2 - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + activeServer, + decommissioningServer1, + decommissioningServer2 ) - ); + .build(); } @Test public void testBroadcastToSingleDataSource() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + final ForeverBroadcastDistributionRule rule = + new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1), - largeSegments.get(2), - largeSegments2.get(0), - largeSegments2.get(1) - ).build(), + makeCoordinartorRuntimeParams( + druidCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1), + largeSegments.get(2), + largeSegments2.get(0), + largeSegments2.get(1) + ), smallSegment ); @@ -310,6 +301,19 @@ public void testBroadcastToSingleDataSource() Assert.assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); } + private static DruidCoordinatorRuntimeParams makeCoordinartorRuntimeParams( + DruidCluster druidCluster, + DataSegment... usedSegments + ) + { + return CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withUsedSegmentsInTest(usedSegments) + .build(); + } + /** * Servers: * name | segments @@ -326,19 +330,17 @@ public void testBroadcastToSingleDataSource() @Test public void testBroadcastDecommissioning() { - final ForeverBroadcastDistributionRule rule = new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); + final ForeverBroadcastDistributionRule rule = + new ForeverBroadcastDistributionRule(ImmutableList.of("large_source")); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(secondCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(secondCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1) - ).build(), + makeCoordinartorRuntimeParams( + secondCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1) + ), smallSegment ); @@ -359,18 +361,15 @@ public void testBroadcastToMultipleDataSources() CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1), - largeSegments.get(2), - largeSegments2.get(0), - largeSegments2.get(1) - ).build(), + makeCoordinartorRuntimeParams( + druidCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1), + largeSegments.get(2), + largeSegments2.get(0), + largeSegments2.get(1) + ), smallSegment ); @@ -397,18 +396,15 @@ public void testBroadcastToAllServers() CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest( - smallSegment, - largeSegments.get(0), - largeSegments.get(1), - largeSegments.get(2), - largeSegments2.get(0), - largeSegments2.get(1) - ).build(), + makeCoordinartorRuntimeParams( + druidCluster, + smallSegment, + largeSegments.get(0), + largeSegments.get(1), + largeSegments.get(2), + largeSegments2.get(0), + largeSegments2.get(1) + ), smallSegment ); 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..b624cf30dd53 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 @@ -26,6 +26,7 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.DruidServer; +import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -36,9 +37,11 @@ import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordinator.BalancerStrategy; import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; +import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers; import org.apache.druid.server.coordinator.CoordinatorStats; import org.apache.druid.server.coordinator.CostBalancerStrategyFactory; import org.apache.druid.server.coordinator.DruidCluster; +import org.apache.druid.server.coordinator.DruidClusterBuilder; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.LoadQueuePeonTester; @@ -56,18 +59,13 @@ import org.junit.Test; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import java.util.stream.Stream; /** */ @@ -138,53 +136,34 @@ public void testLoad() EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1) + .toImmutableDruidServer(), + mockPeon + ) + ) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - new DruidServer( - "serverNorm", - "hostNorm", - null, - 1000, - ServerType.HISTORICAL, - DruidServer.DEFAULT_TIER, - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer( + "serverNorm", + "hostNorm", + null, + 1000, + ServerType.HISTORICAL, + DruidServer.DEFAULT_TIER, + 0 + ).toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER)); @@ -192,6 +171,21 @@ public void testLoad() EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); } + private DruidCoordinatorRuntimeParams makeCoordinatorRuntimeParams( + DruidCluster druidCluster, + DataSegment... usedSegments + ) + { + return CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(usedSegments) + .build(); + } + @Test public void testLoadPrimaryAssignDoesNotOverAssign() { @@ -213,47 +207,18 @@ public void testLoadPrimaryAssignDoesNotOverAssign() EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ), new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + ImmutableDruidServer server1 = + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); + ImmutableDruidServer server2 = + new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer(); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1, mockPeon), new ServerHolder(server2, mockPeon)) + .build(); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + makeCoordinatorRuntimeParams(druidCluster, segment), segment ); @@ -264,50 +229,17 @@ public void testLoadPrimaryAssignDoesNotOverAssign() final LoadQueuePeon loadingPeon = createLoadingPeon(ImmutableList.of(segment)); EasyMock.replay(loadingPeon); - DruidCluster afterLoad = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - loadingPeon - ), new ServerHolder( - new DruidServer( - "serverHot2", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 1 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) - ) - ); + DruidCluster afterLoad = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1, loadingPeon), new ServerHolder(server2, mockPeon)) + .build(); + CoordinatorStats statsAfterLoadPrimary = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(afterLoad) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(afterLoad)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + makeCoordinatorRuntimeParams(afterLoad, segment), segment ); - Assert.assertEquals(0, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); @@ -330,72 +262,36 @@ public void testLoadPriority() EasyMock.replay(throttler, mockPeon1, mockPeon2, mockBalancerStrategy); - final LoadRule rule = createLoadRule(ImmutableMap.of( - "tier1", 10, - "tier2", 10 - )); + final LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 10, "tier2", 10)); - final DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + final DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Stream.of( - new ServerHolder( - new DruidServer( - "server1", - "host1", - null, - 1000, - ServerType.HISTORICAL, - "tier1", - 0 - ).toImmutableDruidServer(), - mockPeon1 - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + new ServerHolder( + new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "tier1", 0) + .toImmutableDruidServer(), + mockPeon1 + ) + ) + .addTier( "tier2", - Stream.of( - new ServerHolder( - new DruidServer( - "server2", - "host2", - null, - 1000, - ServerType.HISTORICAL, - "tier2", - 1 - ).toImmutableDruidServer(), - mockPeon2 - ), - new ServerHolder( - new DruidServer( - "server3", - "host3", - null, - 1000, - ServerType.HISTORICAL, - "tier2", - 1 - ).toImmutableDruidServer(), - mockPeon2 - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("server2", "host2", null, 1000, ServerType.HISTORICAL, "tier2", 1) + .toImmutableDruidServer(), + mockPeon2 + ), + new ServerHolder( + new DruidServer("server3", "host3", null, 1000, ServerType.HISTORICAL, "tier2", 1) + .toImmutableDruidServer(), + mockPeon2 + ) ) - ); + .build(); final DataSegment segment = createDataSegment("foo"); - final CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + final CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(0L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); @@ -421,15 +317,7 @@ public void testDrop() final DataSegment segment = createDataSegment("foo"); - DruidServer server1 = new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); + DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0); server1.addDataSegment(segment); DruidServer server2 = new DruidServer( "serverNorm", @@ -450,41 +338,17 @@ public void testDrop() DruidServer.DEFAULT_TIER, 0 ); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "hot", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon)) + .addTier( DruidServer.DEFAULT_TIER, - Stream.of( - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server3.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server2.toImmutableDruidServer(), mockPeon), + new ServerHolder(server3.toImmutableDruidServer(), mockPeon) ) - ); + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", DruidServer.DEFAULT_TIER)); @@ -505,43 +369,32 @@ public void testLoadWithNonExistentTier() EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of( - "nonExistentTier", 1, - "hot", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1)); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + mockPeon + ) ) - ); + .build(); final DataSegment segment = createDataSegment("foo"); CoordinatorStats stats = rule.run( null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), + CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(segment) + .build(), segment ); @@ -561,62 +414,25 @@ public void testDropWithNonExistentTier() .times(2); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); - LoadRule rule = createLoadRule(ImmutableMap.of( - "nonExistentTier", 1, - "hot", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1)); final DataSegment segment = createDataSegment("foo"); - DruidServer server1 = new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); - DruidServer server2 = new DruidServer( - "serverHo2t", - "hostHot2", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ); + DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0); + DruidServer server2 = new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0); server1.addDataSegment(segment); server2.addDataSegment(segment); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - server1.toImmutableDruidServer(), - mockPeon - ), - new ServerHolder( - server2.toImmutableDruidServer(), - mockPeon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon) ) - ); + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); @@ -634,46 +450,33 @@ public void testMaxLoadingQueueSize() final LoadQueuePeonTester peon = new LoadQueuePeonTester(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "hot", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("hot", 1)); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "hot", - Stream.of( - new ServerHolder( - new DruidServer( - "serverHot", - "hostHot", - null, - 1000, - ServerType.HISTORICAL, - "hot", - 0 - ).toImmutableDruidServer(), - peon - ) - ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) + new ServerHolder( + new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0) + .toImmutableDruidServer(), + peon + ) ) - ); + .build(); DataSegment dataSegment1 = createDataSegment("ds1"); DataSegment dataSegment2 = createDataSegment("ds2"); DataSegment dataSegment3 = createDataSegment("ds3"); - DruidCoordinatorRuntimeParams params = - DruidCoordinatorRuntimeParams - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) - .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) - .build(); + DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers + .newBuilder() + .withDruidCluster(druidCluster) + .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) + .withReplicationManager(throttler) + .withBalancerStrategy(mockBalancerStrategy) + .withUsedSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) + .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build()) + .build(); CoordinatorStats stats1 = rule.run(null, params, dataSegment1); CoordinatorStats stats2 = rule.run(null, params, dataSegment2); @@ -696,10 +499,7 @@ public void testLoadDecommissioning() final LoadQueuePeon mockPeon1 = createEmptyPeon(); final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); - LoadRule rule = createLoadRule(ImmutableMap.of( - "tier1", 1, - "tier2", 1 - )); + LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 1, "tier2", 1)); final DataSegment segment = createDataSegment("foo"); @@ -710,27 +510,13 @@ public void testLoadDecommissioning() EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( - "tier1", - Collections.singleton(createServerHolder("tier1", mockPeon1, true)), - "tier2", - Collections.singleton(createServerHolder("tier2", mockPeon2, false)) - ) - ); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("tier1", createServerHolder("tier1", mockPeon1, true)) + .addTier("tier2", createServerHolder("tier2", mockPeon2, false)) + .build(); - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy); @@ -771,23 +557,13 @@ public void testLoadReplicaDuringDecommissioning() EasyMock.replay(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier("tier1", holder1, holder2) + .addTier("tier2", holder3, holder4) + .build(); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of("tier1", Arrays.asList(holder1, holder2), "tier2", Arrays.asList(holder3, holder4)) - ); - - CoordinatorStats stats = rule.run( - null, - DruidCoordinatorRuntimeParams.newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment).build(), - segment - ); + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); @@ -820,40 +596,21 @@ public void testDropDuringDecommissioning() DruidServer server2 = createServer("tier1"); server2.addDataSegment(segment2); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Arrays.asList( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true), - new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false) - ) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false) ) - ); - - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment1, segment2) .build(); - CoordinatorStats stats = rule.run( - null, - params, - segment1 - ); + + DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, segment1, segment2); + CoordinatorStats stats = rule.run(null, params, segment1); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); - stats = rule.run( - null, - params, - segment2 - ); + stats = rule.run(null, params, segment2); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); - EasyMock.verify(throttler, mockPeon); } @@ -885,32 +642,17 @@ public void testRedundantReplicaDropDuringDecommissioning() DruidServer server3 = createServer("tier1"); server3.addDataSegment(segment1); - DruidCluster druidCluster = new DruidCluster( - null, - ImmutableMap.of( + DruidCluster druidCluster = DruidClusterBuilder + .newBuilder() + .addTier( "tier1", - Arrays.asList( - new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false), - new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true), - new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false) - ) + new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false), + new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true), + new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false) ) - ); - - DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams - .newBuilder() - .withDruidCluster(druidCluster) - .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) - .withReplicationManager(throttler) - .withBalancerStrategy(mockBalancerStrategy) - .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) - .withAvailableSegmentsInTest(segment1) .build(); - CoordinatorStats stats = rule.run( - null, - params, - segment1 - ); + + CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment1), segment1); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size()); Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size()); 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..f678364b38ba 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() - .withKillDataSourceWhitelist(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 expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, long expectedMergeBytesLimit, int expectedMergeSegmentsLimit, int expectedMaxSegmentsToMove, @@ -254,14 +255,17 @@ private void assertConfig( int expectedReplicationThrottleLimit, int expectedBalancerComputeThreads, boolean expectedEmitingBalancingStats, - Set expectedKillableDatasources, - boolean expectedKillAllDataSources, + Set expectedSpecificDataSourcesToKillUnusedSegmentsIn, + boolean expectedKillUnusedSegmentsInAllDataSources, int expectedMaxSegmentsInNodeLoadingQueue, - Set decommissioning, + Set decommissioningNodes, int decommissioningMaxPercentOfMaxSegmentsToMove ) { - Assert.assertEquals(expectedMillisToWaitBeforeDeleting, config.getMillisToWaitBeforeDeleting()); + Assert.assertEquals( + expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments, + config.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments() + ); Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit()); Assert.assertEquals(expectedMaxSegmentsToMove, config.getMaxSegmentsToMove()); @@ -269,10 +273,16 @@ 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(expectedKillAllDataSources, config.isKillAllDataSources()); + Assert.assertEquals( + expectedSpecificDataSourcesToKillUnusedSegmentsIn, + config.getSpecificDataSourcesToKillUnusedSegmentsIn() + ); + Assert.assertEquals(expectedKillUnusedSegmentsInAllDataSources, config.isKillUnusedSegmentsInAllDataSources()); Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue()); - Assert.assertEquals(decommissioning, config.getDecommissioningNodes()); - Assert.assertEquals(decommissioningMaxPercentOfMaxSegmentsToMove, config.getDecommissioningMaxPercentOfMaxSegmentsToMove()); + Assert.assertEquals(decommissioningNodes, config.getDecommissioningNodes()); + 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 1ae294bd1957..cb463fd51759 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 @@ -20,7 +20,9 @@ package org.apache.druid.server.http; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import com.google.common.collect.Sets; import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.DruidDataSource; @@ -32,6 +34,7 @@ import org.apache.druid.java.util.common.Intervals; import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataSegmentManager; +import org.apache.druid.metadata.UnknownSegmentIdException; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.TableDataSource; import org.apache.druid.server.coordination.DruidServerMetadata; @@ -170,14 +173,8 @@ public void testGetFullQueryableDataSources() EasyMock.expectLastCall().times(1); EasyMock.replay(inventoryView, server, request); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - null, - new AuthConfig(), - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER); Response response = dataSourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); Assert.assertEquals(200, response.getStatus()); @@ -213,9 +210,7 @@ public void testSecuredGetFullQueryableDataSources() request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server) - ).once(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); // second request EasyMock.expect(server.getDataSources()).andReturn( @@ -230,9 +225,7 @@ public void testSecuredGetFullQueryableDataSources() request.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); EasyMock.expectLastCall().times(1); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server) - ).once(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.replay(inventoryView, server, request); AuthorizerMapper authMapper = new AuthorizerMapper(null) { @@ -255,25 +248,13 @@ public Access authorize(AuthenticationResult authenticationResult1, Resource res } }; - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - null, - new AuthConfig(), - authMapper - ); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, authMapper); Response response = dataSourcesResource.getQueryableDataSources("full", null, request); Set result = (Set) response.getEntity(); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(1, result.size()); - Assert.assertEquals( - listDataSources.subList(0, 1).stream() - .map(DruidDataSource::toImmutableDruidDataSource) - .collect(Collectors.toSet()), - new HashSet<>(result) - ); + Assert.assertEquals(listDataSources.get(0).toImmutableDruidDataSource(), Iterables.getOnlyElement(result)); response = dataSourcesResource.getQueryableDataSources(null, null, request); List result1 = (List) response.getEntity(); @@ -302,14 +283,8 @@ public void testGetSimpleQueryableDataSources() EasyMock.expectLastCall().times(1); EasyMock.replay(inventoryView, server, request); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - null, - new AuthConfig(), - AuthTestUtils.TEST_AUTHORIZER_MAPPER - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, null, AuthTestUtils.TEST_AUTHORIZER_MAPPER); Response response = dataSourcesResource.getQueryableDataSources(null, "simple", request); Assert.assertEquals(200, response.getStatus()); List> results = (List>) response.getEntity(); @@ -333,8 +308,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); @@ -349,8 +324,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); } @@ -367,8 +342,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")); @@ -386,31 +361,22 @@ public void testSimpleGetTheDataSource() @Test public void testSimpleGetTheDataSourceManyTiers() { - EasyMock.expect(server.getDataSource("datasource1")).andReturn( - listDataSources.get(0) - ).atLeastOnce(); + EasyMock.expect(server.getDataSource("datasource1")).andReturn(listDataSources.get(0)).atLeastOnce(); EasyMock.expect(server.getTier()).andReturn("cold").atLeastOnce(); DruidServer server2 = EasyMock.createStrictMock(DruidServer.class); - EasyMock.expect(server2.getDataSource("datasource1")).andReturn( - listDataSources.get(1) - ).atLeastOnce(); + EasyMock.expect(server2.getDataSource("datasource1")).andReturn(listDataSources.get(1)).atLeastOnce(); EasyMock.expect(server2.getTier()).andReturn("hot").atLeastOnce(); DruidServer server3 = EasyMock.createStrictMock(DruidServer.class); - EasyMock.expect(server3.getDataSource("datasource1")).andReturn( - listDataSources.get(1) - ).atLeastOnce(); + EasyMock.expect(server3.getDataSource("datasource1")).andReturn(listDataSources.get(1)).atLeastOnce(); EasyMock.expect(server3.getTier()).andReturn("cold").atLeastOnce(); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server, server2, server3) - ).atLeastOnce(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server, server2, server3)).atLeastOnce(); EasyMock.replay(inventoryView, server, server2, server3); - DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = dataSourcesResource.getTheDataSource("datasource1", null); + DataSourcesResource dataSourcesResource = 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")); @@ -447,9 +413,8 @@ public void testSimpleGetTheDataSourceWithReplicatedSegments() EasyMock.replay(inventoryView); - DataSourcesResource dataSourcesResource = - new DataSourcesResource(inventoryView, null, null, null, new AuthConfig(), null); - Response response = dataSourcesResource.getTheDataSource("datasource1", null); + DataSourcesResource dataSourcesResource = new DataSourcesResource(inventoryView, null, null, null, null); + Response response = dataSourcesResource.getDataSource("datasource1", null); Assert.assertEquals(200, response.getStatus()); Map> result1 = (Map>) response.getEntity(); Assert.assertEquals(2, ((Map) (result1.get("tiers").get("tier1"))).get("segmentCount")); @@ -464,7 +429,7 @@ public void testSimpleGetTheDataSourceWithReplicatedSegments() Assert.assertEquals(30L, result1.get("segments").get("size")); Assert.assertEquals(60L, result1.get("segments").get("replicatedSize")); - response = dataSourcesResource.getTheDataSource("datasource2", null); + response = dataSourcesResource.getDataSource("datasource2", null); Assert.assertEquals(200, response.getStatus()); Map> result2 = (Map>) response.getEntity(); Assert.assertEquals(1, ((Map) (result2.get("tiers").get("tier1"))).get("segmentCount")); @@ -486,27 +451,37 @@ public void testGetSegmentDataSourceIntervals() server.addDataSegment(dataSegmentList.get(0)); server.addDataSegment(dataSegmentList.get(1)); server.addDataSegment(dataSegmentList.get(2)); - EasyMock.expect(inventoryView.getInventory()).andReturn( - ImmutableList.of(server) - ).atLeastOnce(); + EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).atLeastOnce(); EasyMock.replay(inventoryView); List expectedIntervals = new ArrayList<>(); 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()); @@ -514,7 +489,11 @@ 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()) { @@ -529,7 +508,7 @@ public void testGetSegmentDataSourceIntervals() } @Test - public void testGetSegmentDataSourceSpecificInterval() + public void testGetServedSegmentsInIntervalInDataSource() { server = new DruidServer("who", "host", null, 1234, ServerType.HISTORICAL, "tier1", 0); server.addDataSegment(dataSegmentList.get(0)); @@ -539,8 +518,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, @@ -548,7 +527,7 @@ public void testGetSegmentDataSourceSpecificInterval() ); Assert.assertEquals(null, response.getEntity()); - response = dataSourcesResource.getSegmentDataSourceSpecificInterval( + response = dataSourcesResource.getServedSegmentsInInterval( "datasource1", "2010-03-01/P1D", null, @@ -556,16 +535,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", @@ -583,7 +562,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()) { @@ -598,7 +577,7 @@ public void testGetSegmentDataSourceSpecificInterval() } @Test - public void testDeleteDataSourceSpecificInterval() + public void testKillSegmentsInIntervalInDataSource() { String interval = "2010-01-01_P1D"; Interval theInterval = Intervals.of(interval.replace('_', '/')); @@ -608,15 +587,9 @@ public void testDeleteDataSourceSpecificInterval() EasyMock.expectLastCall().once(); EasyMock.replay(indexingServiceClient, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - indexingServiceClient, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.deleteDataSourceSpecificInterval("datasource1", interval); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null); + Response response = dataSourcesResource.killSegmentsInInterval("datasource1", interval); Assert.assertEquals(200, response.getStatus()); Assert.assertEquals(null, response.getEntity()); @@ -624,22 +597,23 @@ public void testDeleteDataSourceSpecificInterval() } @Test - public void testDeleteDataSource() + public void testMarkAsUnusedAllSegmentsInDataSource() { IndexingServiceClient indexingServiceClient = EasyMock.createStrictMock(IndexingServiceClient.class); EasyMock.replay(indexingServiceClient, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - null, - null, - indexingServiceClient, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.deleteDataSource("datasource", "true", "???"); - Assert.assertEquals(400, response.getStatus()); - Assert.assertNotNull(response.getEntity()); - Assert.assertTrue(response.getEntity().toString().contains("java.lang.IllegalArgumentException")); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, null, indexingServiceClient, null); + try { + Response response = + dataSourcesResource.markAsUnusedAllSegmentsOrKillSegmentsInInterval("datasource", "true", "???"); + // 400 (Bad Request) or an IllegalArgumentException is expected. + Assert.assertEquals(400, response.getStatus()); + Assert.assertNotNull(response.getEntity()); + Assert.assertTrue(response.getEntity().toString().contains("java.lang.IllegalArgumentException")); + } + catch (IllegalArgumentException ignore) { + // expected + } EasyMock.verify(indexingServiceClient, server); } @@ -650,14 +624,8 @@ public void testIsHandOffComplete() MetadataRuleManager databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class); Rule loadRule = new IntervalLoadRule(Intervals.of("2013-01-02T00:00:00Z/2013-01-03T00:00:00Z"), null); Rule dropRule = new IntervalDropRule(Intervals.of("2013-01-01T00:00:00Z/2013-01-02T00:00:00Z")); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - null, - databaseRuleManager, - null, - new AuthConfig(), - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, null, databaseRuleManager, null, null); // test dropped EasyMock.expect(databaseRuleManager.getRulesWithDefault("dataSource1")) @@ -691,17 +659,14 @@ public void testIsHandOffComplete() String interval3 = "2013-01-02T02:00:00Z/2013-01-02T03:00:00Z"; SegmentLoadInfo segmentLoadInfo = new SegmentLoadInfo(createSegment(Intervals.of(interval3), "v1", 1)); segmentLoadInfo.addServer(createHistoricalServerMetadata("test")); - VersionedIntervalTimeline timeline = new VersionedIntervalTimeline( - null) + VersionedIntervalTimeline timeline = + new VersionedIntervalTimeline(null) { @Override public List> lookupWithIncompletePartitions(Interval interval) { - PartitionHolder partitionHolder = new PartitionHolder<>(new NumberedPartitionChunk<>( - 1, - 1, - segmentLoadInfo - )); + PartitionHolder partitionHolder = + new PartitionHolder<>(new NumberedPartitionChunk<>(1, 1, segmentLoadInfo)); List> ret = new ArrayList<>(); ret.add(new TimelineObjectHolder<>(Intervals.of(interval3), "v1", partitionHolder)); return ret; @@ -723,310 +688,212 @@ public List> lookupWithIncompleteP } @Test - public void testEnableDatasourceSegment() + public void testMarkSegmentAsUsed() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); - EasyMock.expect(metadataSegmentManager.enableSegment(dataSegmentList.get(0).getId().toString())) - .andReturn(true) - .once(); - EasyMock.replay(metadataSegmentManager); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSegment segment = dataSegmentList.get(0); + EasyMock.expect(segmentsMetadata.markSegmentAsUsed(segment.getId().toString())).andReturn(true).once(); + EasyMock.replay(segmentsMetadata); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - null, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegment(dataSegmentList.get(0).getDataSource(), dataSegmentList.get(0).getId().toString()); + Response response = dataSourcesResource.markSegmentAsUsed(segment.getDataSource(), segment.getId().toString()); Assert.assertEquals(200, response.getStatus()); - EasyMock.verify(metadataSegmentManager); + EasyMock.verify(segmentsMetadata); } @Test - public void testEnableDatasourceSegmentFailed() + public void testMarkSegmentAsUsedNoChange() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); - EasyMock.expect(metadataSegmentManager.enableSegment(dataSegmentList.get(0).getId().toString())) - .andReturn(false) - .once(); - EasyMock.replay(metadataSegmentManager); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSegment segment = dataSegmentList.get(0); + EasyMock.expect(segmentsMetadata.markSegmentAsUsed(segment.getId().toString())).andReturn(false).once(); + EasyMock.replay(segmentsMetadata); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - null, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = new DataSourcesResource(null, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegment(dataSegmentList.get(0).getDataSource(), dataSegmentList.get(0).getId().toString()); - Assert.assertEquals(204, response.getStatus()); - EasyMock.verify(metadataSegmentManager); + Response response = dataSourcesResource.markSegmentAsUsed(segment.getDataSource(), segment.getId().toString()); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("segmentStateChanged", false), response.getEntity()); + EasyMock.verify(segmentsMetadata); } @Test - public void testEnableDatasourceSegmentsInterval() + public void testMarkAsUsedNonOvershadowedSegmentsInterval() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Interval interval = Intervals.of("2010-01-22/P1D"); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(interval))) - .andReturn(3) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); + EasyMock.expect(numUpdatedSegments).andReturn(3).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - interval, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(interval, null) ); Assert.assertEquals(200, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentsIntervalNoneUpdated() + public void testMarkAsUsedNonOvershadowedSegmentsIntervalNoneUpdated() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Interval interval = Intervals.of("2010-01-22/P1D"); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(interval))) - .andReturn(0) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); + EasyMock.expect(numUpdatedSegments).andReturn(0).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - interval, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(interval, null) ); - Assert.assertEquals(204, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentsSet() + public void testMarkAsUsedNonOvershadowedSegmentsSet() throws UnknownSegmentIdException { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Set segmentIds = ImmutableSet.of(dataSegmentList.get(1).getId().toString()); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(segmentIds))) - .andReturn(3) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegments(EasyMock.eq("datasource1"), EasyMock.eq(segmentIds)); + EasyMock.expect(numUpdatedSegments).andReturn(3).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds) ); Assert.assertEquals(200, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentsIntervalException() + public void testMarkAsUsedNonOvershadowedSegmentsIntervalException() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); DruidDataSource dataSource = new DruidDataSource("datasource1", new HashMap<>()); Interval interval = Intervals.of("2010-01-22/P1D"); - EasyMock.expect(metadataSegmentManager.enableSegments(EasyMock.eq("datasource1"), EasyMock.eq(interval))) - .andThrow(new RuntimeException("Error!")) - .once(); + int numUpdatedSegments = + segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(EasyMock.eq("datasource1"), EasyMock.eq(interval)); + EasyMock.expect(numUpdatedSegments).andThrow(new RuntimeException("Error!")).once(); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - interval, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(interval, null) ); Assert.assertEquals(500, response.getStatus()); - EasyMock.verify(metadataSegmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testEnableDatasourceSegmentslNoDatasource() + public void testMarkAsUsedNonOvershadowedSegmentsNoDataSource() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(null).once(); - EasyMock.replay(metadataSegmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - Intervals.of("2010-01-22/P1D"), - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-22/P1D"), null) ); Assert.assertEquals(204, response.getStatus()); - EasyMock.verify(metadataSegmentManager); + EasyMock.verify(segmentsMetadata); } @Test - public void testEnableDatasourceSegmentsInvalidPayloadNoArguments() + public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadNoArguments() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - null - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, null) ); Assert.assertEquals(400, response.getStatus()); } @Test - public void testEnableDatasourceSegmentsInvalidPayloadBothArguments() + public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadBothArguments() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - Intervals.of("2010-01-22/P1D"), - ImmutableSet.of() - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-22/P1D"), ImmutableSet.of()) ); Assert.assertEquals(400, response.getStatus()); } @Test - public void testEnableDatasourceSegmentsInvalidPayloadEmptyArray() + public void testMarkAsUsedNonOvershadowedSegmentsInvalidPayloadEmptyArray() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments( "datasource1", - new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - ImmutableSet.of() - ) + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, ImmutableSet.of()) ); Assert.assertEquals(400, response.getStatus()); } @Test - public void testEnableDatasourceSegmentsNoPayload() + public void testMarkAsUsedNonOvershadowedSegmentsNoPayload() { - MetadataSegmentManager metadataSegmentManager = EasyMock.createMock(MetadataSegmentManager.class); + MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - metadataSegmentManager, - null, - null, - null, - null - ); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.enableDatasourceSegments( - "datasource1", - null - ); + Response response = dataSourcesResource.markAsUsedNonOvershadowedSegments("datasource1", null); Assert.assertEquals(400, response.getStatus()); } @Test public void testSegmentLoadChecksForVersion() { - Interval interval = Intervals.of( - "2011-04-01/2011-04-02" - ); + Interval interval = Intervals.of("2011-04-01/2011-04-02"); Assert.assertFalse( DataSourcesResource.isSegmentLoaded( Collections.singletonList( @@ -1068,9 +935,7 @@ public void testSegmentLoadChecksForVersion() @Test public void testSegmentLoadChecksForAssignableServer() { - Interval interval = Intervals.of( - "2011-04-01/2011-04-02" - ); + Interval interval = Intervals.of("2011-04-01/2011-04-02"); Assert.assertTrue( DataSourcesResource.isSegmentLoaded( Collections.singletonList( @@ -1099,9 +964,7 @@ public void testSegmentLoadChecksForAssignableServer() @Test public void testSegmentLoadChecksForPartitionNumber() { - Interval interval = Intervals.of( - "2011-04-01/2011-04-02" - ); + Interval interval = Intervals.of("2011-04-01/2011-04-02"); Assert.assertTrue( DataSourcesResource.isSegmentLoaded( Collections.singletonList( @@ -1158,218 +1021,159 @@ public void testSegmentLoadChecksForInterval() } @Test - public void testMarkDatasourceUnusedWithSegments() + public void testMarkSegmentsAsUnused() { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final Set segmentIds = dataSegmentList.stream() - .map(ds -> ds.getId().toString()) - .collect(Collectors.toSet()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final Set segmentIds = + dataSegmentList.stream().map(ds -> ds.getId().toString()).collect(Collectors.toSet()); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", segmentIds)).andReturn(1L).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markSegmentsAsUnused("datasource1", segmentIds)).andReturn(1).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 1), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithSegmentsNoContent() + public void testMarkSegmentsAsUnusedNoChanges() { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final Set segmentIds = dataSegmentList.stream() - .map(ds -> ds.getId().toString()) - .collect(Collectors.toSet()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final Set segmentIds = + dataSegmentList.stream().map(ds -> ds.getId().toString()).collect(Collectors.toSet()); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", segmentIds)).andReturn(0L).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markSegmentsAsUnused("datasource1", segmentIds)).andReturn(0).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); - Assert.assertEquals(204, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithSegmentsException() + public void testMarkSegmentsAsUnusedException() { final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final Set segmentIds = dataSegmentList.stream() - .map(ds -> ds.getId().toString()) - .collect(Collectors.toSet()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final Set segmentIds = + dataSegmentList.stream().map(ds -> ds.getId().toString()).collect(Collectors.toSet()); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", segmentIds)) + EasyMock.expect(segmentsMetadata.markSegmentsAsUnused("datasource1", segmentIds)) .andThrow(new RuntimeException("Exception occurred")) .once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - segmentIds - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, segmentIds); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(500, response.getStatus()); Assert.assertNotNull(response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithInterval() + public void testMarkAsUnusedSegmentsInInterval() { final Interval theInterval = Intervals.of("2010-01-01/P1D"); final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", theInterval)).andReturn(1).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(1).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - theInterval, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(200, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); - EasyMock.verify(segmentManager, inventoryView, server); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 1), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithIntervaNoContent() + public void testMarkAsUnusedSegmentsInIntervalNoChanges() { final Interval theInterval = Intervals.of("2010-01-01/P1D"); final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", theInterval)).andReturn(0).once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.expect(segmentsMetadata.markAsUnusedSegmentsInInterval("datasource1", theInterval)).andReturn(0).once(); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - theInterval, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); - Assert.assertEquals(204, response.getStatus()); - Assert.assertEquals(null, response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); + Assert.assertEquals(200, response.getStatus()); + Assert.assertEquals(ImmutableMap.of("numChangedSegments", 0), response.getEntity()); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedWithIntervaException() + public void testMarkAsUnusedSegmentsInIntervalException() { final Interval theInterval = Intervals.of("2010-01-01/P1D"); final DruidDataSource dataSource1 = new DruidDataSource("datasource1", new HashMap<>()); - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); EasyMock.expect(inventoryView.getInventory()).andReturn(ImmutableList.of(server)).once(); EasyMock.expect(server.getDataSource("datasource1")).andReturn(dataSource1).once(); - EasyMock.expect(segmentManager.disableSegments("datasource1", theInterval)) + EasyMock.expect(segmentsMetadata.markAsUnusedSegmentsInInterval("datasource1", theInterval)) .andThrow(new RuntimeException("Exception occurred")) .once(); - EasyMock.replay(segmentManager, inventoryView, server); + EasyMock.replay(segmentsMetadata, inventoryView, server); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - theInterval, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(theInterval, null); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - new AuthConfig(), - null - ); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(500, response.getStatus()); Assert.assertNotNull(response.getEntity()); - EasyMock.verify(segmentManager, inventoryView, server); + EasyMock.verify(segmentsMetadata, inventoryView, server); } @Test - public void testMarkDatasourceUnusedNullPayload() + public void testMarkSegmentsUnusedNullPayload() { - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - null, - null - ); - - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = null; + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", null); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); Assert.assertEquals( @@ -1379,47 +1183,31 @@ public void testMarkDatasourceUnusedNullPayload() } @Test - public void testMarkDatasourceUnusedInvalidPayload() + public void testMarkSegmentsUnusedInvalidPayload() { - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - null, - null - ); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - null, - null - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(null, null); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); } @Test - public void testMarkDatasourceUnusedInvalidPayloadBothArguments() + public void testMarkSegmentsUnusedInvalidPayloadBothArguments() { - final MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); - DataSourcesResource dataSourcesResource = new DataSourcesResource( - inventoryView, - segmentManager, - null, - null, - null, - null - ); + final MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class); + DataSourcesResource dataSourcesResource = + new DataSourcesResource(inventoryView, segmentsMetadata, null, null, null); - final DataSourcesResource.MarkDatasourceSegmentsPayload payload = new DataSourcesResource.MarkDatasourceSegmentsPayload( - Intervals.of("2010-01-01/P1D"), - ImmutableSet.of() - ); + final DataSourcesResource.MarkDataSourceSegmentsPayload payload = + new DataSourcesResource.MarkDataSourceSegmentsPayload(Intervals.of("2010-01-01/P1D"), ImmutableSet.of()); - Response response = dataSourcesResource.markDatasourceUnused("datasource1", payload); + Response response = dataSourcesResource.markSegmentsAsUnused("datasource1", payload); Assert.assertEquals(400, response.getStatus()); Assert.assertNotNull(response.getEntity()); } @@ -1436,15 +1224,7 @@ private DruidServerMetadata createHistoricalServerMetadata(String name) private DruidServerMetadata createServerMetadata(String name, ServerType type) { - return new DruidServerMetadata( - name, - name, - null, - 10000, - type, - "tier", - 1 - ); + return new DruidServerMetadata(name, name, null, 10000, type, "tier", 1); } private DataSegment createSegment(Interval interval, String version, int partitionNumber) 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 42b7e949c880..0b171bdc6feb 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 @@ -555,7 +555,7 @@ public Enumerable scan(DataContext root) for (ImmutableDruidServer druidServer : druidServers) { final Iterable authorizedServerSegments = AuthorizationUtils.filterAuthorizedResources( authenticationResult, - druidServer.getLazyAllSegments(), + druidServer.iterateAllSegments(), SEGMENT_RA_GENERATOR, authorizerMapper ); 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 dcabfbae3f03..08fcb7ebe60b 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 @@ -275,7 +275,8 @@ public void testSegmentMetadataHolderNumRows() final Pair pair = druidServers .stream() .flatMap(druidServer -> druidServer - .getLazyAllSegments().stream() + .iterateAllSegments() + .stream() .filter(segment -> segment.getId().equals(existingSegment.getId())) .map(segment -> Pair.of(druidServer, segment)) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 49e406b0e6eb..ff98ef2517c3 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -105,11 +105,10 @@ import java.nio.charset.StandardCharsets; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.Stream; public class SystemSchemaTest extends CalciteTestBase { @@ -484,19 +483,18 @@ public void testGetTableMap() @Test public void testSegmentsTable() { - final SystemSchema.SegmentsTable segmentsTable = EasyMock .createMockBuilder(SystemSchema.SegmentsTable.class) .withConstructor(druidSchema, metadataView, mapper, authMapper) .createMock(); EasyMock.replay(segmentsTable); - final Set publishedSegments = Stream.of( + final Set publishedSegments = new HashSet<>(Arrays.asList( new SegmentWithOvershadowedStatus(publishedSegment1, true), new SegmentWithOvershadowedStatus(publishedSegment2, false), new SegmentWithOvershadowedStatus(publishedSegment3, false), new SegmentWithOvershadowedStatus(segment1, true), new SegmentWithOvershadowedStatus(segment2, false) - ).collect(Collectors.toSet()); + )); EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once(); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 4d1206b18b74..6aac2ce2eab1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -78,9 +78,8 @@ public SpecificSegmentsQuerySegmentWalker add( ) { final Segment segment = new QueryableIndexSegment(index, descriptor.getId()); - timelines.computeIfAbsent(descriptor.getDataSource(), datasource -> new VersionedIntervalTimeline<>(Ordering.natural())); - - final VersionedIntervalTimeline timeline = timelines.get(descriptor.getDataSource()); + final VersionedIntervalTimeline timeline = timelines + .computeIfAbsent(descriptor.getDataSource(), dsName -> new VersionedIntervalTimeline<>(Ordering.natural())); timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment)); segments.add(descriptor); closeables.add(index); diff --git a/web-console/README.md b/web-console/README.md index 8ea53e85abfb..ceca974feb4e 100644 --- a/web-console/README.md +++ b/web-console/README.md @@ -59,7 +59,7 @@ POST /druid/indexer/v1/worker 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