Refactor SQLMetadataSegmentManager; Change contract of REST met… (#7653)

* Refactor SQLMetadataSegmentManager; Change contract of REST methods in DataSourcesResource

* Style fixes

* Unused imports

* Fix tests

* Fix style

* Comments

* Comment fix

* Remove unresolvable Javadoc references; address comments

* Add comments to ImmutableDruidDataSource

* Merge with master

* Fix bad web-console merge

* Fixes in api-reference.md

* Rename in DruidCoordinatorRuntimeParams

* Fix compilation

* Residual changes
This commit is contained in:
Roman Leventov 2019-07-17 17:18:48 +03:00 committed by GitHub
parent b80f20f769
commit ceb969903f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
85 changed files with 3422 additions and 4007 deletions

View File

@ -311,6 +311,11 @@
<constraint name="k" within="" contains="" /> <constraint name="k" within="" contains="" />
<constraint name="v" within="" contains="" /> <constraint name="v" within="" contains="" />
</searchConfiguration> </searchConfiguration>
<searchConfiguration name="Use collections constructors directly" text="Stream.of($x$).collect(Collectors.$m$())" recursive="true" caseInsensitive="true" type="JAVA">
<constraint name="__context__" target="true" within="" contains="" />
<constraint name="m" within="" contains="" />
<constraint name="x" minCount="0" maxCount="2147483647" within="" contains="" />
</searchConfiguration>
</inspection_tool> </inspection_tool>
<inspection_tool class="SimplifyStreamApiCallChains" enabled="true" level="ERROR" enabled_by_default="true" /> <inspection_tool class="SimplifyStreamApiCallChains" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false"> <inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">

View File

@ -20,6 +20,8 @@
package org.apache.druid.server.coordinator; package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableList; 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.java.util.common.DateTimes;
import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator; import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator;
import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy; import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy;
@ -42,6 +44,7 @@ import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.infra.Blackhole; import org.openjdk.jmh.infra.Blackhole;
import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@ -102,14 +105,10 @@ public class NewestSegmentFirstPolicyBenchmark
); );
} }
dataSources = new HashMap<>(); List<DataSegment> segments = new ArrayList<>();
for (int i = 0; i < numDataSources; i++) { for (int i = 0; i < numDataSources; i++) {
final String dataSource = DATA_SOURCE_PREFIX + i; final String dataSource = DATA_SOURCE_PREFIX + i;
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
);
final int startYear = ThreadLocalRandom.current().nextInt(2000, 2040); final int startYear = ThreadLocalRandom.current().nextInt(2000, 2040);
DateTime date = DateTimes.of(startYear, 1, 1, 0, 0); DateTime date = DateTimes.of(startYear, 1, 1, 0, 0);
@ -127,12 +126,11 @@ public class NewestSegmentFirstPolicyBenchmark
0, 0,
segmentSizeBytes 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 @Benchmark

View File

@ -123,6 +123,11 @@ public class Logger
log.error(StringUtils.nonStrictFormat(message, formatArgs), t); 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) public void wtf(String message, Object... formatArgs)
{ {
log.error(StringUtils.nonStrictFormat("WTF?!: " + message, formatArgs), new Exception()); log.error(StringUtils.nonStrictFormat("WTF?!: " + message, formatArgs), new Exception());

View File

@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.AlertBuilder; import org.apache.druid.java.util.emitter.service.AlertBuilder;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import javax.annotation.Nullable;
import java.io.PrintWriter; import java.io.PrintWriter;
import java.io.StringWriter; import java.io.StringWriter;
@ -61,15 +62,22 @@ public class EmittingLogger extends Logger
return makeAlert(null, message, 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) { if (emitter == null) {
final String errorMessage = StringUtils.format( 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); 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) final AlertBuilder retVal = new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter)

View File

@ -22,6 +22,7 @@ package org.apache.druid.timeline;
import org.apache.druid.timeline.partition.PartitionHolder; import org.apache.druid.timeline.partition.PartitionHolder;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.List; import java.util.List;
@ -50,5 +51,5 @@ public interface TimelineLookup<VersionType, ObjectType>
*/ */
List<TimelineObjectHolder<VersionType, ObjectType>> lookupWithIncompletePartitions(Interval interval); List<TimelineObjectHolder<VersionType, ObjectType>> lookupWithIncompletePartitions(Interval interval);
PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version); @Nullable PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version);
} }

View File

@ -23,16 +23,18 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.Iterators; 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.DateTimes;
import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.timeline.partition.ImmutablePartitionHolder; import org.apache.druid.timeline.partition.ImmutablePartitionHolder;
import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.PartitionHolder; import org.apache.druid.timeline.partition.PartitionHolder;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator; import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
@ -44,7 +46,9 @@ import java.util.NavigableMap;
import java.util.Objects; import java.util.Objects;
import java.util.Set; import java.util.Set;
import java.util.TreeMap; import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.StreamSupport;
/** /**
* VersionedIntervalTimeline is a data structure that manages objects on a specific timeline. * VersionedIntervalTimeline is a data structure that manages objects on a specific timeline.
@ -75,12 +79,11 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
Comparators.intervalsByStartThenEnd() Comparators.intervalsByStartThenEnd()
); );
private final Map<Interval, TreeMap<VersionType, TimelineEntry>> allTimelineEntries = new HashMap<>(); private final Map<Interval, TreeMap<VersionType, TimelineEntry>> allTimelineEntries = new HashMap<>();
private final AtomicInteger numObjects = new AtomicInteger();
private final Comparator<? super VersionType> versionComparator; private final Comparator<? super VersionType> versionComparator;
public VersionedIntervalTimeline( public VersionedIntervalTimeline(Comparator<? super VersionType> versionComparator)
Comparator<? super VersionType> versionComparator
)
{ {
this.versionComparator = versionComparator; this.versionComparator = versionComparator;
} }
@ -92,7 +95,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
public static VersionedIntervalTimeline<String, DataSegment> forSegments(Iterator<DataSegment> segments) public static VersionedIntervalTimeline<String, DataSegment> forSegments(Iterator<DataSegment> segments)
{ {
final VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(Ordering.natural()); final VersionedIntervalTimeline<String, DataSegment> timeline =
new VersionedIntervalTimeline<>(Comparator.naturalOrder());
addSegments(timeline, segments); addSegments(timeline, segments);
return timeline; return timeline;
} }
@ -115,6 +119,28 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
return allTimelineEntries; 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<ObjectType> iterateAllObjects()
{
return CollectionUtils.createLazyCollectionFromStream(
() -> allTimelineEntries
.values()
.stream()
.flatMap((TreeMap<VersionType, TimelineEntry> 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<ObjectType> object) public void add(final Interval interval, VersionType version, PartitionChunk<ObjectType> object)
{ {
addAll(Iterators.singletonIterator(object), o -> interval, o -> version); addAll(Iterators.singletonIterator(object), o -> interval, o -> version);
@ -143,15 +169,19 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
TreeMap<VersionType, TimelineEntry> versionEntry = new TreeMap<>(versionComparator); TreeMap<VersionType, TimelineEntry> versionEntry = new TreeMap<>(versionComparator);
versionEntry.put(version, entry); versionEntry.put(version, entry);
allTimelineEntries.put(interval, versionEntry); allTimelineEntries.put(interval, versionEntry);
numObjects.incrementAndGet();
} else { } else {
entry = exists.get(version); entry = exists.get(version);
if (entry == null) { if (entry == null) {
entry = new TimelineEntry(interval, version, new PartitionHolder<>(object)); entry = new TimelineEntry(interval, version, new PartitionHolder<>(object));
exists.put(version, entry); exists.put(version, entry);
numObjects.incrementAndGet();
} else { } else {
PartitionHolder<ObjectType> partitionHolder = entry.getPartitionHolder(); PartitionHolder<ObjectType> partitionHolder = entry.getPartitionHolder();
partitionHolder.add(object); if (partitionHolder.add(object)) {
numObjects.incrementAndGet();
}
} }
} }
@ -174,6 +204,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
} }
} }
@Nullable
public PartitionChunk<ObjectType> remove(Interval interval, VersionType version, PartitionChunk<ObjectType> chunk) public PartitionChunk<ObjectType> remove(Interval interval, VersionType version, PartitionChunk<ObjectType> chunk)
{ {
try { try {
@ -189,7 +220,11 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
return null; return null;
} }
PartitionChunk<ObjectType> retVal = entry.getPartitionHolder().remove(chunk); PartitionChunk<ObjectType> removedChunk = entry.getPartitionHolder().remove(chunk);
if (removedChunk == null) {
return null;
}
numObjects.decrementAndGet();
if (entry.getPartitionHolder().isEmpty()) { if (entry.getPartitionHolder().isEmpty()) {
versionEntries.remove(version); versionEntries.remove(version);
if (versionEntries.isEmpty()) { if (versionEntries.isEmpty()) {
@ -201,7 +236,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
remove(completePartitionsTimeline, interval, entry, false); remove(completePartitionsTimeline, interval, entry, false);
return retVal; return removedChunk;
} }
finally { finally {
lock.writeLock().unlock(); lock.writeLock().unlock();
@ -209,7 +244,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
} }
@Override @Override
public PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version) public @Nullable PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version)
{ {
try { try {
lock.readLock().lock(); lock.readLock().lock();
@ -217,9 +252,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) { if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) {
TimelineEntry foundEntry = entry.getValue().get(version); TimelineEntry foundEntry = entry.getValue().get(version);
if (foundEntry != null) { if (foundEntry != null) {
return new ImmutablePartitionHolder<ObjectType>( return new ImmutablePartitionHolder<>(foundEntry.getPartitionHolder());
foundEntry.getPartitionHolder()
);
} }
} }
} }
@ -307,6 +340,10 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
); );
} }
/**
* This method should be deduplicated with DataSourcesSnapshot.determineOvershadowedSegments(): see
* https://github.com/apache/incubator-druid/issues/8070.
*/
public Set<TimelineObjectHolder<VersionType, ObjectType>> findOvershadowed() public Set<TimelineObjectHolder<VersionType, ObjectType>> findOvershadowed()
{ {
try { try {
@ -315,8 +352,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = new HashMap<>(); Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = new HashMap<>();
for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) { for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) {
Map<VersionType, TimelineEntry> versionCopy = new HashMap<>(); @SuppressWarnings("unchecked")
versionCopy.putAll(versionEntry.getValue()); Map<VersionType, TimelineEntry> versionCopy = (TreeMap) versionEntry.getValue().clone();
overShadowed.put(versionEntry.getKey(), versionCopy); overShadowed.put(versionEntry.getKey(), versionCopy);
} }

View File

@ -23,7 +23,7 @@ package org.apache.druid.timeline.partition;
*/ */
public class ImmutablePartitionHolder<T> extends PartitionHolder<T> public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
{ {
public ImmutablePartitionHolder(PartitionHolder partitionHolder) public ImmutablePartitionHolder(PartitionHolder<T> partitionHolder)
{ {
super(partitionHolder); super(partitionHolder);
} }
@ -35,7 +35,7 @@ public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
} }
@Override @Override
public void add(PartitionChunk<T> tPartitionChunk) public boolean add(PartitionChunk<T> tPartitionChunk)
{ {
throw new UnsupportedOperationException(); throw new UnsupportedOperationException();
} }

View File

@ -22,72 +22,62 @@ package org.apache.druid.timeline.partition;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Iterators; import com.google.common.collect.Iterators;
import javax.annotation.Nullable;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
import java.util.SortedSet;
import java.util.Spliterator; 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. * An object that clumps together multiple other objects which each represent a shard of some space.
*/ */
public class PartitionHolder<T> implements Iterable<PartitionChunk<T>> public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
{ {
private final TreeSet<PartitionChunk<T>> holderSet; private final TreeMap<PartitionChunk<T>, PartitionChunk<T>> holderMap;
public PartitionHolder(PartitionChunk<T> initialChunk) public PartitionHolder(PartitionChunk<T> initialChunk)
{ {
this.holderSet = new TreeSet<>(); this.holderMap = new TreeMap<>();
add(initialChunk); add(initialChunk);
} }
public PartitionHolder(List<PartitionChunk<T>> initialChunks) public PartitionHolder(List<PartitionChunk<T>> initialChunks)
{ {
this.holderSet = new TreeSet<>(); this.holderMap = new TreeMap<>();
for (PartitionChunk<T> chunk : initialChunks) { for (PartitionChunk<T> chunk : initialChunks) {
add(chunk); add(chunk);
} }
} }
public PartitionHolder(PartitionHolder partitionHolder) public PartitionHolder(PartitionHolder<T> partitionHolder)
{ {
this.holderSet = new TreeSet<>(); this.holderMap = new TreeMap<>();
this.holderSet.addAll(partitionHolder.holderSet); this.holderMap.putAll(partitionHolder.holderMap);
} }
public void add(PartitionChunk<T> chunk) public boolean add(PartitionChunk<T> chunk)
{ {
holderSet.add(chunk); return holderMap.putIfAbsent(chunk, chunk) == null;
} }
@Nullable
public PartitionChunk<T> remove(PartitionChunk<T> chunk) public PartitionChunk<T> remove(PartitionChunk<T> chunk)
{ {
if (!holderSet.isEmpty()) { return holderMap.remove(chunk);
// Somewhat funky implementation in order to return the removed object as it exists in the set
SortedSet<PartitionChunk<T>> tailSet = holderSet.tailSet(chunk, true);
if (!tailSet.isEmpty()) {
PartitionChunk<T> element = tailSet.first();
if (chunk.equals(element)) {
holderSet.remove(element);
return element;
}
}
}
return null;
} }
public boolean isEmpty() public boolean isEmpty()
{ {
return holderSet.isEmpty(); return holderMap.isEmpty();
} }
public boolean isComplete() public boolean isComplete()
{ {
if (holderSet.isEmpty()) { if (holderMap.isEmpty()) {
return false; return false;
} }
Iterator<PartitionChunk<T>> iter = holderSet.iterator(); Iterator<PartitionChunk<T>> iter = holderMap.keySet().iterator();
PartitionChunk<T> curr = iter.next(); PartitionChunk<T> curr = iter.next();
@ -117,7 +107,7 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
public PartitionChunk<T> getChunk(final int partitionNum) public PartitionChunk<T> getChunk(final int partitionNum)
{ {
final Iterator<PartitionChunk<T>> retVal = Iterators.filter( final Iterator<PartitionChunk<T>> retVal = Iterators.filter(
holderSet.iterator(), holderMap.keySet().iterator(),
input -> input.getChunkNumber() == partitionNum input -> input.getChunkNumber() == partitionNum
); );
@ -127,13 +117,13 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
@Override @Override
public Iterator<PartitionChunk<T>> iterator() public Iterator<PartitionChunk<T>> iterator()
{ {
return holderSet.iterator(); return holderMap.keySet().iterator();
} }
@Override @Override
public Spliterator<PartitionChunk<T>> spliterator() public Spliterator<PartitionChunk<T>> spliterator()
{ {
return holderSet.spliterator(); return holderMap.keySet().spliterator();
} }
public Iterable<T> payloads() public Iterable<T> payloads()
@ -153,7 +143,7 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
PartitionHolder that = (PartitionHolder) o; PartitionHolder that = (PartitionHolder) o;
if (!holderSet.equals(that.holderSet)) { if (!holderMap.equals(that.holderMap)) {
return false; return false;
} }
@ -163,14 +153,14 @@ public class PartitionHolder<T> implements Iterable<PartitionChunk<T>>
@Override @Override
public int hashCode() public int hashCode()
{ {
return holderSet.hashCode(); return holderMap.hashCode();
} }
@Override @Override
public String toString() public String toString()
{ {
return "PartitionHolder{" + return "PartitionHolder{" +
"holderSet=" + holderSet + "holderMap=" + holderMap +
'}'; '}';
} }
} }

View File

@ -19,13 +19,16 @@
package org.apache.druid.utils; package org.apache.druid.utils;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps; import com.google.common.collect.Maps;
import java.util.AbstractCollection; import java.util.AbstractCollection;
import java.util.Collection; import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map; import java.util.Map;
import java.util.Spliterator; import java.util.Spliterator;
import java.util.TreeSet;
import java.util.function.Function; import java.util.function.Function;
import java.util.function.Supplier; import java.util.function.Supplier;
import java.util.stream.Stream; import java.util.stream.Stream;
@ -72,6 +75,13 @@ public final class CollectionUtils
}; };
} }
public static <E> TreeSet<E> newTreeSet(Comparator<? super E> comparator, Iterable<E> elements)
{
TreeSet<E> 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 * Returns a transformed map from the given input map where the value is modified based on the given valueMapper
* function. * function.

View File

@ -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;

View File

@ -240,22 +240,22 @@ public class VersionedIntervalTimelineTest
public void testFindEntry() public void testFindEntry()
{ {
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1") timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1") timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1") timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1")
); );
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1") timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1")
); );
@ -279,7 +279,7 @@ public class VersionedIntervalTimelineTest
add("2011-01-02/2011-01-05", "2", 1); add("2011-01-02/2011-01-05", "2", 1);
Assert.assertEquals( Assert.assertEquals(
new ImmutablePartitionHolder<Integer>(new PartitionHolder<Integer>(makeSingle(1))), new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle(1))),
timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1") timeline.findEntry(Intervals.of("2011-01-02T02/2011-01-04"), "1")
); );
} }
@ -1564,11 +1564,11 @@ public class VersionedIntervalTimelineTest
{ {
timeline = makeStringIntegerTimeline(); timeline = makeStringIntegerTimeline();
add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-05/2011-04-07", "1", new SingleElementPartitionChunk<>(1));
add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-07/2011-04-09", "1", new SingleElementPartitionChunk<>(1));
add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-15/2011-04-17", "1", new SingleElementPartitionChunk<>(1));
add("2011-04-17/2011-04-19", "1", new SingleElementPartitionChunk<Integer>(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-03"), "0"));
Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-05"), "0"));
@ -1629,11 +1629,11 @@ public class VersionedIntervalTimelineTest
{ {
timeline = makeStringIntegerTimeline(); timeline = makeStringIntegerTimeline();
add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-05/2011-04-09", "11", new SingleElementPartitionChunk<>(1));
add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-07/2011-04-11", "12", new SingleElementPartitionChunk<>(1));
add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk<Integer>(1)); add("2011-04-15/2011-04-19", "12", new SingleElementPartitionChunk<>(1));
add("2011-04-17/2011-04-21", "11", new SingleElementPartitionChunk<Integer>(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")); Assert.assertFalse(timeline.isOvershadowed(Intervals.of("2011-04-01/2011-04-03"), "0"));
@ -1730,13 +1730,13 @@ public class VersionedIntervalTimelineTest
{ {
return Pair.of( return Pair.of(
Intervals.of(intervalString), Intervals.of(intervalString),
Pair.of(version, new PartitionHolder<Integer>(values)) Pair.of(version, new PartitionHolder<>(values))
); );
} }
private SingleElementPartitionChunk<Integer> makeSingle(Integer value) private SingleElementPartitionChunk<Integer> makeSingle(Integer value)
{ {
return new SingleElementPartitionChunk<Integer>(value); return new SingleElementPartitionChunk<>(value);
} }
private void add(String interval, String version, Integer value) private void add(String interval, String version, Integer value)
@ -1808,7 +1808,7 @@ public class VersionedIntervalTimelineTest
private VersionedIntervalTimeline<String, Integer> makeStringIntegerTimeline() private VersionedIntervalTimeline<String, Integer> makeStringIntegerTimeline()
{ {
return new VersionedIntervalTimeline<String, Integer>(Ordering.natural()); return new VersionedIntervalTimeline<>(Ordering.natural());
} }
} }

View File

@ -37,7 +37,8 @@ A data deletion tutorial is available at [Tutorial: Deleting data](../tutorials/
## Kill Task ## 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 ```json
{ {

View File

@ -113,15 +113,17 @@ Returns the serialized JSON of segments to load and drop for each Historical pro
* `/druid/coordinator/v1/metadata/datasources` * `/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` * `/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}` * `/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}` * `/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": <number>}` 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}` * `/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": <boolean>}` 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` * `/druid/coordinator/v1/datasources/{dataSourceName}/markUsed`
@ -259,7 +265,9 @@ JSON Request Payload:
* `/druid/coordinator/v1/datasources/{dataSourceName}` * `/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": <number>}` 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}` * `/druid/coordinator/v1/datasources/{dataSourceName}/intervals/{interval}`
* `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}` * `@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}` * `/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": <boolean>}` 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 #### Retention Rules
@ -595,7 +605,7 @@ Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_`
* `/druid/indexer/v1/worker` * `/druid/indexer/v1/worker`
Retreives current overlord dynamic configuration. Retrieves current overlord dynamic configuration.
* `/druid/indexer/v1/worker/history?interval={interval}&counter={count}` * `/druid/indexer/v1/worker/history?interval={interval}&counter={count}`

View File

@ -390,7 +390,7 @@ public class MaterializedViewSupervisor implements Supervisor
// drop derivative segments which interval equals the interval in toDeleteBaseSegments // drop derivative segments which interval equals the interval in toDeleteBaseSegments
for (Interval interval : toDropInterval.keySet()) { for (Interval interval : toDropInterval.keySet()) {
for (DataSegment segment : derivativeSegments.get(interval)) { 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. // data of the latest interval will be built firstly.
@ -498,7 +498,7 @@ public class MaterializedViewSupervisor implements Supervisor
{ {
log.info("Clear all metadata of dataSource %s", dataSource); log.info("Clear all metadata of dataSource %s", dataSource);
metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL); metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL);
segmentManager.removeDataSource(dataSource); segmentManager.markAsUnusedAllSegmentsInDataSource(dataSource);
metadataStorageCoordinator.deleteDataSourceMetadata(dataSource); metadataStorageCoordinator.deleteDataSourceMetadata(dataSource);
} }

View File

@ -91,6 +91,7 @@ import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.ShardSpec; import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.utils.CircularBuffer; import org.apache.druid.utils.CircularBuffer;
import org.apache.druid.utils.CollectionUtils;
import org.codehaus.plexus.util.FileUtils; import org.codehaus.plexus.util.FileUtils;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -444,8 +445,7 @@ public class IndexTask extends AbstractTask implements ChatHandler
toolbox.getTaskActionClient(), toolbox.getTaskActionClient(),
intervals intervals
); );
versions = locks.entrySet().stream() versions = CollectionUtils.mapValues(locks, TaskLock::getVersion);
.collect(Collectors.toMap(Entry::getKey, entry -> entry.getValue().getVersion()));
dataSchema = ingestionSchema.getDataSchema().withGranularitySpec( dataSchema = ingestionSchema.getDataSchema().withGranularitySpec(
ingestionSchema.getDataSchema() ingestionSchema.getDataSchema()

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import org.apache.druid.client.indexing.ClientKillQuery;
import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolbox;
@ -37,6 +38,9 @@ import java.util.List;
import java.util.Map; 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 public class KillTask extends AbstractFixedIntervalTask
{ {

View File

@ -73,7 +73,9 @@ public class SegmentListActionsTest
expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval())); 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) private DataSegment createSegment(Interval interval, String version)

View File

@ -19,11 +19,13 @@
package org.apache.druid.client; package org.apache.druid.client;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; 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.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
@ -31,40 +33,88 @@ import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.stream.Collectors;
/** /**
* An immutable snapshot of fields from {@link org.apache.druid.metadata.SQLMetadataSegmentManager} (dataSources and * An immutable snapshot information about used segments and overshadowed segments for
* overshadowedSegments). Getters of {@link org.apache.druid.metadata.MetadataSegmentManager} should use this snapshot * {@link org.apache.druid.metadata.SQLMetadataSegmentManager}.
* to return dataSources and overshadowedSegments.
*/ */
public class DataSourcesSnapshot public class DataSourcesSnapshot
{ {
private final Map<String, ImmutableDruidDataSource> dataSources; public static DataSourcesSnapshot fromUsedSegments(
private final ImmutableSet<SegmentId> overshadowedSegments; Iterable<DataSegment> segments,
ImmutableMap<String, String> dataSourceProperties
public DataSourcesSnapshot(
Map<String, ImmutableDruidDataSource> dataSources
) )
{ {
this.dataSources = dataSources; Map<String, DruidDataSource> 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<String, VersionedIntervalTimeline<String, DataSegment>> usedSegmentsTimelinesPerDataSource,
ImmutableMap<String, String> dataSourceProperties
)
{
Map<String, ImmutableDruidDataSource> 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<String, ImmutableDruidDataSource> dataSourcesWithAllUsedSegments;
private final Map<String, VersionedIntervalTimeline<String, DataSegment>> usedSegmentsTimelinesPerDataSource;
private final ImmutableSet<SegmentId> overshadowedSegments;
public DataSourcesSnapshot(Map<String, ImmutableDruidDataSource> dataSourcesWithAllUsedSegments)
{
this(
dataSourcesWithAllUsedSegments,
CollectionUtils.mapValues(
dataSourcesWithAllUsedSegments,
dataSource -> VersionedIntervalTimeline.forSegments(dataSource.getSegments())
)
);
}
private DataSourcesSnapshot(
Map<String, ImmutableDruidDataSource> dataSourcesWithAllUsedSegments,
Map<String, VersionedIntervalTimeline<String, DataSegment>> usedSegmentsTimelinesPerDataSource
)
{
this.dataSourcesWithAllUsedSegments = dataSourcesWithAllUsedSegments;
this.usedSegmentsTimelinesPerDataSource = usedSegmentsTimelinesPerDataSource;
this.overshadowedSegments = ImmutableSet.copyOf(determineOvershadowedSegments()); this.overshadowedSegments = ImmutableSet.copyOf(determineOvershadowedSegments());
} }
public Collection<ImmutableDruidDataSource> getDataSources() public Collection<ImmutableDruidDataSource> getDataSourcesWithAllUsedSegments()
{ {
return dataSources.values(); return dataSourcesWithAllUsedSegments.values();
} }
public Map<String, ImmutableDruidDataSource> getDataSourcesMap() public Map<String, ImmutableDruidDataSource> getDataSourcesMap()
{ {
return dataSources; return dataSourcesWithAllUsedSegments;
} }
@Nullable @Nullable
public ImmutableDruidDataSource getDataSource(String dataSourceName) public ImmutableDruidDataSource getDataSource(String dataSourceName)
{ {
return dataSources.get(dataSourceName); return dataSourcesWithAllUsedSegments.get(dataSourceName);
}
public Map<String, VersionedIntervalTimeline<String, DataSegment>> getUsedSegmentsTimelinesPerDataSource()
{
return usedSegmentsTimelinesPerDataSource;
} }
public ImmutableSet<SegmentId> getOvershadowedSegments() public ImmutableSet<SegmentId> getOvershadowedSegments()
@ -72,40 +122,48 @@ public class DataSourcesSnapshot
return overshadowedSegments; return overshadowedSegments;
} }
@Nullable /**
public Iterable<DataSegment> 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<DataSegment> iterateAllUsedSegmentsInSnapshot()
{ {
if (dataSources == null) { return () -> dataSourcesWithAllUsedSegments
return null; .values()
} .stream()
return () -> dataSources.values().stream() .flatMap(dataSource -> dataSource.getSegments().stream())
.flatMap(dataSource -> dataSource.getSegments().stream()) .iterator();
.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 * @return overshadowed segment Ids list
*/ */
private List<SegmentId> determineOvershadowedSegments() private List<SegmentId> determineOvershadowedSegments()
{ {
final List<DataSegment> segments = dataSources.values().stream()
.flatMap(ds -> ds.getSegments().stream())
.collect(Collectors.toList());
final Map<String, VersionedIntervalTimeline<String, DataSegment>> 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 // 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, // 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. // so building this collection shouldn't generate a lot of garbage.
final List<SegmentId> overshadowedSegments = new ArrayList<>(); final List<SegmentId> overshadowedSegments = new ArrayList<>();
for (DataSegment dataSegment : segments) { for (ImmutableDruidDataSource dataSource : dataSourcesWithAllUsedSegments.values()) {
final VersionedIntervalTimeline<String, DataSegment> timeline = timelines.get(dataSegment.getDataSource()); VersionedIntervalTimeline<String, DataSegment> usedSegmentsTimeline =
if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) { usedSegmentsTimelinesPerDataSource.get(dataSource.getName());
overshadowedSegments.add(dataSegment.getId()); for (DataSegment segment : dataSource.getSegments()) {
if (usedSegmentsTimeline.isOvershadowed(segment.getInterval(), segment.getVersion())) {
overshadowedSegments.add(segment.getId());
}
} }
} }
return overshadowedSegments; return overshadowedSegments;

View File

@ -30,12 +30,13 @@ import java.util.Collections;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; 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. * 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 * Concurrency: could be updated concurrently via {@link #addSegment}, {@link #removeSegment}, and {@link
* concurrently (e. g. via {@link #getSegments}) as well. * #removeSegmentsIf}, and accessed concurrently (e. g. via {@link #getSegments}) as well.
* *
* @see ImmutableDruidDataSource - an immutable counterpart of this class * @see ImmutableDruidDataSource - an immutable counterpart of this class
*/ */
@ -44,10 +45,7 @@ public class DruidDataSource
private final String name; private final String name;
private final Map<String, String> properties; private final Map<String, String> properties;
/** /**
* This map needs to be concurrent because it should be possible to iterate the segments of the data source * This map needs to be concurrent to support concurrent iteration and updates.
* (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.
*/ */
private final ConcurrentMap<SegmentId, DataSegment> idToSegmentMap = new ConcurrentHashMap<>(); private final ConcurrentMap<SegmentId, DataSegment> idToSegmentMap = new ConcurrentHashMap<>();
@ -80,6 +78,14 @@ public class DruidDataSource
return Collections.unmodifiableCollection(idToSegmentMap.values()); return Collections.unmodifiableCollection(idToSegmentMap.values());
} }
/**
* Removes segments for which the given filter returns true.
*/
public void removeSegmentsIf(Predicate<DataSegment> filter)
{
idToSegmentMap.values().removeIf(filter);
}
public DruidDataSource addSegment(DataSegment dataSegment) public DruidDataSource addSegment(DataSegment dataSegment)
{ {
idToSegmentMap.put(dataSegment.getId(), dataSegment); idToSegmentMap.put(dataSegment.getId(), dataSegment);
@ -99,7 +105,7 @@ public class DruidDataSource
* Returns the removed segment, or null if there was no segment with the given {@link SegmentId} in this * Returns the removed segment, or null if there was no segment with the given {@link SegmentId} in this
* DruidDataSource. * DruidDataSource.
*/ */
public DataSegment removeSegment(SegmentId segmentId) public @Nullable DataSegment removeSegment(SegmentId segmentId)
{ {
return idToSegmentMap.remove(segmentId); return idToSegmentMap.remove(segmentId);
} }
@ -126,7 +132,6 @@ public class DruidDataSource
@Override @Override
public boolean equals(Object o) public boolean equals(Object o)
{ {
//noinspection Contract
throw new UnsupportedOperationException("Use ImmutableDruidDataSource instead"); throw new UnsupportedOperationException("Use ImmutableDruidDataSource instead");
} }

View File

@ -232,7 +232,6 @@ public class DruidServer implements Comparable<DruidServer>
segmentId segmentId
); );
// Returning null from the lambda here makes the ConcurrentHashMap to not record any entry. // Returning null from the lambda here makes the ConcurrentHashMap to not record any entry.
//noinspection ReturnOfNull
return null; return null;
} }
DataSegment segment = dataSource.removeSegment(segmentId); DataSegment segment = dataSource.removeSegment(segmentId);
@ -244,7 +243,6 @@ public class DruidServer implements Comparable<DruidServer>
log.warn("Asked to remove data segment that doesn't exist!? server[%s], segment[%s]", getName(), 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. // Returning null from the lambda here makes the ConcurrentHashMap to remove the current entry.
//noinspection ReturnOfNull
return dataSource.isEmpty() ? null : dataSource; return dataSource.isEmpty() ? null : dataSource;
} }
); );

View File

@ -123,6 +123,8 @@ public class ImmutableDruidDataSource
@Override @Override
public boolean equals(Object o) 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) { if (this == o) {
return true; return true;
} }
@ -146,6 +148,8 @@ public class ImmutableDruidDataSource
@Override @Override
public int hashCode() 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);
} }
} }

View File

@ -42,19 +42,19 @@ public class ImmutableDruidServer
private final DruidServerMetadata metadata; private final DruidServerMetadata metadata;
private final long currSize; private final long currSize;
private final ImmutableMap<String, ImmutableDruidDataSource> dataSources; private final ImmutableMap<String, ImmutableDruidDataSource> dataSources;
private final int totalSegments; private final int numSegments;
public ImmutableDruidServer( public ImmutableDruidServer(
DruidServerMetadata metadata, DruidServerMetadata metadata,
long currSize, long currSize,
ImmutableMap<String, ImmutableDruidDataSource> dataSources, ImmutableMap<String, ImmutableDruidDataSource> dataSources,
int totalSegments int numSegments
) )
{ {
this.metadata = Preconditions.checkNotNull(metadata); this.metadata = Preconditions.checkNotNull(metadata);
this.currSize = currSize; this.currSize = currSize;
this.dataSources = dataSources; this.dataSources = dataSources;
this.totalSegments = totalSegments; this.numSegments = numSegments;
} }
public String getName() public String getName()
@ -128,23 +128,26 @@ public class ImmutableDruidServer
} }
/** /**
* Returns a lazy collection with all segments in all data sources, stored on this ImmutableDruidServer. The order * Returns a lazy collection with all segments in all data sources stored on this ImmutableDruidServer to be used for
* of segments in this collection is unspecified. * iteration or {@link Collection#stream()} transformation. The order of segments in this collection is unspecified.
*
* Calling {@link Collection#size()} on the returned collection is cheap, O(1).
* *
* Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an * 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 * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only
* once rather than several times. * once rather than several times.
*/ */
public Collection<DataSegment> getLazyAllSegments() public Collection<DataSegment> iterateAllSegments()
{ {
return CollectionUtils.createLazyCollectionFromStream( return CollectionUtils.createLazyCollectionFromStream(
() -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()), () -> dataSources.values().stream().flatMap(dataSource -> dataSource.getSegments().stream()),
totalSegments numSegments
); );
} }
public int getNumSegments()
{
return numSegments;
}
public String getURL() public String getURL()
{ {
if (metadata.getHostAndTlsPort() != null) { if (metadata.getHostAndTlsPort() != null) {

View File

@ -29,6 +29,10 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Objects; 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 public class ClientCompactQuery implements ClientQuery
{ {
private final String dataSource; private final String dataSource;
@ -136,7 +140,7 @@ public class ClientCompactQuery implements ClientQuery
@Override @Override
public String toString() public String toString()
{ {
return "ClientCompactQuery{" + return getClass().getSimpleName() + "{" +
"dataSource='" + dataSource + '\'' + "dataSource='" + dataSource + '\'' +
", segments=" + segments + ", segments=" + segments +
", interval=" + interval + ", interval=" + interval +

View File

@ -43,17 +43,17 @@ public class ClientCompactQueryTuningConfig
private final Long pushTimeout; private final Long pushTimeout;
public static ClientCompactQueryTuningConfig from( public static ClientCompactQueryTuningConfig from(
@Nullable UserCompactTuningConfig userCompactTuningConfig, @Nullable UserCompactTuningConfig userCompactionTaskQueryTuningConfig,
@Nullable Integer maxRowsPerSegment @Nullable Integer maxRowsPerSegment
) )
{ {
return new ClientCompactQueryTuningConfig( return new ClientCompactQueryTuningConfig(
maxRowsPerSegment, maxRowsPerSegment,
userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxRowsInMemory(), userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxRowsInMemory(),
userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxTotalRows(), userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxTotalRows(),
userCompactTuningConfig == null ? null : userCompactTuningConfig.getIndexSpec(), userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getIndexSpec(),
userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxPendingPersists(), userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxPendingPersists(),
userCompactTuningConfig == null ? null : userCompactTuningConfig.getPushTimeout() userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getPushTimeout()
); );
} }
@ -150,7 +150,7 @@ public class ClientCompactQueryTuningConfig
@Override @Override
public String toString() public String toString()
{ {
return "ClientCompactQueryTuningConfig{" + return getClass().getSimpleName() + "{" +
"maxRowsPerSegment=" + maxRowsPerSegment + "maxRowsPerSegment=" + maxRowsPerSegment +
", maxRowsInMemory=" + maxRowsInMemory + ", maxRowsInMemory=" + maxRowsInMemory +
", maxTotalRows=" + maxTotalRows + ", maxTotalRows=" + maxTotalRows +

View File

@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import org.joda.time.Interval; 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 public class ClientKillQuery implements ClientQuery
{ {

View File

@ -24,7 +24,12 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
import com.fasterxml.jackson.annotation.JsonTypeInfo; 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") @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = { @JsonSubTypes(value = {

View File

@ -24,6 +24,7 @@ import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -32,89 +33,105 @@ import java.util.List;
import java.util.Set; 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 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);
boolean enableSegment(String segmentId); int markAsUsedNonOvershadowedSegmentsInInterval(String dataSource, Interval interval);
int markAsUsedNonOvershadowedSegments(String dataSource, Set<String> 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<String> segmentIds); int markAsUnusedAllSegmentsInDataSource(String dataSource);
boolean removeDataSource(String dataSource); int markAsUnusedSegmentsInInterval(String dataSource, Interval interval);
int markSegmentsAsUnused(String dataSource, Set<String> 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<String> segmentIds);
int disableSegments(String dataSource, Interval interval);
boolean isStarted();
@Nullable
ImmutableDruidDataSource getDataSource(String dataSourceName);
/** /**
* Returns a collection of known datasources. * 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
* Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has * returns null.
* not yet been polled.)
*/ */
@Nullable @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSource);
Collection<ImmutableDruidDataSource> getDataSources();
/**
* 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<ImmutableDruidDataSource> getImmutableDataSourcesWithAllUsedSegments();
/**
* Returns a set of overshadowed segment ids.
*/
Set<SegmentId> getOvershadowedSegments();
/**
* Returns a snapshot of DruidDataSources and overshadowed segments
*/
DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments();
/** /**
* Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is * 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 * 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 * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than
* several times. * 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<DataSegment> iterateAllUsedSegments();
Iterable<DataSegment> iterateAllSegments();
Collection<String> getAllDataSourceNames();
/** /**
* 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 * Performance warning: this method makes a query into the database.
* not yet been polled.) *
* 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 Collection<String> retrieveAllDataSourceNames();
Set<SegmentId> getOvershadowedSegments();
/** /**
* Returns a snapshot of DruidDataSources and overshadowed segments * Returns top N unused segment intervals with the end time no later than the specified maxEndTime when ordered by
* * segment start time, end time.
*/ */
@Nullable List<Interval> getUnusedSegmentIntervals(String dataSource, DateTime maxEndTime, int limit);
DataSourcesSnapshot getDataSourcesSnapshot();
/**
* Returns top N unused segment intervals in given interval when ordered by segment start time, end time.
*/
List<Interval> getUnusedSegmentIntervals(String dataSource, Interval interval, int limit);
@VisibleForTesting @VisibleForTesting
void poll(); void poll();

View File

@ -33,4 +33,9 @@ public class MetadataSegmentManagerConfig
{ {
return pollDuration; return pollDuration;
} }
public void setPollDuration(Period pollDuration)
{
this.pollDuration = pollDuration;
}
} }

View File

@ -200,7 +200,7 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
{ {
try { try {
// poll() is synchronized together with start() and stop() to ensure that when stop() exits, poll() // 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 // immediately because the localStartedOrder doesn't match the new currentStartOrder). It's needed
// to avoid flakiness in SQLMetadataRuleManagerTest. // to avoid flakiness in SQLMetadataRuleManagerTest.
// See https://github.com/apache/incubator-druid/issues/6028 // See https://github.com/apache/incubator-druid/issues/6028

View File

@ -19,13 +19,23 @@
package org.apache.druid.metadata; 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<String> unknownSegmentIds;
UnknownSegmentIdException(Collection<String> segmentIds)
{ {
super(message); super("Cannot find segment ids " + segmentIds);
this.unknownSegmentIds = segmentIds;
}
public Collection<String> getUnknownSegmentIds()
{
return unknownSegmentIds;
} }
} }

View File

@ -19,10 +19,16 @@
package org.apache.druid.server; package org.apache.druid.server;
import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import javax.ws.rs.core.MultivaluedMap;
import javax.ws.rs.core.UriInfo;
public class JettyUtils public class JettyUtils
{ {
private static final Logger log = new Logger(JettyUtils.class);
/** /**
* Concatenate URI parts, in a way that is useful for proxy servlets. * Concatenate URI parts, in a way that is useful for proxy servlets.
* *
@ -46,4 +52,38 @@ public class JettyUtils
return url.toString(); 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<String, String> 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;
}
} }

View File

@ -48,7 +48,7 @@ public class CoordinatorDynamicConfig
{ {
public static final String CONFIG_KEY = "coordinator.config"; public static final String CONFIG_KEY = "coordinator.config";
private final long millisToWaitBeforeDeleting; private final long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
private final long mergeBytesLimit; private final long mergeBytesLimit;
private final int mergeSegmentsLimit; private final int mergeSegmentsLimit;
private final int maxSegmentsToMove; private final int maxSegmentsToMove;
@ -56,13 +56,29 @@ public class CoordinatorDynamicConfig
private final int replicationThrottleLimit; private final int replicationThrottleLimit;
private final int balancerComputeThreads; private final int balancerComputeThreads;
private final boolean emitBalancingStats; private final boolean emitBalancingStats;
private final boolean killAllDataSources;
private final Set<String> 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<String> specificDataSourcesToKillUnusedSegmentsIn;
private final Set<String> decommissioningNodes; private final Set<String> decommissioningNodes;
private final int decommissioningMaxPercentOfMaxSegmentsToMove; private final int decommissioningMaxPercentOfMaxSegmentsToMove;
// The pending segments of the dataSources in this list are not killed. /**
private final Set<String> 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<String> dataSourcesToNotKillStalePendingSegmentsIn;
/** /**
* The maximum number of segments that could be queued for loading to any given server. * The maximum number of segments that could be queued for loading to any given server.
@ -74,7 +90,10 @@ public class CoordinatorDynamicConfig
@JsonCreator @JsonCreator
public CoordinatorDynamicConfig( 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("mergeBytesLimit") long mergeBytesLimit,
@JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit, @JsonProperty("mergeSegmentsLimit") int mergeSegmentsLimit,
@JsonProperty("maxSegmentsToMove") int maxSegmentsToMove, @JsonProperty("maxSegmentsToMove") int maxSegmentsToMove,
@ -82,19 +101,26 @@ public class CoordinatorDynamicConfig
@JsonProperty("replicationThrottleLimit") int replicationThrottleLimit, @JsonProperty("replicationThrottleLimit") int replicationThrottleLimit,
@JsonProperty("balancerComputeThreads") int balancerComputeThreads, @JsonProperty("balancerComputeThreads") int balancerComputeThreads,
@JsonProperty("emitBalancingStats") boolean emitBalancingStats, @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
// Type is Object here so that we can support both string and list as // strings in the update request. See https://github.com/apache/incubator-druid/issues/3055.
// coordinator console can not send array of strings in the update request. // Keeping the legacy 'killDataSourceWhitelist' property name for backward compatibility. When the project is
// See https://github.com/apache/incubator-druid/issues/3055 // updated to Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152
@JsonProperty("killDataSourceWhitelist") Object killableDataSources, @JsonProperty("killDataSourceWhitelist") Object specificDataSourcesToKillUnusedSegmentsIn,
@JsonProperty("killAllDataSources") boolean killAllDataSources, // Keeping the legacy 'killAllDataSources' property name for backward compatibility. When the project is
@JsonProperty("killPendingSegmentsSkipList") Object protectedPendingSegmentDatasources, // 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("maxSegmentsInNodeLoadingQueue") int maxSegmentsInNodeLoadingQueue,
@JsonProperty("decommissioningNodes") Object decommissioningNodes, @JsonProperty("decommissioningNodes") Object decommissioningNodes,
@JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") int decommissioningMaxPercentOfMaxSegmentsToMove @JsonProperty("decommissioningMaxPercentOfMaxSegmentsToMove") int decommissioningMaxPercentOfMaxSegmentsToMove
) )
{ {
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments =
leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
this.mergeBytesLimit = mergeBytesLimit; this.mergeBytesLimit = mergeBytesLimit;
this.mergeSegmentsLimit = mergeSegmentsLimit; this.mergeSegmentsLimit = mergeSegmentsLimit;
this.maxSegmentsToMove = maxSegmentsToMove; this.maxSegmentsToMove = maxSegmentsToMove;
@ -102,9 +128,10 @@ public class CoordinatorDynamicConfig
this.replicationThrottleLimit = replicationThrottleLimit; this.replicationThrottleLimit = replicationThrottleLimit;
this.balancerComputeThreads = Math.max(balancerComputeThreads, 1); this.balancerComputeThreads = Math.max(balancerComputeThreads, 1);
this.emitBalancingStats = emitBalancingStats; this.emitBalancingStats = emitBalancingStats;
this.killAllDataSources = killAllDataSources; this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources;
this.killableDataSources = parseJsonStringOrArray(killableDataSources); this.specificDataSourcesToKillUnusedSegmentsIn = parseJsonStringOrArray(specificDataSourcesToKillUnusedSegmentsIn);
this.protectedPendingSegmentDatasources = parseJsonStringOrArray(protectedPendingSegmentDatasources); this.dataSourcesToNotKillStalePendingSegmentsIn =
parseJsonStringOrArray(dataSourcesToNotKillStalePendingSegmentsIn);
this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue;
this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes); this.decommissioningNodes = parseJsonStringOrArray(decommissioningNodes);
Preconditions.checkArgument( Preconditions.checkArgument(
@ -113,8 +140,10 @@ public class CoordinatorDynamicConfig
); );
this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove;
if (this.killAllDataSources && !this.killableDataSources.isEmpty()) { if (this.killUnusedSegmentsInAllDataSources && !this.specificDataSourcesToKillUnusedSegmentsIn.isEmpty()) {
throw new IAE("can't have killAllDataSources and non-empty killDataSourceWhitelist"); throw new IAE(
"can't have killUnusedSegmentsInAllDataSources and non-empty specificDataSourcesToKillUnusedSegmentsIn"
);
} }
} }
@ -152,10 +181,10 @@ public class CoordinatorDynamicConfig
return Preconditions.checkNotNull(watch(configManager).get(), "Got null config from watcher?!"); return Preconditions.checkNotNull(watch(configManager).get(), "Got null config from watcher?!");
} }
@JsonProperty @JsonProperty("millisToWaitBeforeDeleting")
public long getMillisToWaitBeforeDeleting() public long getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
{ {
return millisToWaitBeforeDeleting; return leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
} }
@JsonProperty @JsonProperty
@ -200,30 +229,22 @@ public class CoordinatorDynamicConfig
return balancerComputeThreads; return balancerComputeThreads;
} }
/**
* List of dataSources for which kill tasks are sent in
* {@link org.apache.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller}.
*/
@JsonProperty("killDataSourceWhitelist") @JsonProperty("killDataSourceWhitelist")
public Set<String> getKillableDataSources() public Set<String> getSpecificDataSourcesToKillUnusedSegmentsIn()
{ {
return killableDataSources; return specificDataSourcesToKillUnusedSegmentsIn;
} }
@JsonProperty @JsonProperty("killAllDataSources")
public boolean isKillAllDataSources() public boolean isKillUnusedSegmentsInAllDataSources()
{ {
return killAllDataSources; return killUnusedSegmentsInAllDataSources;
} }
/** @JsonProperty("killPendingSegmentsSkipList")
* List of dataSources for which pendingSegments are NOT cleaned up public Set<String> getDataSourcesToNotKillStalePendingSegmentsIn()
* in {@link DruidCoordinatorCleanupPendingSegments}.
*/
@JsonProperty
public Set<String> getProtectedPendingSegmentDatasources()
{ {
return protectedPendingSegmentDatasources; return dataSourcesToNotKillStalePendingSegmentsIn;
} }
@JsonProperty @JsonProperty
@ -233,9 +254,9 @@ public class CoordinatorDynamicConfig
} }
/** /**
* List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning' servers, * List of historical servers to 'decommission'. Coordinator will not assign new segments to 'decommissioning'
* and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate specified by * servers, and segments will be moved away from them to be placed on non-decommissioning servers at the maximum rate
* {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove}. * specified by {@link CoordinatorDynamicConfig#getDecommissioningMaxPercentOfMaxSegmentsToMove}.
* *
* @return list of host:port entries * @return list of host:port entries
*/ */
@ -270,7 +291,8 @@ public class CoordinatorDynamicConfig
public String toString() public String toString()
{ {
return "CoordinatorDynamicConfig{" + return "CoordinatorDynamicConfig{" +
"millisToWaitBeforeDeleting=" + millisToWaitBeforeDeleting + "leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments="
+ leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments +
", mergeBytesLimit=" + mergeBytesLimit + ", mergeBytesLimit=" + mergeBytesLimit +
", mergeSegmentsLimit=" + mergeSegmentsLimit + ", mergeSegmentsLimit=" + mergeSegmentsLimit +
", maxSegmentsToMove=" + maxSegmentsToMove + ", maxSegmentsToMove=" + maxSegmentsToMove +
@ -278,9 +300,9 @@ public class CoordinatorDynamicConfig
", replicationThrottleLimit=" + replicationThrottleLimit + ", replicationThrottleLimit=" + replicationThrottleLimit +
", balancerComputeThreads=" + balancerComputeThreads + ", balancerComputeThreads=" + balancerComputeThreads +
", emitBalancingStats=" + emitBalancingStats + ", emitBalancingStats=" + emitBalancingStats +
", killAllDataSources=" + killAllDataSources + ", killUnusedSegmentsInAllDataSources=" + killUnusedSegmentsInAllDataSources +
", killDataSourceWhitelist=" + killableDataSources + ", specificDataSourcesToKillUnusedSegmentsIn=" + specificDataSourcesToKillUnusedSegmentsIn +
", protectedPendingSegmentDatasources=" + protectedPendingSegmentDatasources + ", dataSourcesToNotKillStalePendingSegmentsIn=" + dataSourcesToNotKillStalePendingSegmentsIn +
", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue + ", maxSegmentsInNodeLoadingQueue=" + maxSegmentsInNodeLoadingQueue +
", decommissioningNodes=" + decommissioningNodes + ", decommissioningNodes=" + decommissioningNodes +
", decommissioningMaxPercentOfMaxSegmentsToMove=" + decommissioningMaxPercentOfMaxSegmentsToMove + ", decommissioningMaxPercentOfMaxSegmentsToMove=" + decommissioningMaxPercentOfMaxSegmentsToMove +
@ -299,7 +321,8 @@ public class CoordinatorDynamicConfig
CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o; CoordinatorDynamicConfig that = (CoordinatorDynamicConfig) o;
if (millisToWaitBeforeDeleting != that.millisToWaitBeforeDeleting) { if (leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments !=
that.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments) {
return false; return false;
} }
if (mergeBytesLimit != that.mergeBytesLimit) { if (mergeBytesLimit != that.mergeBytesLimit) {
@ -323,16 +346,16 @@ public class CoordinatorDynamicConfig
if (emitBalancingStats != that.emitBalancingStats) { if (emitBalancingStats != that.emitBalancingStats) {
return false; return false;
} }
if (killAllDataSources != that.killAllDataSources) { if (killUnusedSegmentsInAllDataSources != that.killUnusedSegmentsInAllDataSources) {
return false; return false;
} }
if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) { if (maxSegmentsInNodeLoadingQueue != that.maxSegmentsInNodeLoadingQueue) {
return false; return false;
} }
if (!Objects.equals(killableDataSources, that.killableDataSources)) { if (!Objects.equals(specificDataSourcesToKillUnusedSegmentsIn, that.specificDataSourcesToKillUnusedSegmentsIn)) {
return false; return false;
} }
if (!Objects.equals(protectedPendingSegmentDatasources, that.protectedPendingSegmentDatasources)) { if (!Objects.equals(dataSourcesToNotKillStalePendingSegmentsIn, that.dataSourcesToNotKillStalePendingSegmentsIn)) {
return false; return false;
} }
if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) { if (!Objects.equals(decommissioningNodes, that.decommissioningNodes)) {
@ -345,7 +368,7 @@ public class CoordinatorDynamicConfig
public int hashCode() public int hashCode()
{ {
return Objects.hash( return Objects.hash(
millisToWaitBeforeDeleting, leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
mergeBytesLimit, mergeBytesLimit,
mergeSegmentsLimit, mergeSegmentsLimit,
maxSegmentsToMove, maxSegmentsToMove,
@ -353,10 +376,10 @@ public class CoordinatorDynamicConfig
replicationThrottleLimit, replicationThrottleLimit,
balancerComputeThreads, balancerComputeThreads,
emitBalancingStats, emitBalancingStats,
killAllDataSources, killUnusedSegmentsInAllDataSources,
maxSegmentsInNodeLoadingQueue, maxSegmentsInNodeLoadingQueue,
killableDataSources, specificDataSourcesToKillUnusedSegmentsIn,
protectedPendingSegmentDatasources, dataSourcesToNotKillStalePendingSegmentsIn,
decommissioningNodes, decommissioningNodes,
decommissioningMaxPercentOfMaxSegmentsToMove decommissioningMaxPercentOfMaxSegmentsToMove
); );
@ -369,19 +392,20 @@ public class CoordinatorDynamicConfig
public static class Builder public static class Builder
{ {
private static final long DEFAULT_MILLIS_TO_WAIT_BEFORE_DELETING = TimeUnit.MINUTES.toMillis(15); private static final long DEFAULT_LEADING_TIME_MILLIS_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS =
private static final long DEFAULT_MERGE_BYTES_LIMIT = 524288000L; 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_MERGE_SEGMENTS_LIMIT = 100;
private static final int DEFAULT_MAX_SEGMENTS_TO_MOVE = 5; private static final int DEFAULT_MAX_SEGMENTS_TO_MOVE = 5;
private static final int DEFAULT_REPLICANT_LIFETIME = 15; private static final int DEFAULT_REPLICANT_LIFETIME = 15;
private static final int DEFAULT_REPLICATION_THROTTLE_LIMIT = 10; private static final int DEFAULT_REPLICATION_THROTTLE_LIMIT = 10;
private static final int DEFAULT_BALANCER_COMPUTE_THREADS = 1; private static final int DEFAULT_BALANCER_COMPUTE_THREADS = 1;
private static final boolean DEFAULT_EMIT_BALANCING_STATS = false; 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_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE = 0;
private static final int DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70; private static final int DEFAULT_DECOMMISSIONING_MAX_SEGMENTS_TO_MOVE_PERCENT = 70;
private Long millisToWaitBeforeDeleting; private Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
private Long mergeBytesLimit; private Long mergeBytesLimit;
private Integer mergeSegmentsLimit; private Integer mergeSegmentsLimit;
private Integer maxSegmentsToMove; private Integer maxSegmentsToMove;
@ -389,9 +413,9 @@ public class CoordinatorDynamicConfig
private Integer replicationThrottleLimit; private Integer replicationThrottleLimit;
private Boolean emitBalancingStats; private Boolean emitBalancingStats;
private Integer balancerComputeThreads; private Integer balancerComputeThreads;
private Object killableDataSources; private Object specificDataSourcesToKillUnusedSegmentsIn;
private Boolean killAllDataSources; private Boolean killUnusedSegmentsInAllDataSources;
private Object killPendingSegmentsSkipList; private Object dataSourcesToNotKillStalePendingSegmentsIn;
private Integer maxSegmentsInNodeLoadingQueue; private Integer maxSegmentsInNodeLoadingQueue;
private Object decommissioningNodes; private Object decommissioningNodes;
private Integer decommissioningMaxPercentOfMaxSegmentsToMove; private Integer decommissioningMaxPercentOfMaxSegmentsToMove;
@ -402,7 +426,8 @@ public class CoordinatorDynamicConfig
@JsonCreator @JsonCreator
public Builder( public Builder(
@JsonProperty("millisToWaitBeforeDeleting") @Nullable Long millisToWaitBeforeDeleting, @JsonProperty("millisToWaitBeforeDeleting")
@Nullable Long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
@JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit, @JsonProperty("mergeBytesLimit") @Nullable Long mergeBytesLimit,
@JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit, @JsonProperty("mergeSegmentsLimit") @Nullable Integer mergeSegmentsLimit,
@JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove, @JsonProperty("maxSegmentsToMove") @Nullable Integer maxSegmentsToMove,
@ -410,15 +435,17 @@ public class CoordinatorDynamicConfig
@JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit, @JsonProperty("replicationThrottleLimit") @Nullable Integer replicationThrottleLimit,
@JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads, @JsonProperty("balancerComputeThreads") @Nullable Integer balancerComputeThreads,
@JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats, @JsonProperty("emitBalancingStats") @Nullable Boolean emitBalancingStats,
@JsonProperty("killDataSourceWhitelist") @Nullable Object killableDataSources, @JsonProperty("killDataSourceWhitelist") @Nullable Object specificDataSourcesToKillUnusedSegmentsIn,
@JsonProperty("killAllDataSources") @Nullable Boolean killAllDataSources, @JsonProperty("killAllDataSources") @Nullable Boolean killUnusedSegmentsInAllDataSources,
@JsonProperty("killPendingSegmentsSkipList") @Nullable Object killPendingSegmentsSkipList, @JsonProperty("killPendingSegmentsSkipList") @Nullable Object dataSourcesToNotKillStalePendingSegmentsIn,
@JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue, @JsonProperty("maxSegmentsInNodeLoadingQueue") @Nullable Integer maxSegmentsInNodeLoadingQueue,
@JsonProperty("decommissioningNodes") @Nullable Object decommissioningNodes, @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.mergeBytesLimit = mergeBytesLimit;
this.mergeSegmentsLimit = mergeSegmentsLimit; this.mergeSegmentsLimit = mergeSegmentsLimit;
this.maxSegmentsToMove = maxSegmentsToMove; this.maxSegmentsToMove = maxSegmentsToMove;
@ -426,17 +453,17 @@ public class CoordinatorDynamicConfig
this.replicationThrottleLimit = replicationThrottleLimit; this.replicationThrottleLimit = replicationThrottleLimit;
this.balancerComputeThreads = balancerComputeThreads; this.balancerComputeThreads = balancerComputeThreads;
this.emitBalancingStats = emitBalancingStats; this.emitBalancingStats = emitBalancingStats;
this.killAllDataSources = killAllDataSources; this.specificDataSourcesToKillUnusedSegmentsIn = specificDataSourcesToKillUnusedSegmentsIn;
this.killableDataSources = killableDataSources; this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources;
this.killPendingSegmentsSkipList = killPendingSegmentsSkipList; this.dataSourcesToNotKillStalePendingSegmentsIn = dataSourcesToNotKillStalePendingSegmentsIn;
this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue; this.maxSegmentsInNodeLoadingQueue = maxSegmentsInNodeLoadingQueue;
this.decommissioningNodes = decommissioningNodes; this.decommissioningNodes = decommissioningNodes;
this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove; this.decommissioningMaxPercentOfMaxSegmentsToMove = decommissioningMaxPercentOfMaxSegmentsToMove;
} }
public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting) public Builder withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(long leadingTimeMillis)
{ {
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting; this.leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments = leadingTimeMillis;
return this; return this;
} }
@ -482,15 +509,15 @@ public class CoordinatorDynamicConfig
return this; return this;
} }
public Builder withKillDataSourceWhitelist(Set<String> killDataSourceWhitelist) public Builder withSpecificDataSourcesToKillUnusedSegmentsIn(Set<String> dataSources)
{ {
this.killableDataSources = killDataSourceWhitelist; this.specificDataSourcesToKillUnusedSegmentsIn = dataSources;
return this; return this;
} }
public Builder withKillAllDataSources(boolean killAllDataSources) public Builder withKillUnusedSegmentsInAllDataSources(boolean killUnusedSegmentsInAllDataSources)
{ {
this.killAllDataSources = killAllDataSources; this.killUnusedSegmentsInAllDataSources = killUnusedSegmentsInAllDataSources;
return this; return this;
} }
@ -515,7 +542,9 @@ public class CoordinatorDynamicConfig
public CoordinatorDynamicConfig build() public CoordinatorDynamicConfig build()
{ {
return new CoordinatorDynamicConfig( 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, mergeBytesLimit == null ? DEFAULT_MERGE_BYTES_LIMIT : mergeBytesLimit,
mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit, mergeSegmentsLimit == null ? DEFAULT_MERGE_SEGMENTS_LIMIT : mergeSegmentsLimit,
maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove, maxSegmentsToMove == null ? DEFAULT_MAX_SEGMENTS_TO_MOVE : maxSegmentsToMove,
@ -523,9 +552,11 @@ public class CoordinatorDynamicConfig
replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit, replicationThrottleLimit == null ? DEFAULT_REPLICATION_THROTTLE_LIMIT : replicationThrottleLimit,
balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads, balancerComputeThreads == null ? DEFAULT_BALANCER_COMPUTE_THREADS : balancerComputeThreads,
emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats, emitBalancingStats == null ? DEFAULT_EMIT_BALANCING_STATS : emitBalancingStats,
killableDataSources, specificDataSourcesToKillUnusedSegmentsIn,
killAllDataSources == null ? DEFAULT_KILL_ALL_DATA_SOURCES : killAllDataSources, killUnusedSegmentsInAllDataSources == null
killPendingSegmentsSkipList, ? DEFAULT_KILL_UNUSED_SEGMENTS_IN_ALL_DATA_SOURCES
: killUnusedSegmentsInAllDataSources,
dataSourcesToNotKillStalePendingSegmentsIn,
maxSegmentsInNodeLoadingQueue == null maxSegmentsInNodeLoadingQueue == null
? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE ? DEFAULT_MAX_SEGMENTS_IN_NODE_LOADING_QUEUE
: maxSegmentsInNodeLoadingQueue, : maxSegmentsInNodeLoadingQueue,
@ -539,7 +570,9 @@ public class CoordinatorDynamicConfig
public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults) public CoordinatorDynamicConfig build(CoordinatorDynamicConfig defaults)
{ {
return new CoordinatorDynamicConfig( return new CoordinatorDynamicConfig(
millisToWaitBeforeDeleting == null ? defaults.getMillisToWaitBeforeDeleting() : millisToWaitBeforeDeleting, leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments == null
? defaults.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
: leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit, mergeBytesLimit == null ? defaults.getMergeBytesLimit() : mergeBytesLimit,
mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit, mergeSegmentsLimit == null ? defaults.getMergeSegmentsLimit() : mergeSegmentsLimit,
maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove, maxSegmentsToMove == null ? defaults.getMaxSegmentsToMove() : maxSegmentsToMove,
@ -547,11 +580,15 @@ public class CoordinatorDynamicConfig
replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit, replicationThrottleLimit == null ? defaults.getReplicationThrottleLimit() : replicationThrottleLimit,
balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads, balancerComputeThreads == null ? defaults.getBalancerComputeThreads() : balancerComputeThreads,
emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats, emitBalancingStats == null ? defaults.emitBalancingStats() : emitBalancingStats,
killableDataSources == null ? defaults.getKillableDataSources() : killableDataSources, specificDataSourcesToKillUnusedSegmentsIn == null
killAllDataSources == null ? defaults.isKillAllDataSources() : killAllDataSources, ? defaults.getSpecificDataSourcesToKillUnusedSegmentsIn()
killPendingSegmentsSkipList == null : specificDataSourcesToKillUnusedSegmentsIn,
? defaults.getProtectedPendingSegmentDatasources() killUnusedSegmentsInAllDataSources == null
: killPendingSegmentsSkipList, ? defaults.isKillUnusedSegmentsInAllDataSources()
: killUnusedSegmentsInAllDataSources,
dataSourcesToNotKillStalePendingSegmentsIn == null
? defaults.getDataSourcesToNotKillStalePendingSegmentsIn()
: dataSourcesToNotKillStalePendingSegmentsIn,
maxSegmentsInNodeLoadingQueue == null maxSegmentsInNodeLoadingQueue == null
? defaults.getMaxSegmentsInNodeLoadingQueue() ? defaults.getMaxSegmentsInNodeLoadingQueue()
: maxSegmentsInNodeLoadingQueue, : maxSegmentsInNodeLoadingQueue,

View File

@ -117,7 +117,7 @@ public class CostBalancerStrategy implements BalancerStrategy
// since x_0 <= y_0, Y must overlap X if y_0 < x_1 // since x_0 <= y_0, Y must overlap X if y_0 < x_1
if (y0 < x1) { if (y0 < x1) {
/** /*
* We have two possible cases of overlap: * We have two possible cases of overlap:
* *
* X = [ A )[ B )[ C ) or [ A )[ B ) * X = [ A )[ B )[ C ) or [ A )[ B )
@ -151,7 +151,7 @@ public class CostBalancerStrategy implements BalancerStrategy
intervalCost(beta, beta, gamma) + // cost(B, C) intervalCost(beta, beta, gamma) + // cost(B, C)
2 * (beta + FastMath.exp(-beta) - 1); // cost(B, B) 2 * (beta + FastMath.exp(-beta) - 1); // cost(B, B)
} else { } else {
/** /*
* In the case where there is no overlap: * In the case where there is no overlap:
* *
* Given that x_0 <= y_0, * Given that x_0 <= y_0,
@ -258,9 +258,14 @@ public class CostBalancerStrategy implements BalancerStrategy
{ {
double cost = 0; double cost = 0;
for (ServerHolder server : serverHolders) { for (ServerHolder server : serverHolders) {
Iterable<DataSegment> segments = server.getServer().getLazyAllSegments(); // segments are dumped into an array because it's probably better than iterating the iterateAllSegments() result
for (DataSegment s : segments) { // quadratically in a loop, which can generate garbage in the form of Stream, Spliterator, Iterator, etc. objects
cost += computeJointSegmentsCost(s, segments); // 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; return cost;
@ -280,7 +285,7 @@ public class CostBalancerStrategy implements BalancerStrategy
{ {
double cost = 0; double cost = 0;
for (ServerHolder server : serverHolders) { for (ServerHolder server : serverHolders) {
for (DataSegment segment : server.getServer().getLazyAllSegments()) { for (DataSegment segment : server.getServer().iterateAllSegments()) {
cost += computeJointSegmentsCost(segment, segment); cost += computeJointSegmentsCost(segment, segment);
} }
} }
@ -288,10 +293,7 @@ public class CostBalancerStrategy implements BalancerStrategy
} }
@Override @Override
public void emitStats( public void emitStats(String tier, CoordinatorStats stats, List<ServerHolder> serverHolderList)
String tier,
CoordinatorStats stats, List<ServerHolder> serverHolderList
)
{ {
final double initialTotalCost = calculateInitialTotalCost(serverHolderList); final double initialTotalCost = calculateInitialTotalCost(serverHolderList);
final double normalization = calculateNormalization(serverHolderList); final double normalization = calculateNormalization(serverHolderList);
@ -334,7 +336,7 @@ public class CostBalancerStrategy implements BalancerStrategy
// the sum of the costs of other (exclusive of the proposalSegment) segments on the server // the sum of the costs of other (exclusive of the proposalSegment) segments on the server
cost += computeJointSegmentsCost( cost += computeJointSegmentsCost(
proposalSegment, 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 // plus the costs of segments that will be loaded

View File

@ -47,8 +47,8 @@ public class DiskNormalizedCostBalancerStrategy extends CostBalancerStrategy
} }
int nSegments = 1; int nSegments = 1;
if (server.getServer().getLazyAllSegments().size() > 0) { if (server.getServer().getNumSegments() > 0) {
nSegments = server.getServer().getLazyAllSegments().size(); nSegments = server.getServer().getNumSegments();
} }
double normalizedCost = cost / nSegments; double normalizedCost = cost / nSegments;

View File

@ -22,11 +22,13 @@ package org.apache.druid.server.coordinator;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.IAE;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
@ -35,7 +37,6 @@ import java.util.NavigableSet;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
/** /**
* Contains a representation of the current state of the cluster by tier. * Contains a representation of the current state of the cluster by tier.
@ -43,6 +44,16 @@ import java.util.stream.StreamSupport;
*/ */
public class DruidCluster public class DruidCluster
{ {
/** This static factory method must be called only from inside DruidClusterBuilder in tests. */
@VisibleForTesting
static DruidCluster createDruidClusterFromBuilderInTest(
@Nullable Set<ServerHolder> realtimes,
Map<String, Iterable<ServerHolder>> historicals
)
{
return new DruidCluster(realtimes, historicals);
}
private final Set<ServerHolder> realtimes; private final Set<ServerHolder> realtimes;
private final Map<String, NavigableSet<ServerHolder>> historicals; private final Map<String, NavigableSet<ServerHolder>> historicals;
@ -52,8 +63,7 @@ public class DruidCluster
this.historicals = new HashMap<>(); this.historicals = new HashMap<>();
} }
@VisibleForTesting private DruidCluster(
public DruidCluster(
@Nullable Set<ServerHolder> realtimes, @Nullable Set<ServerHolder> realtimes,
Map<String, Iterable<ServerHolder>> historicals Map<String, Iterable<ServerHolder>> historicals
) )
@ -62,12 +72,13 @@ public class DruidCluster
this.historicals = historicals this.historicals = historicals
.entrySet() .entrySet()
.stream() .stream()
.collect(Collectors.toMap( .collect(
Map.Entry::getKey, Collectors.toMap(
e -> StreamSupport Map.Entry::getKey,
.stream(e.getValue().spliterator(), false) (Map.Entry<String, Iterable<ServerHolder>> e) ->
.collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))) CollectionUtils.newTreeSet(Comparator.reverseOrder(), e.getValue())
)); )
);
} }
public void add(ServerHolder serverHolder) public void add(ServerHolder serverHolder)

View File

@ -19,13 +19,15 @@
package org.apache.druid.server.coordinator; package org.apache.druid.server.coordinator;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.Ordering; import com.google.common.collect.Ordering;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import com.google.inject.Inject; 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.Object2LongMap;
import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFramework;
@ -76,7 +78,6 @@ import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Duration; import org.joda.time.Duration;
import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection; import java.util.Collection;
@ -84,7 +85,6 @@ import java.util.Comparator;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Callable; import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap; 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 used in historical nodes' {@link LoadQueuePeon}s to make historicals load more recent segment first.
* *
* It is also used in {@link DruidCoordinatorRuntimeParams} for {@link * 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 * 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 * 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 * 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 DruidCoordinatorConfig config;
private final ZkPathsConfig zkPaths; private final ZkPathsConfig zkPaths;
private final JacksonConfigManager configManager; private final JacksonConfigManager configManager;
private final MetadataSegmentManager metadataSegmentManager; private final MetadataSegmentManager segmentsMetadata;
private final ServerInventoryView serverInventoryView; private final ServerInventoryView serverInventoryView;
private final MetadataRuleManager metadataRuleManager; private final MetadataRuleManager metadataRuleManager;
private final CuratorFramework curator; private final CuratorFramework curator;
@ -148,17 +148,13 @@ public class DruidCoordinator
private volatile boolean started = false; private volatile boolean started = false;
private volatile SegmentReplicantLookup segmentReplicantLookup = null; private volatile SegmentReplicantLookup segmentReplicantLookup = null;
/**
* set in {@link CoordinatorRunnable#run()} at start of every coordinator run
*/
private volatile DataSourcesSnapshot dataSourcesSnapshot = null;
@Inject @Inject
public DruidCoordinator( public DruidCoordinator(
DruidCoordinatorConfig config, DruidCoordinatorConfig config,
ZkPathsConfig zkPaths, ZkPathsConfig zkPaths,
JacksonConfigManager configManager, JacksonConfigManager configManager,
MetadataSegmentManager metadataSegmentManager, MetadataSegmentManager segmentsMetadata,
ServerInventoryView serverInventoryView, ServerInventoryView serverInventoryView,
MetadataRuleManager metadataRuleManager, MetadataRuleManager metadataRuleManager,
CuratorFramework curator, CuratorFramework curator,
@ -178,7 +174,7 @@ public class DruidCoordinator
config, config,
zkPaths, zkPaths,
configManager, configManager,
metadataSegmentManager, segmentsMetadata,
serverInventoryView, serverInventoryView,
metadataRuleManager, metadataRuleManager,
curator, curator,
@ -200,7 +196,7 @@ public class DruidCoordinator
DruidCoordinatorConfig config, DruidCoordinatorConfig config,
ZkPathsConfig zkPaths, ZkPathsConfig zkPaths,
JacksonConfigManager configManager, JacksonConfigManager configManager,
MetadataSegmentManager metadataSegmentManager, MetadataSegmentManager segmentsMetadata,
ServerInventoryView serverInventoryView, ServerInventoryView serverInventoryView,
MetadataRuleManager metadataRuleManager, MetadataRuleManager metadataRuleManager,
CuratorFramework curator, CuratorFramework curator,
@ -221,7 +217,7 @@ public class DruidCoordinator
this.zkPaths = zkPaths; this.zkPaths = zkPaths;
this.configManager = configManager; this.configManager = configManager;
this.metadataSegmentManager = metadataSegmentManager; this.segmentsMetadata = segmentsMetadata;
this.serverInventoryView = serverInventoryView; this.serverInventoryView = serverInventoryView;
this.metadataRuleManager = metadataRuleManager; this.metadataRuleManager = metadataRuleManager;
this.curator = curator; this.curator = curator;
@ -263,11 +259,7 @@ public class DruidCoordinator
return underReplicationCountsPerDataSourcePerTier; return underReplicationCountsPerDataSourcePerTier;
} }
final Iterable<DataSegment> dataSegments = iterateAvailableDataSegments(); final Iterable<DataSegment> dataSegments = segmentsMetadata.iterateAllUsedSegments();
if (dataSegments == null) {
return underReplicationCountsPerDataSourcePerTier;
}
final DateTime now = DateTimes.nowUtc(); final DateTime now = DateTimes.nowUtc();
@ -295,45 +287,36 @@ public class DruidCoordinator
return underReplicationCountsPerDataSourcePerTier; return underReplicationCountsPerDataSourcePerTier;
} }
public Object2LongMap<String> getSegmentAvailability() public Object2IntMap<String> computeNumsUnavailableUsedSegmentsPerDataSource()
{ {
final Object2LongOpenHashMap<String> retVal = new Object2LongOpenHashMap<>();
if (segmentReplicantLookup == null) { if (segmentReplicantLookup == null) {
return retVal; return Object2IntMaps.emptyMap();
} }
final Iterable<DataSegment> dataSegments = iterateAvailableDataSegments(); final Object2IntOpenHashMap<String> numsUnavailableUsedSegmentsPerDataSource = new Object2IntOpenHashMap<>();
if (dataSegments == null) { final Iterable<DataSegment> dataSegments = segmentsMetadata.iterateAllUsedSegments();
return retVal;
}
for (DataSegment segment : dataSegments) { for (DataSegment segment : dataSegments) {
if (segmentReplicantLookup.getLoadedReplicants(segment.getId()) == 0) { if (segmentReplicantLookup.getLoadedReplicants(segment.getId()) == 0) {
retVal.addTo(segment.getDataSource(), 1); numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 1);
} else { } else {
retVal.addTo(segment.getDataSource(), 0); numsUnavailableUsedSegmentsPerDataSource.addTo(segment.getDataSource(), 0);
} }
} }
return retVal; return numsUnavailableUsedSegmentsPerDataSource;
} }
public Map<String, Double> getLoadStatus() public Map<String, Double> getLoadStatus()
{ {
final Map<String, Double> loadStatus = new HashMap<>(); final Map<String, Double> loadStatus = new HashMap<>();
final Collection<ImmutableDruidDataSource> dataSources = Optional.ofNullable(dataSourcesSnapshot) final Collection<ImmutableDruidDataSource> dataSources =
.map(m -> m.getDataSources()) segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments();
.orElse(null);
if (dataSources == null) {
return loadStatus;
}
for (ImmutableDruidDataSource dataSource : dataSources) { for (ImmutableDruidDataSource dataSource : dataSources) {
final Set<DataSegment> segments = Sets.newHashSet(dataSource.getSegments()); final Set<DataSegment> segments = Sets.newHashSet(dataSource.getSegments());
final int availableSegmentSize = segments.size(); final int numUsedSegments = segments.size();
// remove loaded segments // remove loaded segments
for (DruidServer druidServer : serverInventoryView.getInventory()) { for (DruidServer druidServer : serverInventoryView.getInventory()) {
@ -346,10 +329,10 @@ public class DruidCoordinator
} }
} }
} }
final int unloadedSegmentSize = segments.size(); final int numUnloadedSegments = segments.size();
loadStatus.put( loadStatus.put(
dataSource.getName(), dataSource.getName(),
100 * ((double) (availableSegmentSize - unloadedSegmentSize) / (double) availableSegmentSize) 100 * ((double) (numUsedSegments - numUnloadedSegments) / (double) numUsedSegments)
); );
} }
@ -371,10 +354,10 @@ public class DruidCoordinator
return CoordinatorCompactionConfig.current(configManager); return CoordinatorCompactionConfig.current(configManager);
} }
public void removeSegment(DataSegment segment) public void markSegmentAsUnused(DataSegment segment)
{ {
log.info("Removing Segment[%s]", segment.getId()); log.info("Marking segment[%s] as unused", segment.getId());
metadataSegmentManager.removeSegment(segment.getId().toString()); segmentsMetadata.markSegmentAsUnused(segment.getId().toString());
} }
public String getCurrentLeader() public String getCurrentLeader()
@ -382,13 +365,8 @@ public class DruidCoordinator
return coordLeaderSelector.getCurrentLeader(); return coordLeaderSelector.getCurrentLeader();
} }
@VisibleForTesting
void setDataSourcesSnapshotForTest(DataSourcesSnapshot snapshot)
{
dataSourcesSnapshot = snapshot;
}
public void moveSegment( public void moveSegment(
DruidCoordinatorRuntimeParams params,
ImmutableDruidServer fromServer, ImmutableDruidServer fromServer,
ImmutableDruidServer toServer, ImmutableDruidServer toServer,
DataSegment segment, DataSegment segment,
@ -408,9 +386,7 @@ public class DruidCoordinator
throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName()); throw new IAE("Cannot move [%s] to and from the same server [%s]", segmentId, fromServer.getName());
} }
ImmutableDruidDataSource dataSource = Optional.ofNullable(dataSourcesSnapshot) ImmutableDruidDataSource dataSource = params.getDataSourcesSnapshot().getDataSource(segment.getDataSource());
.map(m -> m.getDataSource(segment.getDataSource()))
.orElse(null);
if (dataSource == null) { if (dataSource == null) {
throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId); throw new IAE("Unable to find dataSource for segment [%s] in metadata", segmentId);
} }
@ -488,24 +464,6 @@ public class DruidCoordinator
} }
} }
/**
* 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<DataSegment> iterateAvailableDataSegments()
{
final Iterable<DataSegment> dataSources = Optional.ofNullable(dataSourcesSnapshot)
.map(m -> m.iterateAllSegmentsInSnapshot())
.orElse(null);
return dataSources == null ? null : dataSources;
}
@LifecycleStart @LifecycleStart
public void start() public void start()
{ {
@ -560,7 +518,7 @@ public class DruidCoordinator
log.info("I am the leader of the coordinators, all must bow!"); log.info("I am the leader of the coordinators, all must bow!");
log.info("Starting coordination in [%s]", config.getCoordinatorStartDelay()); log.info("Starting coordination in [%s]", config.getCoordinatorStartDelay());
metadataSegmentManager.start(); segmentsMetadata.startPollingDatabasePeriodically();
metadataRuleManager.start(); metadataRuleManager.start();
lookupCoordinatorManager.start(); lookupCoordinatorManager.start();
serviceAnnouncer.announce(self); serviceAnnouncer.announce(self);
@ -628,7 +586,7 @@ public class DruidCoordinator
serviceAnnouncer.unannounce(self); serviceAnnouncer.unannounce(self);
lookupCoordinatorManager.stop(); lookupCoordinatorManager.stop();
metadataRuleManager.stop(); metadataRuleManager.stop();
metadataSegmentManager.stop(); segmentsMetadata.stopPollingDatabasePeriodically();
} }
} }
@ -648,7 +606,7 @@ public class DruidCoordinator
public abstract class CoordinatorRunnable implements Runnable public abstract class CoordinatorRunnable implements Runnable
{ {
private final long startTime = System.currentTimeMillis(); private final long startTimeNanos = System.nanoTime();
private final List<DruidCoordinatorHelper> helpers; private final List<DruidCoordinatorHelper> helpers;
private final int startingLeaderCounter; private final int startingLeaderCounter;
@ -672,7 +630,7 @@ public class DruidCoordinator
} }
List<Boolean> allStarted = Arrays.asList( List<Boolean> allStarted = Arrays.asList(
metadataSegmentManager.isStarted(), segmentsMetadata.isPollingDatabasePeriodically(),
serverInventoryView.isStarted() serverInventoryView.isStarted()
); );
for (Boolean aBoolean : allStarted) { for (Boolean aBoolean : allStarted) {
@ -690,26 +648,19 @@ public class DruidCoordinator
BalancerStrategy balancerStrategy = factory.createBalancerStrategy(balancerExec); BalancerStrategy balancerStrategy = factory.createBalancerStrategy(balancerExec);
// Do coordinator stuff. // Do coordinator stuff.
dataSourcesSnapshot = metadataSegmentManager.getDataSourcesSnapshot(); DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments();
final Collection<ImmutableDruidDataSource> dataSources = Optional.ofNullable(dataSourcesSnapshot)
.map(m -> m.getDataSources())
.orElse(null);
if (dataSources == null) {
log.info("Metadata store not polled yet, skipping this run.");
return;
}
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams params =
DruidCoordinatorRuntimeParams.newBuilder() DruidCoordinatorRuntimeParams
.withStartTime(startTime) .newBuilder()
.withDataSources(dataSources) .withStartTimeNanos(startTimeNanos)
.withDynamicConfigs(getDynamicConfigs()) .withSnapshotOfDataSourcesWithAllUsedSegments(dataSourcesSnapshot)
.withCompactionConfig(getCompactionConfig()) .withDynamicConfigs(getDynamicConfigs())
.withEmitter(emitter) .withCompactionConfig(getCompactionConfig())
.withBalancerStrategy(balancerStrategy) .withEmitter(emitter)
.withDataSourcesSnapshot(dataSourcesSnapshot) .withBalancerStrategy(balancerStrategy)
.build(); .build();
for (DruidCoordinatorHelper helper : helpers) { for (DruidCoordinatorHelper helper : helpers) {
// Don't read state and run state in the same helper otherwise racy conditions may exist // Don't read state and run state in the same helper otherwise racy conditions may exist
if (coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) { if (coordLeaderSelector.isLeader() && startingLeaderCounter == coordLeaderSelector.localTerm()) {

View File

@ -69,15 +69,15 @@ public class DruidCoordinatorCleanupPendingSegments implements DruidCoordinatorH
// is no running/pending/waiting tasks. // is no running/pending/waiting tasks.
Preconditions.checkState(!createdTimes.isEmpty(), "Failed to gather createdTimes of 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). // (DateTimes.nowUtc() - KEEP_PENDING_SEGMENTS_OFFSET).
final DateTime pendingSegmentsCleanupEndTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET); final DateTime stalePendingSegmentsCutoffCreationTime = createdTimes.get(0).minus(KEEP_PENDING_SEGMENTS_OFFSET);
for (String dataSource : params.getDataSources().keySet()) { for (String dataSource : params.getUsedSegmentsTimelinesPerDataSource().keySet()) {
if (!params.getCoordinatorDynamicConfig().getProtectedPendingSegmentDatasources().contains(dataSource)) { if (!params.getCoordinatorDynamicConfig().getDataSourcesToNotKillStalePendingSegmentsIn().contains(dataSource)) {
log.info( log.info(
"Killed [%d] pendingSegments created until [%s] for dataSource[%s]", "Killed [%d] pendingSegments created until [%s] for dataSource[%s]",
indexingServiceClient.killPendingSegments(dataSource, pendingSegmentsCleanupEndTime), indexingServiceClient.killPendingSegments(dataSource, stalePendingSegmentsCutoffCreationTime),
pendingSegmentsCleanupEndTime, stalePendingSegmentsCutoffCreationTime,
dataSource dataSource
); );
} }

View File

@ -21,8 +21,8 @@ package org.apache.druid.server.coordinator;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.client.DataSourcesSnapshot; 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.common.DateTimes;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManager;
@ -36,6 +36,7 @@ import java.util.Collection;
import java.util.HashMap; import java.util.HashMap;
import java.util.Map; import java.util.Map;
import java.util.TreeSet; 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 * 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. * duplicate segments are encountered in the iterable.
*/ */
public static TreeSet<DataSegment> createAvailableSegmentsSet(Iterable<DataSegment> availableSegments) private static TreeSet<DataSegment> createUsedSegmentsSet(Iterable<DataSegment> usedSegments)
{ {
TreeSet<DataSegment> segmentsSet = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST); TreeSet<DataSegment> segmentsSet = new TreeSet<>(DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST);
availableSegments.forEach(segmentsSet::add); usedSegments.forEach(segmentsSet::add);
return segmentsSet; return segmentsSet;
} }
private final long startTime; private final long startTimeNanos;
private final DruidCluster druidCluster; private final DruidCluster druidCluster;
private final MetadataRuleManager databaseRuleManager; private final MetadataRuleManager databaseRuleManager;
private final SegmentReplicantLookup segmentReplicantLookup; private final SegmentReplicantLookup segmentReplicantLookup;
private final Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources; private final @Nullable TreeSet<DataSegment> usedSegments;
private final @Nullable TreeSet<DataSegment> availableSegments; private final @Nullable DataSourcesSnapshot dataSourcesSnapshot;
private final Map<String, LoadQueuePeon> loadManagementPeons; private final Map<String, LoadQueuePeon> loadManagementPeons;
private final ReplicationThrottler replicationManager; private final ReplicationThrottler replicationManager;
private final ServiceEmitter emitter; private final ServiceEmitter emitter;
@ -67,15 +68,14 @@ public class DruidCoordinatorRuntimeParams
private final CoordinatorStats stats; private final CoordinatorStats stats;
private final DateTime balancerReferenceTimestamp; private final DateTime balancerReferenceTimestamp;
private final BalancerStrategy balancerStrategy; private final BalancerStrategy balancerStrategy;
private final DataSourcesSnapshot dataSourcesSnapshot;
private DruidCoordinatorRuntimeParams( private DruidCoordinatorRuntimeParams(
long startTime, long startTimeNanos,
DruidCluster druidCluster, DruidCluster druidCluster,
MetadataRuleManager databaseRuleManager, MetadataRuleManager databaseRuleManager,
SegmentReplicantLookup segmentReplicantLookup, SegmentReplicantLookup segmentReplicantLookup,
Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources, @Nullable TreeSet<DataSegment> usedSegments,
@Nullable TreeSet<DataSegment> availableSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot,
Map<String, LoadQueuePeon> loadManagementPeons, Map<String, LoadQueuePeon> loadManagementPeons,
ReplicationThrottler replicationManager, ReplicationThrottler replicationManager,
ServiceEmitter emitter, ServiceEmitter emitter,
@ -83,16 +83,15 @@ public class DruidCoordinatorRuntimeParams
CoordinatorCompactionConfig coordinatorCompactionConfig, CoordinatorCompactionConfig coordinatorCompactionConfig,
CoordinatorStats stats, CoordinatorStats stats,
DateTime balancerReferenceTimestamp, DateTime balancerReferenceTimestamp,
BalancerStrategy balancerStrategy, BalancerStrategy balancerStrategy
DataSourcesSnapshot dataSourcesSnapshot
) )
{ {
this.startTime = startTime; this.startTimeNanos = startTimeNanos;
this.druidCluster = druidCluster; this.druidCluster = druidCluster;
this.databaseRuleManager = databaseRuleManager; this.databaseRuleManager = databaseRuleManager;
this.segmentReplicantLookup = segmentReplicantLookup; this.segmentReplicantLookup = segmentReplicantLookup;
this.dataSources = dataSources; this.usedSegments = usedSegments;
this.availableSegments = availableSegments; this.dataSourcesSnapshot = dataSourcesSnapshot;
this.loadManagementPeons = loadManagementPeons; this.loadManagementPeons = loadManagementPeons;
this.replicationManager = replicationManager; this.replicationManager = replicationManager;
this.emitter = emitter; this.emitter = emitter;
@ -101,12 +100,11 @@ public class DruidCoordinatorRuntimeParams
this.stats = stats; this.stats = stats;
this.balancerReferenceTimestamp = balancerReferenceTimestamp; this.balancerReferenceTimestamp = balancerReferenceTimestamp;
this.balancerStrategy = balancerStrategy; this.balancerStrategy = balancerStrategy;
this.dataSourcesSnapshot = dataSourcesSnapshot;
} }
public long getStartTime() public long getStartTimeNanos()
{ {
return startTime; return startTimeNanos;
} }
public DruidCluster getDruidCluster() public DruidCluster getDruidCluster()
@ -124,15 +122,20 @@ public class DruidCoordinatorRuntimeParams
return segmentReplicantLookup; return segmentReplicantLookup;
} }
public Map<String, VersionedIntervalTimeline<String, DataSegment>> getDataSources() /**
* Creates and returns a "dataSource -> VersionedIntervalTimeline[version String, DataSegment]" map with "used"
* segments.
*/
public Map<String, VersionedIntervalTimeline<String, DataSegment>> getUsedSegmentsTimelinesPerDataSource()
{ {
return dataSources; Preconditions.checkState(dataSourcesSnapshot != null, "dataSourcesSnapshot or usedSegments must be set");
return dataSourcesSnapshot.getUsedSegmentsTimelinesPerDataSource();
} }
public TreeSet<DataSegment> getAvailableSegments() public TreeSet<DataSegment> getUsedSegments()
{ {
Preconditions.checkState(availableSegments != null, "availableSegments must be set"); Preconditions.checkState(usedSegments != null, "usedSegments or dataSourcesSnapshot must be set");
return availableSegments; return usedSegments;
} }
public Map<String, LoadQueuePeon> getLoadManagementPeons() public Map<String, LoadQueuePeon> getLoadManagementPeons()
@ -175,16 +178,20 @@ public class DruidCoordinatorRuntimeParams
return balancerStrategy; return balancerStrategy;
} }
public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()
{
long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos();
long lagNanos = TimeUnit.MILLISECONDS.toNanos(
coordinatorDynamicConfig.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
);
return nanosElapsedSinceCoordinatorStart > lagNanos;
}
public DataSourcesSnapshot getDataSourcesSnapshot() public DataSourcesSnapshot getDataSourcesSnapshot()
{ {
return dataSourcesSnapshot; return dataSourcesSnapshot;
} }
public boolean hasDeletionWaitTimeElapsed()
{
return (System.currentTimeMillis() - getStartTime() > coordinatorDynamicConfig.getMillisToWaitBeforeDeleting());
}
public static Builder newBuilder() public static Builder newBuilder()
{ {
return new Builder(); return new Builder();
@ -193,12 +200,12 @@ public class DruidCoordinatorRuntimeParams
public Builder buildFromExisting() public Builder buildFromExisting()
{ {
return new Builder( return new Builder(
startTime, startTimeNanos,
druidCluster, druidCluster,
databaseRuleManager, databaseRuleManager,
segmentReplicantLookup, segmentReplicantLookup,
dataSources, usedSegments,
availableSegments, dataSourcesSnapshot,
loadManagementPeons, loadManagementPeons,
replicationManager, replicationManager,
emitter, emitter,
@ -210,15 +217,15 @@ public class DruidCoordinatorRuntimeParams
); );
} }
public Builder buildFromExistingWithoutAvailableSegments() public Builder buildFromExistingWithoutSegmentsMetadata()
{ {
return new Builder( return new Builder(
startTime, startTimeNanos,
druidCluster, druidCluster,
databaseRuleManager, databaseRuleManager,
segmentReplicantLookup, segmentReplicantLookup,
dataSources, null, // usedSegments
null, // availableSegments null, // dataSourcesSnapshot
loadManagementPeons, loadManagementPeons,
replicationManager, replicationManager,
emitter, emitter,
@ -232,12 +239,12 @@ public class DruidCoordinatorRuntimeParams
public static class Builder public static class Builder
{ {
private long startTime; private @Nullable Long startTimeNanos;
private DruidCluster druidCluster; private DruidCluster druidCluster;
private MetadataRuleManager databaseRuleManager; private MetadataRuleManager databaseRuleManager;
private SegmentReplicantLookup segmentReplicantLookup; private SegmentReplicantLookup segmentReplicantLookup;
private Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources; private @Nullable TreeSet<DataSegment> usedSegments;
private @Nullable TreeSet<DataSegment> availableSegments; private @Nullable DataSourcesSnapshot dataSourcesSnapshot;
private final Map<String, LoadQueuePeon> loadManagementPeons; private final Map<String, LoadQueuePeon> loadManagementPeons;
private ReplicationThrottler replicationManager; private ReplicationThrottler replicationManager;
private ServiceEmitter emitter; private ServiceEmitter emitter;
@ -246,16 +253,15 @@ public class DruidCoordinatorRuntimeParams
private CoordinatorStats stats; private CoordinatorStats stats;
private DateTime balancerReferenceTimestamp; private DateTime balancerReferenceTimestamp;
private BalancerStrategy balancerStrategy; private BalancerStrategy balancerStrategy;
private DataSourcesSnapshot dataSourcesSnapshot;
Builder() private Builder()
{ {
this.startTime = 0; this.startTimeNanos = null;
this.druidCluster = null; this.druidCluster = null;
this.databaseRuleManager = null; this.databaseRuleManager = null;
this.segmentReplicantLookup = null; this.segmentReplicantLookup = null;
this.dataSources = new HashMap<>(); this.usedSegments = null;
this.availableSegments = null; this.dataSourcesSnapshot = null;
this.loadManagementPeons = new HashMap<>(); this.loadManagementPeons = new HashMap<>();
this.replicationManager = null; this.replicationManager = null;
this.emitter = null; this.emitter = null;
@ -263,16 +269,15 @@ public class DruidCoordinatorRuntimeParams
this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build(); this.coordinatorDynamicConfig = CoordinatorDynamicConfig.builder().build();
this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty(); this.coordinatorCompactionConfig = CoordinatorCompactionConfig.empty();
this.balancerReferenceTimestamp = DateTimes.nowUtc(); this.balancerReferenceTimestamp = DateTimes.nowUtc();
this.dataSourcesSnapshot = null;
} }
Builder( Builder(
long startTime, long startTimeNanos,
DruidCluster cluster, DruidCluster cluster,
MetadataRuleManager databaseRuleManager, MetadataRuleManager databaseRuleManager,
SegmentReplicantLookup segmentReplicantLookup, SegmentReplicantLookup segmentReplicantLookup,
Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources, @Nullable TreeSet<DataSegment> usedSegments,
@Nullable TreeSet<DataSegment> availableSegments, @Nullable DataSourcesSnapshot dataSourcesSnapshot,
Map<String, LoadQueuePeon> loadManagementPeons, Map<String, LoadQueuePeon> loadManagementPeons,
ReplicationThrottler replicationManager, ReplicationThrottler replicationManager,
ServiceEmitter emitter, ServiceEmitter emitter,
@ -283,12 +288,12 @@ public class DruidCoordinatorRuntimeParams
BalancerStrategy balancerStrategy BalancerStrategy balancerStrategy
) )
{ {
this.startTime = startTime; this.startTimeNanos = startTimeNanos;
this.druidCluster = cluster; this.druidCluster = cluster;
this.databaseRuleManager = databaseRuleManager; this.databaseRuleManager = databaseRuleManager;
this.segmentReplicantLookup = segmentReplicantLookup; this.segmentReplicantLookup = segmentReplicantLookup;
this.dataSources = dataSources; this.usedSegments = usedSegments;
this.availableSegments = availableSegments; this.dataSourcesSnapshot = dataSourcesSnapshot;
this.loadManagementPeons = loadManagementPeons; this.loadManagementPeons = loadManagementPeons;
this.replicationManager = replicationManager; this.replicationManager = replicationManager;
this.emitter = emitter; this.emitter = emitter;
@ -301,13 +306,14 @@ public class DruidCoordinatorRuntimeParams
public DruidCoordinatorRuntimeParams build() public DruidCoordinatorRuntimeParams build()
{ {
Preconditions.checkNotNull(startTimeNanos, "startTime must be set");
return new DruidCoordinatorRuntimeParams( return new DruidCoordinatorRuntimeParams(
startTime, startTimeNanos,
druidCluster, druidCluster,
databaseRuleManager, databaseRuleManager,
segmentReplicantLookup, segmentReplicantLookup,
dataSources, usedSegments,
availableSegments, dataSourcesSnapshot,
loadManagementPeons, loadManagementPeons,
replicationManager, replicationManager,
emitter, emitter,
@ -315,14 +321,13 @@ public class DruidCoordinatorRuntimeParams
coordinatorCompactionConfig, coordinatorCompactionConfig,
stats, stats,
balancerReferenceTimestamp, balancerReferenceTimestamp,
balancerStrategy, balancerStrategy
dataSourcesSnapshot
); );
} }
public Builder withStartTime(long time) public Builder withStartTimeNanos(long startTimeNanos)
{ {
startTime = time; this.startTimeNanos = startTimeNanos;
return this; return this;
} }
@ -344,58 +349,40 @@ public class DruidCoordinatorRuntimeParams
return this; return this;
} }
public Builder withDataSources(Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources) public Builder withSnapshotOfDataSourcesWithAllUsedSegments(DataSourcesSnapshot snapshot)
{ {
this.dataSources = dataSources; this.usedSegments = createUsedSegmentsSet(snapshot.iterateAllUsedSegmentsInSnapshot());
this.dataSourcesSnapshot = snapshot;
return this; return this;
} }
public Builder withDataSources(Collection<ImmutableDruidDataSource> dataSourcesCollection) /** This method must be used in test code only. */
@VisibleForTesting
public Builder withUsedSegmentsInTest(DataSegment... usedSegments)
{ {
dataSourcesCollection.forEach( return withUsedSegmentsInTest(Arrays.asList(usedSegments));
dataSource -> { }
VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.computeIfAbsent(
dataSource.getName(),
k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER)
);
dataSource.getSegments().forEach( /** This method must be used in test code only. */
segment -> timeline.add( @VisibleForTesting
segment.getInterval(), public Builder withUsedSegmentsInTest(Collection<DataSegment> usedSegments)
segment.getVersion(), {
segment.getShardSpec().createChunk(segment) this.usedSegments = createUsedSegmentsSet(usedSegments);
) this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments(usedSegments, ImmutableMap.of());
); return this;
} }
/** This method must be used in test code only. */
@VisibleForTesting
public Builder withUsedSegmentsTimelinesPerDataSourceInTest(
Map<String, VersionedIntervalTimeline<String, DataSegment>> usedSegmentsTimelinesPerDataSource
)
{
this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegmentsTimelines(
usedSegmentsTimelinesPerDataSource,
ImmutableMap.of()
); );
return this; usedSegments = createUsedSegmentsSet(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot());
}
/** This method must be used in test code only. */
@VisibleForTesting
public Builder withAvailableSegmentsInTest(DataSegment... availableSegments)
{
return withAvailableSegmentsInTest(Arrays.asList(availableSegments));
}
/** This method must be used in test code only. */
@VisibleForTesting
public Builder withAvailableSegmentsInTest(Collection<DataSegment> availableSegments)
{
return setAvailableSegments(createAvailableSegmentsSet(availableSegments));
}
/**
* 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<DataSegment> availableSegments)
{
//noinspection ObjectEquality
if (availableSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) {
throw new IllegalArgumentException("Expected DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST");
}
this.availableSegments = availableSegments;
return this; return this;
} }
@ -446,11 +433,5 @@ public class DruidCoordinatorRuntimeParams
this.balancerStrategy = balancerStrategy; this.balancerStrategy = balancerStrategy;
return this; return this;
} }
public Builder withDataSourcesSnapshot(DataSourcesSnapshot snapshot)
{
this.dataSourcesSnapshot = snapshot;
return this;
}
} }
} }

View File

@ -34,7 +34,7 @@ final class ReservoirSegmentSampler
int numSoFar = 0; int numSoFar = 0;
for (ServerHolder server : serverHolders) { for (ServerHolder server : serverHolders) {
for (DataSegment segment : server.getServer().getLazyAllSegments()) { for (DataSegment segment : server.getServer().iterateAllSegments()) {
int randNum = ThreadLocalRandom.current().nextInt(numSoFar + 1); int randNum = ThreadLocalRandom.current().nextInt(numSoFar + 1);
// w.p. 1 / (numSoFar+1), swap out the server and segment // w.p. 1 / (numSoFar+1), swap out the server and segment
if (randNum == numSoFar) { if (randNum == numSoFar) {

View File

@ -43,7 +43,7 @@ public class SegmentReplicantLookup
for (ServerHolder serverHolder : serversByType) { for (ServerHolder serverHolder : serversByType) {
ImmutableDruidServer server = serverHolder.getServer(); ImmutableDruidServer server = serverHolder.getServer();
for (DataSegment segment : server.getLazyAllSegments()) { for (DataSegment segment : server.iterateAllSegments()) {
Integer numReplicants = segmentsInCluster.get(segment.getId(), server.getTier()); Integer numReplicants = segmentsInCluster.get(segment.getId(), server.getTier());
if (numReplicants == null) { if (numReplicants == null) {
numReplicants = 0; numReplicants = 0;

View File

@ -94,8 +94,8 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
) )
{ {
if (params.getAvailableSegments().size() == 0) { if (params.getUsedSegments().size() == 0) {
log.warn("Metadata segments are not available. Cannot balance."); log.info("Metadata segments are not available. Cannot balance.");
// suppress emit zero stats // suppress emit zero stats
return; return;
} }
@ -134,7 +134,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
int numSegments = 0; int numSegments = 0;
for (ServerHolder sourceHolder : servers) { for (ServerHolder sourceHolder : servers) {
numSegments += sourceHolder.getServer().getLazyAllSegments().size(); numSegments += sourceHolder.getServer().getNumSegments();
} }
if (numSegments == 0) { if (numSegments == 0) {
@ -195,13 +195,12 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
log.info("All servers to move segments from are empty, ending run."); log.info("All servers to move segments from are empty, ending run.");
break; break;
} }
// DruidCoordinatorRuntimeParams.getAvailableSegments originate from MetadataSegmentManager, i. e. that's a // DruidCoordinatorRuntimeParams.getUsedSegments originate from SegmentsMetadata, i. e. that's a set of segments
// "desired" or "theoretical" set of segments. segmentToMoveHolder.getSegment originates from ServerInventoryView, // that *should* be loaded. segmentToMoveHolder.getSegment originates from ServerInventoryView, i. e. that may be
// i. e. that may be any segment that happens to be loaded on some server, even if it "shouldn't" from the // any segment that happens to be loaded on some server, even if it is not used. (Coordinator closes such
// "theoretical" point of view (Coordinator closes such discrepancies eventually via // discrepancies eventually via DruidCoordinatorUnloadUnusedSegments). Therefore the picked segmentToMoveHolder's
// DruidCoordinatorCleanupUnneeded). Therefore the picked segmentToMoveHolder's segment may not need to be // segment may not need to be balanced.
// balanced. boolean needToBalancePickedSegment = params.getUsedSegments().contains(segmentToMoveHolder.getSegment());
boolean needToBalancePickedSegment = params.getAvailableSegments().contains(segmentToMoveHolder.getSegment());
if (needToBalancePickedSegment) { if (needToBalancePickedSegment) {
final DataSegment segmentToMove = segmentToMoveHolder.getSegment(); final DataSegment segmentToMove = segmentToMoveHolder.getSegment();
final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer(); final ImmutableDruidServer fromServer = segmentToMoveHolder.getFromServer();
@ -269,6 +268,7 @@ public class DruidCoordinatorBalancer implements DruidCoordinatorHelper
movingSegments.put(segmentId, segment); movingSegments.put(segmentId, segment);
callback = () -> movingSegments.remove(segmentId); callback = () -> movingSegments.remove(segmentId);
coordinator.moveSegment( coordinator.moveSegment(
params,
fromServer, fromServer,
toServer, toServer,
segmentToMove, segmentToMove,

View File

@ -46,41 +46,40 @@ public class DruidCoordinatorCleanupOvershadowed implements DruidCoordinatorHelp
@Override @Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{ {
// 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;
}
CoordinatorStats stats = new CoordinatorStats(); CoordinatorStats stats = new CoordinatorStats();
// Delete segments that are old DruidCluster cluster = params.getDruidCluster();
// Unservice old partitions if we've had enough time to make sure we aren't flapping with old data Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = new HashMap<>();
if (params.hasDeletionWaitTimeElapsed()) {
DruidCluster cluster = params.getDruidCluster();
Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = new HashMap<>();
for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) { for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
for (ServerHolder serverHolder : serverHolders) { for (ServerHolder serverHolder : serverHolders) {
ImmutableDruidServer server = serverHolder.getServer(); ImmutableDruidServer server = serverHolder.getServer();
for (ImmutableDruidDataSource dataSource : server.getDataSources()) { for (ImmutableDruidDataSource dataSource : server.getDataSources()) {
VersionedIntervalTimeline<String, DataSegment> timeline = timelines.get(dataSource.getName()); VersionedIntervalTimeline<String, DataSegment> timeline = timelines
if (timeline == null) { .computeIfAbsent(
timeline = new VersionedIntervalTimeline<>(Comparator.naturalOrder()); dataSource.getName(),
timelines.put(dataSource.getName(), timeline); dsName -> new VersionedIntervalTimeline<>(Comparator.naturalOrder())
} );
VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator());
VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator());
}
}
}
//Remove all segments in db that are overshadowed by served segments
for (DataSegment dataSegment : params.getAvailableSegments()) {
VersionedIntervalTimeline<String, DataSegment> timeline = timelines.get(dataSegment.getDataSource());
if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) {
coordinator.removeSegment(dataSegment);
stats.addToGlobalStat("overShadowedCount", 1);
} }
} }
} }
return params.buildFromExisting()
.withCoordinatorStats(stats) // Mark all segments as unused in db that are overshadowed by served segments
.build(); for (DataSegment dataSegment : params.getUsedSegments()) {
VersionedIntervalTimeline<String, DataSegment> 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();
} }
} }

View File

@ -43,22 +43,19 @@ public class DruidCoordinatorCleanupUnneeded implements DruidCoordinatorHelper
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{ {
CoordinatorStats stats = new CoordinatorStats(); CoordinatorStats stats = new CoordinatorStats();
Set<DataSegment> availableSegments = params.getAvailableSegments(); Set<DataSegment> usedSegments = params.getUsedSegments();
DruidCluster cluster = params.getDruidCluster(); DruidCluster cluster = params.getDruidCluster();
// Drop segments that no longer exist in the available segments configuration, *if* it has been populated. (It's // Unload segments that are no longer marked as used from historical servers, *if* the usedSegments collection has
// also filled atomically, so if there are any segments at all, we should have all of them.) // been populated. Used segments must be already populated because otherwise the earlier helper
// // DruidCoordinatorUsedSegmentsLoader would have canceled the Coordinator's run.
// 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.
for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) { for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
for (ServerHolder serverHolder : serverHolders) { for (ServerHolder serverHolder : serverHolders) {
ImmutableDruidServer server = serverHolder.getServer(); ImmutableDruidServer server = serverHolder.getServer();
for (ImmutableDruidDataSource dataSource : server.getDataSources()) { for (ImmutableDruidDataSource dataSource : server.getDataSources()) {
for (DataSegment segment : dataSource.getSegments()) { for (DataSegment segment : dataSource.getSegments()) {
if (!availableSegments.contains(segment)) { if (!usedSegments.contains(segment)) {
LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName()); LoadQueuePeon queuePeon = params.getLoadManagementPeons().get(server.getName());
if (!queuePeon.getSegmentsToDrop().contains(segment)) { if (!queuePeon.getSegmentsToDrop().contains(segment)) {

View File

@ -19,6 +19,7 @@
package org.apache.druid.server.coordinator.helper; package org.apache.druid.server.coordinator.helper;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2LongMap; import it.unimi.dsi.fastutil.objects.Object2LongMap;
import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.common.logger.Logger;
@ -214,14 +215,14 @@ public class DruidCoordinatorLogger implements DruidCoordinatorHelper
); );
}); });
coordinator.getSegmentAvailability().object2LongEntrySet().forEach( coordinator.computeNumsUnavailableUsedSegmentsPerDataSource().object2IntEntrySet().forEach(
(final Object2LongMap.Entry<String> entry) -> { (final Object2IntMap.Entry<String> entry) -> {
final String dataSource = entry.getKey(); final String dataSource = entry.getKey();
final long count = entry.getLongValue(); final int numUnavailableUsedSegmentsInDataSource = entry.getIntValue();
emitter.emit( emitter.emit(
new ServiceMetricEvent.Builder() new ServiceMetricEvent.Builder()
.setDimension(DruidMetrics.DATASOURCE, dataSource).build( .setDimension(DruidMetrics.DATASOURCE, dataSource).build(
"segment/unavailable/count", count "segment/unavailable/count", numUnavailableUsedSegmentsInDataSource
) )
); );
} }
@ -247,7 +248,7 @@ public class DruidCoordinatorLogger implements DruidCoordinatorHelper
emitter.emit( emitter.emit(
new ServiceMetricEvent.Builder().build( new ServiceMetricEvent.Builder().build(
"compact/task/count", "compact/task/count",
stats.getGlobalStat("compactTaskCount") stats.getGlobalStat(DruidCoordinatorSegmentCompactor.COMPACT_TASK_COUNT)
) )
); );
@ -264,7 +265,7 @@ public class DruidCoordinatorLogger implements DruidCoordinatorHelper
// Emit segment metrics // Emit segment metrics
final Stream<DataSegment> allSegments = params final Stream<DataSegment> allSegments = params
.getDataSources() .getUsedSegmentsTimelinesPerDataSource()
.values() .values()
.stream() .stream()
.flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream())

View File

@ -19,9 +19,7 @@
package org.apache.druid.server.coordinator.helper; package org.apache.druid.server.coordinator.helper;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists; 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.common.DateTimes;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.MetadataRuleManager; import org.apache.druid.metadata.MetadataRuleManager;
@ -36,7 +34,6 @@ import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.Set; import java.util.Set;
/** /**
@ -44,7 +41,7 @@ import java.util.Set;
public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
{ {
private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorRuleRunner.class); 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; private final ReplicationThrottler replicatorThrottler;
@ -83,34 +80,28 @@ public class DruidCoordinatorRuleRunner implements DruidCoordinatorHelper
return params; return params;
} }
// find available segments which are not overshadowed by other segments in DB // Get used segments which are overshadowed by other used segments. Those would not need to be loaded and
// only those would need to be loaded/dropped // eventually will be unloaded from Historical servers. Segments overshadowed by *served* used segments are marked
// anything overshadowed by served segments is dropped automatically by DruidCoordinatorCleanupOvershadowed // as unused in DruidCoordinatorMarkAsUnusedOvershadowedSegments, and then eventually Coordinator sends commands to
// If metadata store hasn't been polled yet, use empty overshadowed list // Historical nodes to unload such segments in DruidCoordinatorUnloadUnusedSegments.
final DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot(); Set<SegmentId> overshadowed = params.getDataSourcesSnapshot().getOvershadowedSegments();
Set<SegmentId> overshadowed = ImmutableSet.of();
if (dataSourcesSnapshot != null) {
overshadowed = Optional
.ofNullable(dataSourcesSnapshot.getOvershadowedSegments())
.orElse(ImmutableSet.of());
}
for (String tier : cluster.getTierNames()) { for (String tier : cluster.getTierNames()) {
replicatorThrottler.updateReplicationState(tier); replicatorThrottler.updateReplicationState(tier);
} }
DruidCoordinatorRuntimeParams paramsWithReplicationManager = params DruidCoordinatorRuntimeParams paramsWithReplicationManager = params
.buildFromExistingWithoutAvailableSegments() .buildFromExistingWithoutSegmentsMetadata()
.withReplicationManager(replicatorThrottler) .withReplicationManager(replicatorThrottler)
.build(); .build();
// Run through all matched rules for available segments // Run through all matched rules for used segments
DateTime now = DateTimes.nowUtc(); DateTime now = DateTimes.nowUtc();
MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager(); MetadataRuleManager databaseRuleManager = paramsWithReplicationManager.getDatabaseRuleManager();
final List<SegmentId> segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES); final List<SegmentId> segmentsWithMissingRules = Lists.newArrayListWithCapacity(MAX_MISSING_RULES);
int missingRules = 0; int missingRules = 0;
for (DataSegment segment : params.getAvailableSegments()) { for (DataSegment segment : params.getUsedSegments()) {
if (overshadowed.contains(segment.getId())) { if (overshadowed.contains(segment.getId())) {
// Skipping overshadowed segments // Skipping overshadowed segments
continue; continue;

View File

@ -76,7 +76,8 @@ public class DruidCoordinatorSegmentCompactor implements DruidCoordinatorHelper
final CoordinatorStats stats = new CoordinatorStats(); final CoordinatorStats stats = new CoordinatorStats();
if (dynamicConfig.getMaxCompactionTaskSlots() > 0) { if (dynamicConfig.getMaxCompactionTaskSlots() > 0) {
Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources = params.getDataSources(); Map<String, VersionedIntervalTimeline<String, DataSegment>> dataSources =
params.getUsedSegmentsTimelinesPerDataSource();
List<DataSourceCompactionConfig> compactionConfigList = dynamicConfig.getCompactionConfigs(); List<DataSourceCompactionConfig> compactionConfigList = dynamicConfig.getCompactionConfigs();
if (compactionConfigList != null && !compactionConfigList.isEmpty()) { if (compactionConfigList != null && !compactionConfigList.isEmpty()) {

View File

@ -19,14 +19,12 @@
package org.apache.druid.server.coordinator.helper; 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.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.DruidCoordinator; import org.apache.druid.server.coordinator.DruidCoordinator;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import java.util.TreeSet;
public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper
{ {
private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class); private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class);
@ -41,50 +39,27 @@ public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper
@Override @Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{ {
log.info("Starting coordination. Getting available segments."); log.info("Starting coordination. Getting used segments.");
final Iterable<DataSegment> dataSegments = coordinator.iterateAvailableDataSegments(); DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot();
if (dataSegments == null) { for (DataSegment segment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) {
log.info("Metadata store not polled yet, canceling this run."); if (segment.getSize() < 0) {
return null; 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 // Log info about all used 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<DataSegment> availableSegmentsWithSizeChecking = Iterables.transform(
dataSegments,
segment -> {
if (segment.getSize() < 0) {
log.makeAlert("No size on a segment")
.addData("segment", segment)
.emit();
}
return segment;
}
);
final TreeSet<DataSegment> availableSegments =
DruidCoordinatorRuntimeParams.createAvailableSegmentsSet(availableSegmentsWithSizeChecking);
// Log info about all available segments
if (log.isDebugEnabled()) { if (log.isDebugEnabled()) {
log.debug("Available DataSegments"); log.debug("Used Segments");
for (DataSegment dataSegment : availableSegments) { for (DataSegment dataSegment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) {
log.debug(" %s", dataSegment); log.debug(" %s", dataSegment);
} }
} }
log.info("Found [%,d] available segments.", availableSegments.size()); log.info("Found [%,d] used segments.", params.getUsedSegments().size());
return params.buildFromExisting() return params;
.setAvailableSegments(availableSegments)
.build();
} }
} }

View File

@ -31,10 +31,15 @@ import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Collection; import java.util.Collection;
import java.util.List; 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 public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
{ {
@ -46,12 +51,12 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
private long lastKillTime = 0; private long lastKillTime = 0;
private final MetadataSegmentManager segmentManager; private final MetadataSegmentManager segmentsMetadata;
private final IndexingServiceClient indexingServiceClient; private final IndexingServiceClient indexingServiceClient;
@Inject @Inject
public DruidCoordinatorSegmentKiller( public DruidCoordinatorSegmentKiller(
MetadataSegmentManager segmentManager, MetadataSegmentManager segmentsMetadata,
IndexingServiceClient indexingServiceClient, IndexingServiceClient indexingServiceClient,
DruidCoordinatorConfig config DruidCoordinatorConfig config
) )
@ -75,30 +80,36 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
this.maxSegmentsToKill this.maxSegmentsToKill
); );
this.segmentManager = segmentManager; this.segmentsMetadata = segmentsMetadata;
this.indexingServiceClient = indexingServiceClient; this.indexingServiceClient = indexingServiceClient;
} }
@Override @Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{ {
boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources(); boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources();
Collection<String> whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources(); Collection<String> specificDataSourcesToKill =
params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn();
if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) { if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) {
log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled."); log.error(
"killAllDataSources can't be true when specificDataSourcesToKill is non-empty. No kill tasks are scheduled."
);
return params; return params;
} }
Collection<String> dataSourcesToKill = specificDataSourcesToKill;
if (killAllDataSources) { 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(); lastKillTime = System.currentTimeMillis();
for (String dataSource : whitelist) { for (String dataSource : dataSourcesToKill) {
final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill); final Interval intervalToKill = findIntervalForKill(dataSource, maxSegmentsToKill);
if (intervalToKill != null) { if (intervalToKill != null) {
try { try {
indexingServiceClient.killSegments(dataSource, intervalToKill); indexingServiceClient.killSegments(dataSource, intervalToKill);
@ -117,13 +128,11 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
} }
@VisibleForTesting @VisibleForTesting
Interval findIntervalForKillTask(String dataSource, int limit) @Nullable
Interval findIntervalForKill(String dataSource, int limit)
{ {
List<Interval> unusedSegmentIntervals = segmentManager.getUnusedSegmentIntervals( List<Interval> unusedSegmentIntervals =
dataSource, segmentsMetadata.getUnusedSegmentIntervals(dataSource, DateTimes.nowUtc().minus(retainDuration), limit);
new Interval(DateTimes.EPOCH, DateTimes.nowUtc().minus(retainDuration)),
limit
);
if (unusedSegmentIntervals != null && unusedSegmentIntervals.size() > 0) { if (unusedSegmentIntervals != null && unusedSegmentIntervals.size() > 0) {
return JodaUtils.umbrellaInterval(unusedSegmentIntervals); return JodaUtils.umbrellaInterval(unusedSegmentIntervals);

View File

@ -33,7 +33,7 @@ public abstract class DropRule implements Rule
public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment) public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment)
{ {
CoordinatorStats stats = new CoordinatorStats(); CoordinatorStats stats = new CoordinatorStats();
coordinator.removeSegment(segment); coordinator.markSegmentAsUnused(segment);
stats.addToGlobalStat("deletedCount", 1); stats.addToGlobalStat("deletedCount", 1);
return stats; return stats;
} }

View File

@ -52,13 +52,13 @@ public interface Rule
boolean appliesTo(Interval interval, DateTime referenceTimestamp); boolean appliesTo(Interval interval, DateTime referenceTimestamp);
/** /**
* {@link DruidCoordinatorRuntimeParams#getAvailableSegments()} must not be called in Rule's code, because the * {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used
* available segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is * segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because
* because {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions: * {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions:
* "DruidCoordinatorHelperParams" and "RuleParams" which contain params that only {@link * "DruidCoordinatorHelperParams" and "RuleParams" which contain params that only {@link
* org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper}s and Rules need, respectively. * 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", * 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 * See https://github.com/apache/incubator-druid/issues/7228
*/ */

View File

@ -91,7 +91,7 @@ public class CoordinatorResource
) )
{ {
if (simple != null) { if (simple != null) {
return Response.ok(coordinator.getSegmentAvailability()).build(); return Response.ok(coordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).build();
} }
if (full != null) { if (full != null) {

View File

@ -22,10 +22,12 @@ package org.apache.druid.server.http;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting; import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.sun.jersey.spi.container.ResourceFilters; import com.sun.jersey.spi.container.ResourceFilters;
import org.apache.commons.lang.StringUtils;
import org.apache.druid.client.CoordinatorServerView; import org.apache.druid.client.CoordinatorServerView;
import org.apache.druid.client.DruidDataSource; import org.apache.druid.client.DruidDataSource;
import org.apache.druid.client.DruidServer; import org.apache.druid.client.DruidServer;
@ -50,7 +52,6 @@ import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.server.coordinator.rules.LoadRule;
import org.apache.druid.server.coordinator.rules.Rule; import org.apache.druid.server.coordinator.rules.Rule;
import org.apache.druid.server.http.security.DatasourceResourceFilter; 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.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
@ -97,27 +98,24 @@ public class DataSourcesResource
private static final Logger log = new Logger(DataSourcesResource.class); private static final Logger log = new Logger(DataSourcesResource.class);
private final CoordinatorServerView serverInventoryView; private final CoordinatorServerView serverInventoryView;
private final MetadataSegmentManager databaseSegmentManager; private final MetadataSegmentManager segmentsMetadata;
private final MetadataRuleManager databaseRuleManager; private final MetadataRuleManager metadataRuleManager;
private final IndexingServiceClient indexingServiceClient; private final IndexingServiceClient indexingServiceClient;
private final AuthConfig authConfig;
private final AuthorizerMapper authorizerMapper; private final AuthorizerMapper authorizerMapper;
@Inject @Inject
public DataSourcesResource( public DataSourcesResource(
CoordinatorServerView serverInventoryView, CoordinatorServerView serverInventoryView,
MetadataSegmentManager databaseSegmentManager, MetadataSegmentManager segmentsMetadata,
MetadataRuleManager databaseRuleManager, MetadataRuleManager metadataRuleManager,
@Nullable IndexingServiceClient indexingServiceClient, @Nullable IndexingServiceClient indexingServiceClient,
AuthConfig authConfig,
AuthorizerMapper authorizerMapper AuthorizerMapper authorizerMapper
) )
{ {
this.serverInventoryView = serverInventoryView; this.serverInventoryView = serverInventoryView;
this.databaseSegmentManager = databaseSegmentManager; this.segmentsMetadata = segmentsMetadata;
this.databaseRuleManager = databaseRuleManager; this.metadataRuleManager = metadataRuleManager;
this.indexingServiceClient = indexingServiceClient; this.indexingServiceClient = indexingServiceClient;
this.authConfig = authConfig;
this.authorizerMapper = authorizerMapper; this.authorizerMapper = authorizerMapper;
} }
@ -130,24 +128,17 @@ public class DataSourcesResource
) )
{ {
Response.ResponseBuilder builder = Response.ok(); Response.ResponseBuilder builder = Response.ok();
final Set<ImmutableDruidDataSource> datasources = InventoryViewUtils.getSecuredDataSources( final Set<ImmutableDruidDataSource> datasources =
req, InventoryViewUtils.getSecuredDataSources(req, serverInventoryView, authorizerMapper);
serverInventoryView,
authorizerMapper
);
final Object entity; final Object entity;
if (full != null) { if (full != null) {
entity = datasources; entity = datasources;
} else if (simple != null) { } else if (simple != null) {
entity = datasources.stream() entity = datasources.stream().map(this::makeSimpleDatasource).collect(Collectors.toList());
.map(this::makeSimpleDatasource)
.collect(Collectors.toList());
} else { } else {
entity = datasources.stream() entity = datasources.stream().map(ImmutableDruidDataSource::getName).collect(Collectors.toList());
.map(ImmutableDruidDataSource::getName)
.collect(Collectors.toList());
} }
return builder.entity(entity).build(); return builder.entity(entity).build();
@ -157,7 +148,7 @@ public class DataSourcesResource
@Path("/{dataSourceName}") @Path("/{dataSourceName}")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getTheDataSource( public Response getDataSource(
@PathParam("dataSourceName") final String dataSourceName, @PathParam("dataSourceName") final String dataSourceName,
@QueryParam("full") final String full @QueryParam("full") final String full
) )
@ -165,7 +156,7 @@ public class DataSourcesResource
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.noContent().build(); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
if (full != null) { if (full != null) {
@ -175,33 +166,128 @@ public class DataSourcesResource
return Response.ok(getSimpleDatasource(dataSourceName)).build(); return Response.ok(getSimpleDatasource(dataSourceName)).build();
} }
private interface MarkSegments
{
int markSegments() throws UnknownSegmentIdException;
}
@POST @POST
@Path("/{dataSourceName}") @Path("/{dataSourceName}")
@Consumes(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response enableDataSource( public Response markAsUsedAllNonOvershadowedSegments(@PathParam("dataSourceName") final String dataSourceName)
@PathParam("dataSourceName") final String dataSourceName
)
{ {
if (!databaseSegmentManager.enableDataSource(dataSourceName)) { MarkSegments markSegments = () -> segmentsMetadata.markAsUsedAllNonOvershadowedSegmentsInDataSource(dataSourceName);
return Response.noContent().build(); return doMarkSegments("markAsUsedAllNonOvershadowedSegments", dataSourceName, markSegments);
}
return Response.ok().build();
} }
/* When this method is removed, a new method needs to be introduced corresponding to @POST
the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters). @Path("/{dataSourceName}/markUsed")
Ultimately we want to have no method with kill parameter - @Produces(MediaType.APPLICATION_JSON)
DELETE `{dataSourceName}` will be used to disable datasource and @ResourceFilters(DatasourceResourceFilter.class)
DELETE `{dataSourceName}/intervals/{interval}` will be used to nuke segments 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<String> 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<String> 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 (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 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 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 @DELETE
@Deprecated @Deprecated
@Path("/{dataSourceName}") @Path("/{dataSourceName}")
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
public Response deleteDataSource( public Response markAsUnusedAllSegmentsOrKillSegmentsInInterval(
@PathParam("dataSourceName") final String dataSourceName, @PathParam("dataSourceName") final String dataSourceName,
@QueryParam("kill") final String kill, @QueryParam("kill") final String kill,
@QueryParam("interval") final String interval @QueryParam("interval") final String interval
@ -211,47 +297,20 @@ public class DataSourcesResource
return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); return Response.ok(ImmutableMap.of("error", "no indexing service found")).build();
} }
if (kill != null && Boolean.valueOf(kill)) { boolean killSegments = kill != null && Boolean.valueOf(kill);
try { if (killSegments) {
indexingServiceClient.killSegments(dataSourceName, Intervals.of(interval)); return killSegmentsInInterval(dataSourceName, 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();
}
} else { } else {
if (!databaseSegmentManager.removeDataSource(dataSourceName)) { MarkSegments markSegments = () -> segmentsMetadata.markAsUnusedAllSegmentsInDataSource(dataSourceName);
return Response.noContent().build(); return doMarkSegments("markAsUnusedAllSegments", dataSourceName, markSegments);
}
} }
return Response.ok().build();
} }
@DELETE @DELETE
@Path("/{dataSourceName}/intervals/{interval}") @Path("/{dataSourceName}/intervals/{interval}")
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
public Response deleteDataSourceSpecificInterval( public Response killSegmentsInInterval(
@PathParam("dataSourceName") final String dataSourceName, @PathParam("dataSourceName") final String dataSourceName,
@PathParam("interval") final String interval @PathParam("interval") final String interval
) )
@ -259,28 +318,32 @@ public class DataSourcesResource
if (indexingServiceClient == null) { if (indexingServiceClient == null) {
return Response.ok(ImmutableMap.of("error", "no indexing service found")).build(); 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('_', '/')); final Interval theInterval = Intervals.of(interval.replace('_', '/'));
try { try {
indexingServiceClient.killSegments(dataSourceName, theInterval); indexingServiceClient.killSegments(dataSourceName, theInterval);
return Response.ok().build();
} }
catch (Exception e) { catch (Exception e) {
return Response.serverError() return Response
.entity(ImmutableMap.of( .serverError()
"error", .entity(
"Exception occurred. Are you sure you have an indexing service?", ImmutableMap.of(
"message", "error", "Exception occurred. Are you sure you have an indexing service?",
e.toString() "message", e.toString()
)) )
.build(); )
.build();
} }
return Response.ok().build();
} }
@GET @GET
@Path("/{dataSourceName}/intervals") @Path("/{dataSourceName}/intervals")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getSegmentDataSourceIntervals( public Response getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@QueryParam("simple") String simple, @QueryParam("simple") String simple,
@QueryParam("full") String full @QueryParam("full") String full
@ -289,21 +352,22 @@ public class DataSourcesResource
if (simple == null && full == null) { if (simple == null && full == null) {
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.noContent().build(); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
final Comparator<Interval> comparator = Comparators.intervalsByStartThenEnd().reversed(); final Comparator<Interval> comparator = Comparators.intervalsByStartThenEnd().reversed();
Set<Interval> intervals = new TreeSet<>(comparator); Set<Interval> intervals = new TreeSet<>(comparator);
dataSource.getSegments().forEach(segment -> intervals.add(segment.getInterval())); dataSource.getSegments().forEach(segment -> intervals.add(segment.getInterval()));
return Response.ok(intervals).build(); return Response.ok(intervals).build();
} else {
return getServedSegmentsInInterval(dataSourceName, full != null, interval -> true);
} }
return getSegmentDataSourceIntervals(dataSourceName, full != null, interval -> true);
} }
@GET @GET
@Path("/{dataSourceName}/intervals/{interval}") @Path("/{dataSourceName}/intervals/{interval}")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getSegmentDataSourceSpecificInterval( public Response getServedSegmentsInInterval(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@PathParam("interval") String interval, @PathParam("interval") String interval,
@QueryParam("simple") String simple, @QueryParam("simple") String simple,
@ -314,7 +378,7 @@ public class DataSourcesResource
if (simple == null && full == null) { if (simple == null && full == null) {
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.noContent().build(); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
final Set<SegmentId> segmentIds = new TreeSet<>(); final Set<SegmentId> segmentIds = new TreeSet<>();
for (DataSegment dataSegment : dataSource.getSegments()) { for (DataSegment dataSegment : dataSource.getSegments()) {
@ -324,7 +388,7 @@ public class DataSourcesResource
} }
return Response.ok(segmentIds).build(); return Response.ok(segmentIds).build();
} }
return getSegmentDataSourceIntervals(dataSourceName, full != null, theInterval::contains); return getServedSegmentsInInterval(dataSourceName, full != null, theInterval::contains);
} }
/** /**
@ -337,7 +401,7 @@ public class DataSourcesResource
count count
} }
private Response getSegmentDataSourceIntervals( private Response getServedSegmentsInInterval(
String dataSourceName, String dataSourceName,
boolean full, boolean full,
Predicate<Interval> intervalFilter Predicate<Interval> intervalFilter
@ -346,7 +410,7 @@ public class DataSourcesResource
final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.noContent().build(); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
final Comparator<Interval> comparator = Comparators.intervalsByStartThenEnd().reversed(); final Comparator<Interval> comparator = Comparators.intervalsByStartThenEnd().reversed();
@ -357,10 +421,13 @@ public class DataSourcesResource
if (intervalFilter.test(dataSegment.getInterval())) { if (intervalFilter.test(dataSegment.getInterval())) {
Map<SegmentId, Object> segments = retVal.computeIfAbsent(dataSegment.getInterval(), i -> new HashMap<>()); Map<SegmentId, Object> segments = retVal.computeIfAbsent(dataSegment.getInterval(), i -> new HashMap<>());
Pair<DataSegment, Set<String>> val = getServersWhereSegmentIsServed(dataSegment.getId()); Pair<DataSegment, Set<String>> segmentAndServers = getServersWhereSegmentIsServed(dataSegment.getId());
if (val != null) { if (segmentAndServers != null) {
segments.put(dataSegment.getId(), ImmutableMap.of("metadata", val.lhs, "servers", val.rhs)); segments.put(
dataSegment.getId(),
ImmutableMap.of("metadata", segmentAndServers.lhs, "servers", segmentAndServers.rhs)
);
} }
} }
} }
@ -385,14 +452,14 @@ public class DataSourcesResource
@Path("/{dataSourceName}/segments") @Path("/{dataSourceName}/segments")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getSegmentDataSourceSegments( public Response getAllServedSegments(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@QueryParam("full") String full @QueryParam("full") String full
) )
{ {
ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.noContent().build(); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
Response.ResponseBuilder builder = Response.ok(); Response.ResponseBuilder builder = Response.ok();
@ -407,14 +474,14 @@ public class DataSourcesResource
@Path("/{dataSourceName}/segments/{segmentId}") @Path("/{dataSourceName}/segments/{segmentId}")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getSegmentDataSourceSegment( public Response getServedSegment(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@PathParam("segmentId") String segmentId @PathParam("segmentId") String segmentId
) )
{ {
ImmutableDruidDataSource dataSource = getDataSource(dataSourceName); ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.noContent().build(); return logAndCreateDataSourceNotFoundResponse(dataSourceName);
} }
for (SegmentId possibleSegmentId : SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId)) { for (SegmentId possibleSegmentId : SegmentId.iteratePossibleParsingsWithDataSource(dataSourceName, segmentId)) {
@ -423,44 +490,40 @@ public class DataSourcesResource
return Response.ok(ImmutableMap.of("metadata", retVal.lhs, "servers", retVal.rhs)).build(); return Response.ok(ImmutableMap.of("metadata", retVal.lhs, "servers", retVal.rhs)).build();
} }
} }
log.warn("Segment id [%s] is unknown", segmentId);
return Response.noContent().build(); return Response.noContent().build();
} }
@DELETE @DELETE
@Path("/{dataSourceName}/segments/{segmentId}") @Path("/{dataSourceName}/segments/{segmentId}")
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response deleteDatasourceSegment( public Response markSegmentAsUnused(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@PathParam("segmentId") String segmentId @PathParam("segmentId") String segmentId
) )
{ {
if (databaseSegmentManager.removeSegment(segmentId)) { boolean segmentStateChanged = segmentsMetadata.markSegmentAsUnused(segmentId);
return Response.ok().build(); return Response.ok(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build();
}
return Response.noContent().build();
} }
@POST @POST
@Path("/{dataSourceName}/segments/{segmentId}") @Path("/{dataSourceName}/segments/{segmentId}")
@Consumes(MediaType.APPLICATION_JSON) @Consumes(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response enableDatasourceSegment( public Response markSegmentAsUsed(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@PathParam("segmentId") String segmentId @PathParam("segmentId") String segmentId
) )
{ {
if (!databaseSegmentManager.enableSegment(segmentId)) { boolean segmentStateChanged = segmentsMetadata.markSegmentAsUsed(segmentId);
return Response.noContent().build(); return Response.ok().entity(ImmutableMap.of("segmentStateChanged", segmentStateChanged)).build();
}
return Response.ok().build();
} }
@GET @GET
@Path("/{dataSourceName}/tiers") @Path("/{dataSourceName}/tiers")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getSegmentDataSourceTiers(@PathParam("dataSourceName") String dataSourceName) public Response getTiersWhereSegmentsAreServed(@PathParam("dataSourceName") String dataSourceName)
{ {
Set<String> retVal = new HashSet<>(); Set<String> retVal = new HashSet<>();
for (DruidServer druidServer : serverInventoryView.getInventory()) { for (DruidServer druidServer : serverInventoryView.getInventory()) {
@ -475,20 +538,22 @@ public class DataSourcesResource
@Nullable @Nullable
private ImmutableDruidDataSource getDataSource(final String dataSourceName) private ImmutableDruidDataSource getDataSource(final String dataSourceName)
{ {
List<ImmutableDruidDataSource> dataSources = serverInventoryView List<DruidDataSource> dataSources = serverInventoryView
.getInventory() .getInventory()
.stream() .stream()
.map(server -> server.getDataSource(dataSourceName)) .map(server -> server.getDataSource(dataSourceName))
.filter(Objects::nonNull) .filter(Objects::nonNull)
.map(DruidDataSource::toImmutableDruidDataSource)
.collect(Collectors.toList()); .collect(Collectors.toList());
if (dataSources.isEmpty()) { if (dataSources.isEmpty()) {
return null; 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<SegmentId, DataSegment> segmentMap = new TreeMap<>(); final SortedMap<SegmentId, DataSegment> segmentMap = new TreeMap<>();
for (ImmutableDruidDataSource dataSource : dataSources) { for (DruidDataSource dataSource : dataSources) {
Iterable<DataSegment> segments = dataSource.getSegments(); Iterable<DataSegment> segments = dataSource.getSegments();
for (DataSegment segment : segments) { for (DataSegment segment : segments) {
segmentMap.put(segment.getId(), segment); segmentMap.put(segment.getId(), segment);
@ -551,7 +616,7 @@ public class DataSourcesResource
continue; continue;
} }
tierDistinctSegments.computeIfAbsent(tier, k -> new HashSet<>()); tierDistinctSegments.computeIfAbsent(tier, t -> new HashSet<>());
long dataSourceSegmentSize = 0; long dataSourceSegmentSize = 0;
long replicatedSegmentSize = 0; long replicatedSegmentSize = 0;
@ -597,14 +662,14 @@ public class DataSourcesResource
} }
/** /**
* Provides serverView for a datasource and Interval which gives details about servers hosting segments for an interval * Provides serverView for a datasource and Interval which gives details about servers hosting segments for an
* Used by the realtime tasks to fetch a view of the interval they are interested in. * interval. Used by the realtime tasks to fetch a view of the interval they are interested in.
*/ */
@GET @GET
@Path("/{dataSourceName}/intervals/{interval}/serverview") @Path("/{dataSourceName}/intervals/{interval}/serverview")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getSegmentDataSourceSpecificInterval( public Response getServedSegmentsInInterval(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@PathParam("interval") String interval, @PathParam("interval") String interval,
@QueryParam("partial") final boolean partial @QueryParam("partial") final boolean partial
@ -619,17 +684,25 @@ public class DataSourcesResource
return Response.ok(new ArrayList<ImmutableSegmentLoadInfo>()).build(); return Response.ok(new ArrayList<ImmutableSegmentLoadInfo>()).build();
} }
Iterable<TimelineObjectHolder<String, SegmentLoadInfo>> lookup = timeline.lookupWithIncompletePartitions(theInterval); return Response.ok(prepareServedSegmentsInInterval(timeline, theInterval)).build();
FunctionalIterable<ImmutableSegmentLoadInfo> retval = FunctionalIterable }
.create(lookup).transformCat(
private Iterable<ImmutableSegmentLoadInfo> prepareServedSegmentsInInterval(
TimelineLookup<String, SegmentLoadInfo> dataSourceServingTimeline,
Interval interval
)
{
Iterable<TimelineObjectHolder<String, SegmentLoadInfo>> lookup =
dataSourceServingTimeline.lookupWithIncompletePartitions(interval);
return FunctionalIterable
.create(lookup)
.transformCat(
(TimelineObjectHolder<String, SegmentLoadInfo> input) -> (TimelineObjectHolder<String, SegmentLoadInfo> input) ->
Iterables.transform( Iterables.transform(
input.getObject(), input.getObject(),
(PartitionChunk<SegmentLoadInfo> chunk) -> (PartitionChunk<SegmentLoadInfo> chunk) -> chunk.getObject().toImmutableSegmentLoadInfo()
chunk.getObject().toImmutableSegmentLoadInfo()
) )
); );
return Response.ok(retval).build();
} }
/** /**
@ -648,7 +721,7 @@ public class DataSourcesResource
) )
{ {
try { try {
final List<Rule> rules = databaseRuleManager.getRulesWithDefault(dataSourceName); final List<Rule> rules = metadataRuleManager.getRulesWithDefault(dataSourceName);
final Interval theInterval = Intervals.of(interval); final Interval theInterval = Intervals.of(interval);
final SegmentDescriptor descriptor = new SegmentDescriptor(theInterval, version, partitionNumber); final SegmentDescriptor descriptor = new SegmentDescriptor(theInterval, version, partitionNumber);
final DateTime now = DateTimes.nowUtc(); final DateTime now = DateTimes.nowUtc();
@ -675,18 +748,9 @@ public class DataSourcesResource
return Response.ok(false).build(); return Response.ok(false).build();
} }
Iterable<TimelineObjectHolder<String, SegmentLoadInfo>> lookup = timeline.lookupWithIncompletePartitions( Iterable<ImmutableSegmentLoadInfo> servedSegmentsInInterval =
theInterval); prepareServedSegmentsInInterval(timeline, theInterval);
FunctionalIterable<ImmutableSegmentLoadInfo> loadInfoIterable = FunctionalIterable if (isSegmentLoaded(servedSegmentsInInterval, descriptor)) {
.create(lookup).transformCat(
(TimelineObjectHolder<String, SegmentLoadInfo> input) ->
Iterables.transform(
input.getObject(),
(PartitionChunk<SegmentLoadInfo> chunk) ->
chunk.getObject().toImmutableSegmentLoadInfo()
)
);
if (isSegmentLoaded(loadInfoIterable, descriptor)) {
return Response.ok(true).build(); return Response.ok(true).build();
} }
@ -698,58 +762,9 @@ public class DataSourcesResource
} }
} }
@POST static boolean isSegmentLoaded(Iterable<ImmutableSegmentLoadInfo> servedSegments, SegmentDescriptor descriptor)
@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
)
{ {
if (payload == null || !payload.isValid()) { for (ImmutableSegmentLoadInfo segmentLoadInfo : servedSegments) {
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<String> 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<ImmutableSegmentLoadInfo> serverView, SegmentDescriptor descriptor)
{
for (ImmutableSegmentLoadInfo segmentLoadInfo : serverView) {
if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval()) if (segmentLoadInfo.getSegment().getInterval().contains(descriptor.getInterval())
&& segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber() && segmentLoadInfo.getSegment().getShardSpec().getPartitionNum() == descriptor.getPartitionNumber()
&& segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0 && segmentLoadInfo.getSegment().getVersion().compareTo(descriptor.getVersion()) >= 0
@ -762,68 +777,14 @@ public class DataSourcesResource
return false; 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 @VisibleForTesting
protected static class MarkDatasourceSegmentsPayload protected static class MarkDataSourceSegmentsPayload
{ {
private final Interval interval; private final Interval interval;
private final Set<String> segmentIds; private final Set<String> segmentIds;
@JsonCreator @JsonCreator
public MarkDatasourceSegmentsPayload( public MarkDataSourceSegmentsPayload(
@JsonProperty("interval") Interval interval, @JsonProperty("interval") Interval interval,
@JsonProperty("segmentIds") Set<String> segmentIds @JsonProperty("segmentIds") Set<String> segmentIds
) )

View File

@ -23,15 +23,15 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.collect.Collections2; import com.google.common.collect.Collections2;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.inject.Inject; import com.google.inject.Inject;
import com.sun.jersey.spi.container.ResourceFilters; import com.sun.jersey.spi.container.ResourceFilters;
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Json;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.metadata.MetadataSegmentManager; 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.http.security.DatasourceResourceFilter;
import org.apache.druid.server.security.AuthConfig;
import org.apache.druid.server.security.AuthorizationUtils; import org.apache.druid.server.security.AuthorizationUtils;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.ResourceAction; import org.apache.druid.server.security.ResourceAction;
@ -40,6 +40,7 @@ import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.SegmentWithOvershadowedStatus; import org.apache.druid.timeline.SegmentWithOvershadowedStatus;
import org.joda.time.Interval; import org.joda.time.Interval;
import javax.annotation.Nullable;
import javax.servlet.http.HttpServletRequest; import javax.servlet.http.HttpServletRequest;
import javax.ws.rs.GET; import javax.ws.rs.GET;
import javax.ws.rs.POST; import javax.ws.rs.POST;
@ -50,12 +51,13 @@ import javax.ws.rs.QueryParam;
import javax.ws.rs.core.Context; import javax.ws.rs.core.Context;
import javax.ws.rs.core.MediaType; import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response; import javax.ws.rs.core.Response;
import javax.ws.rs.core.UriInfo;
import java.util.Collection; import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Optional;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream; import java.util.stream.Stream;
/** /**
@ -63,20 +65,19 @@ import java.util.stream.Stream;
@Path("/druid/coordinator/v1/metadata") @Path("/druid/coordinator/v1/metadata")
public class MetadataResource public class MetadataResource
{ {
private final MetadataSegmentManager metadataSegmentManager; private final MetadataSegmentManager segmentsMetadata;
private final IndexerMetadataStorageCoordinator metadataStorageCoordinator; private final IndexerMetadataStorageCoordinator metadataStorageCoordinator;
private final AuthorizerMapper authorizerMapper; private final AuthorizerMapper authorizerMapper;
@Inject @Inject
public MetadataResource( public MetadataResource(
MetadataSegmentManager metadataSegmentManager, MetadataSegmentManager segmentsMetadata,
IndexerMetadataStorageCoordinator metadataStorageCoordinator, IndexerMetadataStorageCoordinator metadataStorageCoordinator,
AuthConfig authConfig,
AuthorizerMapper authorizerMapper, AuthorizerMapper authorizerMapper,
@Json ObjectMapper jsonMapper @Json ObjectMapper jsonMapper
) )
{ {
this.metadataSegmentManager = metadataSegmentManager; this.segmentsMetadata = segmentsMetadata;
this.metadataStorageCoordinator = metadataStorageCoordinator; this.metadataStorageCoordinator = metadataStorageCoordinator;
this.authorizerMapper = authorizerMapper; this.authorizerMapper = authorizerMapper;
} }
@ -84,29 +85,28 @@ public class MetadataResource
@GET @GET
@Path("/datasources") @Path("/datasources")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
public Response getDatabaseDataSources( public Response getDataSources(
@QueryParam("full") final String full, @QueryParam("full") final String full,
@QueryParam("includeDisabled") final String includeDisabled, @Context final UriInfo uriInfo,
@Context final HttpServletRequest req @Context final HttpServletRequest req
) )
{ {
// If we haven't polled the metadata store yet, use an empty list of datasources. final boolean includeUnused = JettyUtils.getQueryParam(uriInfo, "includeUnused", "includeDisabled") != null;
final Collection<ImmutableDruidDataSource> druidDataSources = Optional.ofNullable(metadataSegmentManager.getDataSources()) Collection<ImmutableDruidDataSource> druidDataSources = null;
.orElse(Collections.emptyList());
final Set<String> dataSourceNamesPreAuth; final Set<String> dataSourceNamesPreAuth;
if (includeDisabled != null) { if (includeUnused) {
dataSourceNamesPreAuth = new TreeSet<>(metadataSegmentManager.getAllDataSourceNames()); dataSourceNamesPreAuth = new TreeSet<>(segmentsMetadata.retrieveAllDataSourceNames());
} else { } else {
dataSourceNamesPreAuth = Sets.newTreeSet( druidDataSources = segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments();
Iterables.transform(druidDataSources, ImmutableDruidDataSource::getName) dataSourceNamesPreAuth = druidDataSources
); .stream()
.map(ImmutableDruidDataSource::getName)
.collect(Collectors.toCollection(TreeSet::new));
} }
final Set<String> dataSourceNamesPostAuth = new TreeSet<>(); final Set<String> dataSourceNamesPostAuth = new TreeSet<>();
Function<String, Iterable<ResourceAction>> raGenerator = datasourceName -> { Function<String, Iterable<ResourceAction>> raGenerator = datasourceName ->
return Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName));
};
Iterables.addAll( Iterables.addAll(
dataSourceNamesPostAuth, dataSourceNamesPostAuth,
@ -118,9 +118,9 @@ public class MetadataResource
) )
); );
// 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 // Always use dataSourceNamesPostAuth to determine the set of returned dataSources
if (full != null && includeDisabled == null) { if (full != null && !includeUnused) {
return Response.ok().entity( return Response.ok().entity(
Collections2.filter(druidDataSources, dataSource -> dataSourceNamesPostAuth.contains(dataSource.getName())) Collections2.filter(druidDataSources, dataSource -> dataSourceNamesPostAuth.contains(dataSource.getName()))
).build(); ).build();
@ -129,74 +129,61 @@ public class MetadataResource
} }
} }
@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 @GET
@Path("/segments") @Path("/segments")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
public Response getDatabaseSegments( public Response getAllUsedSegments(
@Context final HttpServletRequest req, @Context final HttpServletRequest req,
@QueryParam("datasources") final Set<String> datasources, @QueryParam("datasources") final @Nullable Set<String> dataSources,
@QueryParam("includeOvershadowedStatus") final String includeOvershadowedStatus @QueryParam("includeOvershadowedStatus") final String includeOvershadowedStatus
) )
{ {
// If we haven't polled the metadata store yet, use an empty list of datasources.
Collection<ImmutableDruidDataSource> druidDataSources = Optional.ofNullable(metadataSegmentManager.getDataSources())
.orElse(Collections.emptyList());
Stream<ImmutableDruidDataSource> dataSourceStream = druidDataSources.stream();
if (datasources != null && !datasources.isEmpty()) {
dataSourceStream = dataSourceStream.filter(src -> datasources.contains(src.getName()));
}
final Stream<DataSegment> metadataSegments = dataSourceStream.flatMap(t -> t.getSegments().stream());
if (includeOvershadowedStatus != null) { if (includeOvershadowedStatus != null) {
final Iterable<SegmentWithOvershadowedStatus> authorizedSegments = return getAllUsedSegmentsWithOvershadowedStatus(req, dataSources);
findAuthorizedSegmentWithOvershadowedStatus(
req,
metadataSegments
);
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
return builder.entity(authorizedSegments).build();
} else {
final Function<DataSegment, Iterable<ResourceAction>> raGenerator = segment -> Collections.singletonList(
AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
final Iterable<DataSegment> authorizedSegments = AuthorizationUtils.filterAuthorizedResources(
req,
metadataSegments::iterator,
raGenerator,
authorizerMapper
);
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
return builder.entity(authorizedSegments).build();
} }
Collection<ImmutableDruidDataSource> dataSourcesWithUsedSegments =
segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments();
if (dataSources != null && !dataSources.isEmpty()) {
dataSourcesWithUsedSegments = dataSourcesWithUsedSegments
.stream()
.filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName()))
.collect(Collectors.toList());
}
final Stream<DataSegment> usedSegments = dataSourcesWithUsedSegments
.stream()
.flatMap(t -> t.getSegments().stream());
final Function<DataSegment, Iterable<ResourceAction>> raGenerator = segment -> Collections.singletonList(
AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(segment.getDataSource()));
final Iterable<DataSegment> authorizedSegments =
AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper);
Response.ResponseBuilder builder = Response.status(Response.Status.OK);
return builder.entity(authorizedSegments).build();
} }
private Iterable<SegmentWithOvershadowedStatus> findAuthorizedSegmentWithOvershadowedStatus( private Response getAllUsedSegmentsWithOvershadowedStatus(
HttpServletRequest req, HttpServletRequest req,
Stream<DataSegment> metadataSegments @Nullable Set<String> dataSources
) )
{ {
// If metadata store hasn't been polled yet, use empty overshadowed list DataSourcesSnapshot dataSourcesSnapshot = segmentsMetadata.getSnapshotOfDataSourcesWithAllUsedSegments();
final Set<SegmentId> overshadowedSegments = Optional Collection<ImmutableDruidDataSource> dataSourcesWithUsedSegments =
.ofNullable(metadataSegmentManager.getOvershadowedSegments()) dataSourcesSnapshot.getDataSourcesWithAllUsedSegments();
.orElse(Collections.emptySet()); if (dataSources != null && !dataSources.isEmpty()) {
dataSourcesWithUsedSegments = dataSourcesWithUsedSegments
.stream()
.filter(dataSourceWithUsedSegments -> dataSources.contains(dataSourceWithUsedSegments.getName()))
.collect(Collectors.toList());
}
final Stream<DataSegment> usedSegments = dataSourcesWithUsedSegments
.stream()
.flatMap(t -> t.getSegments().stream());
final Set<SegmentId> overshadowedSegments = dataSourcesSnapshot.getOvershadowedSegments();
final Stream<SegmentWithOvershadowedStatus> segmentsWithOvershadowedStatus = metadataSegments final Stream<SegmentWithOvershadowedStatus> usedSegmentsWithOvershadowedStatus = usedSegments
.map(segment -> new SegmentWithOvershadowedStatus( .map(segment -> new SegmentWithOvershadowedStatus(
segment, segment,
overshadowedSegments.contains(segment.getId()) overshadowedSegments.contains(segment.getId())
@ -207,23 +194,45 @@ public class MetadataResource
final Iterable<SegmentWithOvershadowedStatus> authorizedSegments = AuthorizationUtils.filterAuthorizedResources( final Iterable<SegmentWithOvershadowedStatus> authorizedSegments = AuthorizationUtils.filterAuthorizedResources(
req, req,
segmentsWithOvershadowedStatus::iterator, usedSegmentsWithOvershadowedStatus::iterator,
raGenerator, raGenerator,
authorizerMapper 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 @GET
@Path("/datasources/{dataSourceName}/segments") @Path("/datasources/{dataSourceName}/segments")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getDatabaseSegmentDataSourceSegments( public Response getUsedSegmentsInDataSource(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@QueryParam("full") String full @QueryParam("full") String full
) )
{ {
ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); ImmutableDruidDataSource dataSource =
segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build(); return Response.status(Response.Status.NOT_FOUND).build();
} }
@ -236,11 +245,15 @@ public class MetadataResource
return builder.entity(Collections2.transform(dataSource.getSegments(), DataSegment::getId)).build(); 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 @POST
@Path("/datasources/{dataSourceName}/segments") @Path("/datasources/{dataSourceName}/segments")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getDatabaseSegmentDataSourceSegments( public Response getUsedSegmentsInDataSourceForIntervals(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@QueryParam("full") String full, @QueryParam("full") String full,
List<Interval> intervals List<Interval> intervals
@ -260,12 +273,12 @@ public class MetadataResource
@Path("/datasources/{dataSourceName}/segments/{segmentId}") @Path("/datasources/{dataSourceName}/segments/{segmentId}")
@Produces(MediaType.APPLICATION_JSON) @Produces(MediaType.APPLICATION_JSON)
@ResourceFilters(DatasourceResourceFilter.class) @ResourceFilters(DatasourceResourceFilter.class)
public Response getDatabaseSegmentDataSourceSegment( public Response isSegmentUsed(
@PathParam("dataSourceName") String dataSourceName, @PathParam("dataSourceName") String dataSourceName,
@PathParam("segmentId") String segmentId @PathParam("segmentId") String segmentId
) )
{ {
ImmutableDruidDataSource dataSource = metadataSegmentManager.getDataSource(dataSourceName); ImmutableDruidDataSource dataSource = segmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSourceName);
if (dataSource == null) { if (dataSource == null) {
return Response.status(Response.Status.NOT_FOUND).build(); return Response.status(Response.Status.NOT_FOUND).build();
} }

View File

@ -2198,7 +2198,8 @@ public class CachingClusteredClientTest
serverExpectationList.add(serverExpectations); serverExpectationList.add(serverExpectations);
for (int j = 0; j < numChunks; ++j) { for (int j = 0; j < numChunks; ++j) {
DruidServer lastServer = servers[random.nextInt(servers.length)]; 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); DataSegment mockSegment = makeMock(mocks, DataSegment.class);
ServerExpectation<Object> expectation = new ServerExpectation<>( ServerExpectation<Object> expectation = new ServerExpectation<>(

View File

@ -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<DataSegment> segments)
{
EasyMock.expect(mockServer.iterateAllSegments()).andReturn(segments).anyTimes();
EasyMock.expect(mockServer.getNumSegments()).andReturn(segments.size()).anyTimes();
}
private ImmutableDruidServerTests()
{
}
}

View File

@ -32,35 +32,35 @@ public class ClientKillQueryTest
private static final String DATA_SOURCE = "data_source"; private static final String DATA_SOURCE = "data_source";
public static final DateTime START = DateTimes.nowUtc(); public static final DateTime START = DateTimes.nowUtc();
private static final Interval INTERVAL = new Interval(START, START.plus(1)); private static final Interval INTERVAL = new Interval(START, START.plus(1));
ClientKillQuery clientKillQuery; ClientKillQuery clientKillUnusedSegmentsQuery;
@Before @Before
public void setUp() public void setUp()
{ {
clientKillQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL); clientKillUnusedSegmentsQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL);
} }
@After @After
public void tearDown() public void tearDown()
{ {
clientKillQuery = null; clientKillUnusedSegmentsQuery = null;
} }
@Test @Test
public void testGetType() public void testGetType()
{ {
Assert.assertEquals("kill", clientKillQuery.getType()); Assert.assertEquals("kill", clientKillUnusedSegmentsQuery.getType());
} }
@Test @Test
public void testGetDataSource() public void testGetDataSource()
{ {
Assert.assertEquals(DATA_SOURCE, clientKillQuery.getDataSource()); Assert.assertEquals(DATA_SOURCE, clientKillUnusedSegmentsQuery.getDataSource());
} }
@Test @Test
public void testGetInterval() public void testGetInterval()
{ {
Assert.assertEquals(INTERVAL, clientKillQuery.getInterval()); Assert.assertEquals(INTERVAL, clientKillUnusedSegmentsQuery.getInterval());
} }
} }

View File

@ -41,13 +41,7 @@ public class TestUsedSegmentChecker implements UsedSegmentChecker
public Set<DataSegment> findUsedSegments(Set<SegmentIdWithShardSpec> identifiers) public Set<DataSegment> findUsedSegments(Set<SegmentIdWithShardSpec> identifiers)
{ {
final VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(Ordering.natural()); final VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(Ordering.natural());
for (DataSegment dataSegment : appenderatorTester.getPushedSegments()) { VersionedIntervalTimeline.addSegments(timeline, appenderatorTester.getPushedSegments().iterator());
timeline.add(
dataSegment.getInterval(),
dataSegment.getVersion(),
dataSegment.getShardSpec().createChunk(dataSegment)
);
}
final Set<DataSegment> retVal = new HashSet<>(); final Set<DataSegment> retVal = new HashSet<>();
for (SegmentIdWithShardSpec identifier : identifiers) { for (SegmentIdWithShardSpec identifier : identifiers) {

View File

@ -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()
{
}
}

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer; 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.DateTimes;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.DruidServerMetadata;
@ -94,7 +95,7 @@ public class CostBalancerStrategyTest
segments.put(segment.getId(), segment); segments.put(segment.getId(), segment);
EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); 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); EasyMock.replay(druidServer);
serverHolderList.add(new ServerHolder(druidServer, fromPeon)); serverHolderList.add(new ServerHolder(druidServer, fromPeon));

View File

@ -83,7 +83,10 @@ import java.util.concurrent.atomic.AtomicReference;
public class CuratorDruidCoordinatorTest extends CuratorTestBase public class CuratorDruidCoordinatorTest extends CuratorTestBase
{ {
private DruidCoordinator coordinator; private DruidCoordinator coordinator;
private MetadataSegmentManager databaseSegmentManager; private MetadataSegmentManager segmentsMetadata;
private DataSourcesSnapshot dataSourcesSnapshot;
private DruidCoordinatorRuntimeParams coordinatorRuntimeParams;
private ScheduledExecutorFactory scheduledExecutorFactory; private ScheduledExecutorFactory scheduledExecutorFactory;
private ConcurrentMap<String, LoadQueuePeon> loadManagementPeons; private ConcurrentMap<String, LoadQueuePeon> loadManagementPeons;
private LoadQueuePeon sourceLoadQueuePeon; private LoadQueuePeon sourceLoadQueuePeon;
@ -97,7 +100,6 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
private ObjectMapper objectMapper; private ObjectMapper objectMapper;
private JacksonConfigManager configManager; private JacksonConfigManager configManager;
private DruidNode druidNode; private DruidNode druidNode;
private DataSourcesSnapshot dataSourcesSnapshot;
private static final String SEGPATH = "/druid/segments"; private static final String SEGPATH = "/druid/segments";
private static final String SOURCE_LOAD_PATH = "/druid/loadQueue/localhost:1"; private static final String SOURCE_LOAD_PATH = "/druid/loadQueue/localhost:1";
private static final String DESTINATION_LOAD_PATH = "/druid/loadQueue/localhost:2"; private static final String DESTINATION_LOAD_PATH = "/druid/loadQueue/localhost:2";
@ -126,17 +128,19 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
@Before @Before
public void setUp() throws Exception 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); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class);
configManager = EasyMock.createNiceMock(JacksonConfigManager.class); configManager = EasyMock.createNiceMock(JacksonConfigManager.class);
dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class);
EasyMock.expect( EasyMock.expect(
configManager.watch( configManager.watch(
EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY),
EasyMock.anyObject(Class.class), EasyMock.anyObject(Class.class),
EasyMock.anyObject() EasyMock.anyObject()
) )
).andReturn(new AtomicReference(CoordinatorDynamicConfig.builder().build())).anyTimes(); ).andReturn(new AtomicReference<>(CoordinatorDynamicConfig.builder().build())).anyTimes();
EasyMock.expect( EasyMock.expect(
configManager.watch( configManager.watch(
EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY), EasyMock.eq(CoordinatorCompactionConfig.CONFIG_KEY),
@ -212,7 +216,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
} }
}, },
configManager, configManager,
databaseSegmentManager, segmentsMetadata,
baseView, baseView,
metadataRuleManager, metadataRuleManager,
curator, curator,
@ -365,13 +369,15 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class);
EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(sourceSegments.get(2)); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(sourceSegments.get(2));
EasyMock.replay(druidDataSource); EasyMock.replay(druidDataSource);
EasyMock.expect(databaseSegmentManager.getDataSource(EasyMock.anyString())).andReturn(druidDataSource); EasyMock.expect(segmentsMetadata.getImmutableDataSourceWithUsedSegments(EasyMock.anyString()))
EasyMock.replay(databaseSegmentManager); .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.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes();
EasyMock.replay(dataSourcesSnapshot); EasyMock.replay(dataSourcesSnapshot);
coordinator.moveSegment( coordinator.moveSegment(
coordinatorRuntimeParams,
source.toImmutableDruidServer(), source.toImmutableDruidServer(),
dest.toImmutableDruidServer(), dest.toImmutableDruidServer(),
sourceSegments.get(2), sourceSegments.get(2),
@ -498,7 +504,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
} }
}, },
configManager, configManager,
databaseSegmentManager, segmentsMetadata,
baseView, baseView,
metadataRuleManager, metadataRuleManager,
curator, curator,
@ -535,14 +541,7 @@ public class CuratorDruidCoordinatorTest extends CuratorTestBase
return DataSegment.builder() return DataSegment.builder()
.dataSource("test_curator_druid_coordinator") .dataSource("test_curator_druid_coordinator")
.interval(Intervals.of(intervalStr)) .interval(Intervals.of(intervalStr))
.loadSpec( .loadSpec(ImmutableMap.of("type", "local", "path", "somewhere"))
ImmutableMap.of(
"type",
"local",
"path",
"somewhere"
)
)
.version(version) .version(version)
.dimensions(ImmutableList.of()) .dimensions(ImmutableList.of())
.metrics(ImmutableList.of()) .metrics(ImmutableList.of())

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.client.ImmutableDruidServerTests;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.server.coordination.DruidServerMetadata;
import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordination.ServerType;
@ -88,7 +89,7 @@ public class DiskNormalizedCostBalancerStrategyTest
segments.add(segment); segments.add(segment);
EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes(); EasyMock.expect(druidServer.getSegment(segment.getId())).andReturn(segment).anyTimes();
} }
EasyMock.expect(druidServer.getLazyAllSegments()).andReturn(segments).anyTimes(); ImmutableDruidServerTests.expectSegments(druidServer, segments);
EasyMock.replay(druidServer); EasyMock.replay(druidServer);
serverHolderList.add(new ServerHolder(druidServer, fromPeon)); serverHolderList.add(new ServerHolder(druidServer, fromPeon));

View File

@ -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<ServerHolder> realtimes = null;
private final Map<String, Iterable<ServerHolder>> 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);
}
}

View File

@ -21,7 +21,6 @@ package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
@ -39,9 +38,7 @@ import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.NavigableSet; import java.util.NavigableSet;
import java.util.Set; import java.util.Set;
import java.util.TreeSet;
import java.util.stream.Collectors; import java.util.stream.Collectors;
import java.util.stream.Stream;
public class DruidClusterTest public class DruidClusterTest
{ {
@ -100,8 +97,9 @@ public class DruidClusterTest
@Before @Before
public void setup() public void setup()
{ {
cluster = new DruidCluster( cluster = DruidClusterBuilder
ImmutableSet.of( .newBuilder()
.withRealtimes(
new ServerHolder( new ServerHolder(
new ImmutableDruidServer( new ImmutableDruidServer(
new DruidServerMetadata("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0), new DruidServerMetadata("name1", "host1", null, 100L, ServerType.REALTIME, "tier1", 0),
@ -111,22 +109,20 @@ public class DruidClusterTest
), ),
new LoadQueuePeonTester() new LoadQueuePeonTester()
) )
),
ImmutableMap.of(
"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())))
) )
); .addTier(
"tier1",
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 @Test

View File

@ -23,6 +23,7 @@ import com.google.common.base.Stopwatch;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.druid.client.ImmutableDruidServer; 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.DateTimes;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
@ -42,10 +43,10 @@ import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; 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 public class DruidCoordinatorBalancerProfiler
{ {
private static final int MAX_SEGMENTS_TO_MOVE = 5; private static final int MAX_SEGMENTS_TO_MOVE = 5;
@ -83,6 +84,7 @@ public class DruidCoordinatorBalancerProfiler
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject() EasyMock.anyObject()
); );
EasyMock.expectLastCall().anyTimes(); EasyMock.expectLastCall().anyTimes();
@ -116,9 +118,9 @@ public class DruidCoordinatorBalancerProfiler
EasyMock.expect(server.getName()).andReturn(Integer.toString(i)).atLeastOnce(); EasyMock.expect(server.getName()).andReturn(Integer.toString(i)).atLeastOnce();
EasyMock.expect(server.getHost()).andReturn(Integer.toString(i)).anyTimes(); EasyMock.expect(server.getHost()).andReturn(Integer.toString(i)).anyTimes();
if (i == 0) { if (i == 0) {
EasyMock.expect(server.getLazyAllSegments()).andReturn(segments).anyTimes(); ImmutableDruidServerTests.expectSegments(server, segments);
} else { } else {
EasyMock.expect(server.getLazyAllSegments()).andReturn(Collections.emptyList()).anyTimes(); ImmutableDruidServerTests.expectSegments(server, Collections.emptyList());
} }
EasyMock.expect(server.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes(); EasyMock.expect(server.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(server); EasyMock.replay(server);
@ -128,56 +130,26 @@ public class DruidCoordinatorBalancerProfiler
serverHolderList.add(new ServerHolder(server, peon)); serverHolderList.add(new ServerHolder(server, peon));
} }
DruidCoordinatorRuntimeParams params = DruidCluster druidCluster = DruidClusterBuilder
DruidCoordinatorRuntimeParams.newBuilder() .newBuilder()
.withDruidCluster( .addTier("normal", serverHolderList.toArray(new ServerHolder[0]))
new DruidCluster( .build();
null, DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
ImmutableMap.of( .newBuilder(druidCluster)
"normal", .withLoadManagementPeons(peonMap)
serverHolderList.stream().collect( .withUsedSegmentsInTest(segments)
Collectors.toCollection( .withDynamicConfigs(
() -> new TreeSet<>( CoordinatorDynamicConfig
DruidCoordinatorBalancerTester.percentUsedComparator .builder()
) .withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE)
) .withReplicantLifetime(500)
) .withReplicationThrottleLimit(5)
) .build()
) )
) .withEmitter(emitter)
.withLoadManagementPeons( .withDatabaseRuleManager(manager)
peonMap .withReplicationManager(new ReplicationThrottler(2, 500))
) .build();
.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();
DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator);
DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(coordinator); DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(coordinator);
@ -197,7 +169,7 @@ public class DruidCoordinatorBalancerProfiler
EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce(); EasyMock.expect(druidServer1.getName()).andReturn("from").atLeastOnce();
EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce();
EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).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.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer1); EasyMock.replay(druidServer1);
@ -205,7 +177,7 @@ public class DruidCoordinatorBalancerProfiler
EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes();
EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(0L).atLeastOnce();
EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).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.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer2); EasyMock.replay(druidServer2);
@ -213,47 +185,28 @@ public class DruidCoordinatorBalancerProfiler
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject() EasyMock.anyObject()
); );
EasyMock.expectLastCall().anyTimes(); EasyMock.expectLastCall().anyTimes();
EasyMock.replay(coordinator); EasyMock.replay(coordinator);
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
DruidCoordinatorRuntimeParams.newBuilder() .newBuilder()
.withDruidCluster( .withDruidCluster(
new DruidCluster( DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"normal", "normal",
Stream.of( new ServerHolder(druidServer1, fromPeon),
new ServerHolder(druidServer1, fromPeon), new ServerHolder(druidServer2, toPeon)
new ServerHolder(druidServer2, toPeon) )
).collect( .build()
Collectors.toCollection( )
() -> new TreeSet<>( .withLoadManagementPeons(ImmutableMap.of("from", fromPeon, "to", toPeon))
DruidCoordinatorBalancerTester.percentUsedComparator .withUsedSegmentsInTest(segments)
) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build())
) .build();
)
)
)
)
.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();
DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator); DruidCoordinatorBalancerTester tester = new DruidCoordinatorBalancerTester(coordinator);
watch.start(); watch.start();
DruidCoordinatorRuntimeParams balanceParams = tester.run(params); DruidCoordinatorRuntimeParams balanceParams = tester.run(params);

View File

@ -20,11 +20,11 @@
package org.apache.druid.server.coordinator; package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.ImmutableDruidServer; 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.DateTimes;
import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
@ -520,19 +520,19 @@ public class DruidCoordinatorBalancerTest
List<Boolean> decommissioning List<Boolean> decommissioning
) )
{ {
return DruidCoordinatorRuntimeParams return CoordinatorRuntimeParamsTestHelpers
.newBuilder() .newBuilder()
.withDruidCluster( .withDruidCluster(
new DruidCluster( DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"normal", "normal",
IntStream IntStream
.range(0, druidServers.size()) .range(0, druidServers.size())
.mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), decommissioning.get(i))) .mapToObj(i -> new ServerHolder(druidServers.get(i), peons.get(i), decommissioning.get(i)))
.collect(Collectors.toSet()) .toArray(ServerHolder[]::new)
) )
) .build()
) )
.withLoadManagementPeons( .withLoadManagementPeons(
IntStream IntStream
@ -540,14 +540,9 @@ public class DruidCoordinatorBalancerTest
.boxed() .boxed()
.collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get)) .collect(Collectors.toMap(i -> String.valueOf(i + 1), peons::get))
) )
.withAvailableSegmentsInTest(segments) .withUsedSegmentsInTest(segments)
.withDynamicConfigs( .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(MAX_SEGMENTS_TO_MOVE).build())
CoordinatorDynamicConfig.builder().withMaxSegmentsToMove( .withBalancerStrategy(balancerStrategy);
MAX_SEGMENTS_TO_MOVE
).build()
)
.withBalancerStrategy(balancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"));
} }
private static void mockDruidServer( private static void mockDruidServer(
@ -563,7 +558,7 @@ public class DruidCoordinatorBalancerTest
EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes(); EasyMock.expect(druidServer.getTier()).andReturn(tier).anyTimes();
EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce(); EasyMock.expect(druidServer.getCurrSize()).andReturn(currentSize).atLeastOnce();
EasyMock.expect(druidServer.getMaxSize()).andReturn(maxSize).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.getHost()).andReturn(name).anyTimes();
EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes(); EasyMock.expect(druidServer.getType()).andReturn(ServerType.HISTORICAL).anyTimes();
if (!segments.isEmpty()) { if (!segments.isEmpty()) {
@ -581,6 +576,7 @@ public class DruidCoordinatorBalancerTest
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(), EasyMock.anyObject(),
EasyMock.anyObject(),
EasyMock.anyObject() EasyMock.anyObject()
); );
EasyMock.expectLastCall().anyTimes(); EasyMock.expectLastCall().anyTimes();

View File

@ -25,17 +25,8 @@ import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.SegmentId;
import java.util.Comparator;
public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer public class DruidCoordinatorBalancerTester extends DruidCoordinatorBalancer
{ {
public static final Comparator<ServerHolder> 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) public DruidCoordinatorBalancerTester(DruidCoordinator coordinator)
{ {

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2LongMap; import it.unimi.dsi.fastutil.objects.Object2LongMap;
import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.CuratorFramework;
import org.apache.curator.framework.recipes.cache.PathChildrenCache; 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 static final long COORDINATOR_PERIOD = 100;
private DruidCoordinator coordinator; private DruidCoordinator coordinator;
private MetadataSegmentManager databaseSegmentManager; private MetadataSegmentManager segmentsMetadata;
private DataSourcesSnapshot dataSourcesSnapshot;
private DruidCoordinatorRuntimeParams coordinatorRuntimeParams;
private SingleServerInventoryView serverInventoryView; private SingleServerInventoryView serverInventoryView;
private ScheduledExecutorFactory scheduledExecutorFactory; private ScheduledExecutorFactory scheduledExecutorFactory;
private DruidServer druidServer; private DruidServer druidServer;
@ -100,20 +104,17 @@ public class DruidCoordinatorTest extends CuratorTestBase
private ObjectMapper objectMapper; private ObjectMapper objectMapper;
private DruidNode druidNode; private DruidNode druidNode;
private LatchableServiceEmitter serviceEmitter = new LatchableServiceEmitter(); private LatchableServiceEmitter serviceEmitter = new LatchableServiceEmitter();
private DataSourcesSnapshot dataSourcesSnapshot;
@Before @Before
public void setUp() throws Exception public void setUp() throws Exception
{ {
druidServer = EasyMock.createMock(DruidServer.class); druidServer = EasyMock.createMock(DruidServer.class);
serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class); serverInventoryView = EasyMock.createMock(SingleServerInventoryView.class);
databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class); segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class);
dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class); dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class);
coordinatorRuntimeParams = EasyMock.createNiceMock(DruidCoordinatorRuntimeParams.class);
metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class); metadataRuleManager = EasyMock.createNiceMock(MetadataRuleManager.class);
JacksonConfigManager configManager = EasyMock.createNiceMock(JacksonConfigManager.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( EasyMock.expect(
configManager.watch( configManager.watch(
EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY), EasyMock.eq(CoordinatorDynamicConfig.CONFIG_KEY),
@ -185,7 +186,7 @@ public class DruidCoordinatorTest extends CuratorTestBase
} }
}, },
configManager, configManager,
databaseSegmentManager, segmentsMetadata,
serverInventoryView, serverInventoryView,
metadataRuleManager, metadataRuleManager,
curator, curator,
@ -251,7 +252,10 @@ public class DruidCoordinatorTest extends CuratorTestBase
ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class); ImmutableDruidDataSource druidDataSource = EasyMock.createNiceMock(ImmutableDruidDataSource.class);
EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment); EasyMock.expect(druidDataSource.getSegment(EasyMock.anyObject(SegmentId.class))).andReturn(segment);
EasyMock.replay(druidDataSource); 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.expect(dataSourcesSnapshot.getDataSource(EasyMock.anyString())).andReturn(druidDataSource).anyTimes();
EasyMock.replay(dataSourcesSnapshot); EasyMock.replay(dataSourcesSnapshot);
scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class); scheduledExecutorFactory = EasyMock.createNiceMock(ScheduledExecutorFactory.class);
@ -288,7 +292,10 @@ public class DruidCoordinatorTest extends CuratorTestBase
EasyMock.expect(serverInventoryView.isSegmentLoadedByServer("to", segment)).andReturn(true).once(); EasyMock.expect(serverInventoryView.isSegmentLoadedByServer("to", segment)).andReturn(true).once();
EasyMock.replay(serverInventoryView); EasyMock.replay(serverInventoryView);
mockCoordinatorRuntimeParams();
coordinator.moveSegment( coordinator.moveSegment(
coordinatorRuntimeParams,
druidServer.toImmutableDruidServer(), druidServer.toImmutableDruidServer(),
druidServer2.toImmutableDruidServer(), druidServer2.toImmutableDruidServer(),
segment, segment,
@ -301,11 +308,14 @@ public class DruidCoordinatorTest extends CuratorTestBase
LoadPeonCallback dropCallback = dropCallbackCapture.getValue(); LoadPeonCallback dropCallback = dropCallbackCapture.getValue();
dropCallback.execute(); dropCallback.execute();
EasyMock.verify(druidServer); EasyMock.verify(druidServer, druidServer2, loadQueuePeon, serverInventoryView, metadataRuleManager);
EasyMock.verify(druidServer2); EasyMock.verify(coordinatorRuntimeParams);
EasyMock.verify(loadQueuePeon); }
EasyMock.verify(serverInventoryView);
EasyMock.verify(metadataRuleManager); private void mockCoordinatorRuntimeParams()
{
EasyMock.expect(coordinatorRuntimeParams.getDataSourcesSnapshot()).andReturn(this.dataSourcesSnapshot).anyTimes();
EasyMock.replay(coordinatorRuntimeParams);
} }
@Test(timeout = 60_000L) @Test(timeout = 60_000L)
@ -395,9 +405,10 @@ public class DruidCoordinatorTest extends CuratorTestBase
Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus()); Assert.assertEquals(ImmutableMap.of(dataSource, 100.0), coordinator.getLoadStatus());
curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getId().toString())); curator.delete().guaranteed().forPath(ZKPaths.makePath(LOADPATH, dataSegment.getId().toString()));
Map segmentAvailability = coordinator.getSegmentAvailability(); Object2IntMap<String> numsUnavailableUsedSegmentsPerDataSource =
Assert.assertEquals(1, segmentAvailability.size()); coordinator.computeNumsUnavailableUsedSegmentsPerDataSource();
Assert.assertEquals(0L, segmentAvailability.get(dataSource)); Assert.assertEquals(1, numsUnavailableUsedSegmentsPerDataSource.size());
Assert.assertEquals(0, numsUnavailableUsedSegmentsPerDataSource.getInt(dataSource));
Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier = Map<String, Object2LongMap<String>> underReplicationCountsPerDataSourcePerTier =
coordinator.computeUnderReplicationCountsPerDataSourcePerTier(); coordinator.computeUnderReplicationCountsPerDataSourcePerTier();
@ -531,21 +542,42 @@ public class DruidCoordinatorTest extends CuratorTestBase
leaderUnannouncerLatch.await(); leaderUnannouncerLatch.await();
EasyMock.verify(serverInventoryView); EasyMock.verify(serverInventoryView);
EasyMock.verify(databaseSegmentManager); EasyMock.verify(segmentsMetadata);
EasyMock.verify(metadataRuleManager); EasyMock.verify(metadataRuleManager);
} }
private void setupMetadataSegmentManagerMock(DruidDataSource dataSource) private void setupMetadataSegmentManagerMock(DruidDataSource dataSource)
{ {
EasyMock.expect(segmentsMetadata.isPollingDatabasePeriodically()).andReturn(true).anyTimes();
EasyMock EasyMock
.expect(dataSourcesSnapshot.iterateAllSegmentsInSnapshot()) .expect(segmentsMetadata.iterateAllUsedSegments())
.andReturn(dataSource.getSegments()) .andReturn(dataSource.getSegments())
.anyTimes(); .anyTimes();
EasyMock EasyMock
.expect(dataSourcesSnapshot.getDataSources()) .expect(segmentsMetadata.getImmutableDataSourcesWithAllUsedSegments())
.andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource())) .andReturn(Collections.singleton(dataSource.toImmutableDruidDataSource()))
.anyTimes(); .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 @Nullable

View File

@ -21,6 +21,7 @@ package org.apache.druid.server.coordinator;
import com.google.common.collect.Lists; import com.google.common.collect.Lists;
import org.apache.druid.client.ImmutableDruidServer; 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.DateTimes;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec; import org.apache.druid.timeline.partition.NoneShardSpec;
@ -138,7 +139,7 @@ public class ReservoirSegmentSamplerTest
EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce(); EasyMock.expect(druidServer1.getName()).andReturn("1").atLeastOnce();
EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer1.getCurrSize()).andReturn(30L).atLeastOnce();
EasyMock.expect(druidServer1.getMaxSize()).andReturn(100L).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.expect(druidServer1.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer1); EasyMock.replay(druidServer1);
@ -146,7 +147,7 @@ public class ReservoirSegmentSamplerTest
EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer2.getTier()).andReturn("normal").anyTimes();
EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer2.getCurrSize()).andReturn(30L).atLeastOnce();
EasyMock.expect(druidServer2.getMaxSize()).andReturn(100L).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.expect(druidServer2.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer2); EasyMock.replay(druidServer2);
@ -154,7 +155,7 @@ public class ReservoirSegmentSamplerTest
EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer3.getTier()).andReturn("normal").anyTimes();
EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer3.getCurrSize()).andReturn(30L).atLeastOnce();
EasyMock.expect(druidServer3.getMaxSize()).andReturn(100L).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.expect(druidServer3.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer3); EasyMock.replay(druidServer3);
@ -162,7 +163,7 @@ public class ReservoirSegmentSamplerTest
EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes(); EasyMock.expect(druidServer4.getTier()).andReturn("normal").anyTimes();
EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce(); EasyMock.expect(druidServer4.getCurrSize()).andReturn(30L).atLeastOnce();
EasyMock.expect(druidServer4.getMaxSize()).andReturn(100L).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.expect(druidServer4.getSegment(EasyMock.anyObject())).andReturn(null).anyTimes();
EasyMock.replay(druidServer4); EasyMock.replay(druidServer4);

View File

@ -115,7 +115,7 @@ public class CachingCostBalancerStrategyTest
{ {
ClusterCostCache.Builder builder = ClusterCostCache.builder(); ClusterCostCache.Builder builder = ClusterCostCache.builder();
serverHolders.forEach( 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); return new CachingCostBalancerStrategy(builder.build(), listeningExecutorService);
} }

View File

@ -20,15 +20,17 @@
package org.apache.druid.server.coordinator.helper; package org.apache.druid.server.coordinator.helper;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet; import com.google.common.collect.ImmutableSet;
import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.client.ImmutableDruidServer; import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.server.coordination.ServerType; 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.CoordinatorStats;
import org.apache.druid.server.coordinator.DruidCluster; 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.DruidCoordinator;
import org.apache.druid.server.coordinator.DruidCoordinatorRuleRunnerTest;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.LoadQueuePeon;
import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.server.coordinator.ServerHolder;
@ -38,17 +40,13 @@ import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Test; import org.junit.Test;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class DruidCoordinatorCleanupOvershadowedTest public class DruidCoordinatorCleanupOvershadowedTest
{ {
DruidCoordinatorCleanupOvershadowed druidCoordinatorCleanupOvershadowed; DruidCoordinatorCleanupOvershadowed druidCoordinatorMarkAsUnusedOvershadowedSegments;
DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class); DruidCoordinator coordinator = EasyMock.createStrictMock(DruidCoordinator.class);
private List<DataSegment> availableSegments; private List<DataSegment> usedSegments;
DateTime start = DateTimes.of("2012-01-01"); DateTime start = DateTimes.of("2012-01-01");
DruidCluster druidCluster; DruidCluster druidCluster;
private LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class); private LoadQueuePeon mockPeon = EasyMock.createMock(LoadQueuePeon.class);
@ -70,8 +68,8 @@ public class DruidCoordinatorCleanupOvershadowedTest
@Test @Test
public void testRun() public void testRun()
{ {
druidCoordinatorCleanupOvershadowed = new DruidCoordinatorCleanupOvershadowed(coordinator); druidCoordinatorMarkAsUnusedOvershadowedSegments = new DruidCoordinatorCleanupOvershadowed(coordinator);
availableSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2); usedSegments = ImmutableList.of(segmentV1, segmentV0, segmentV2);
// Dummy values for comparisons in TreeSet // Dummy values for comparisons in TreeSet
EasyMock.expect(mockPeon.getLoadQueueSize()) EasyMock.expect(mockPeon.getLoadQueueSize())
@ -103,27 +101,26 @@ public class DruidCoordinatorCleanupOvershadowedTest
.andReturn(ImmutableSet.of(segmentV1, segmentV2)) .andReturn(ImmutableSet.of(segmentV1, segmentV2))
.anyTimes(); .anyTimes();
EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes(); EasyMock.expect(druidDataSource.getName()).andReturn("test").anyTimes();
coordinator.removeSegment(segmentV1); coordinator.markSegmentAsUnused(segmentV1);
coordinator.removeSegment(segmentV0); coordinator.markSegmentAsUnused(segmentV0);
EasyMock.expectLastCall(); EasyMock.expectLastCall();
EasyMock.replay(mockPeon, coordinator, druidServer, druidDataSource); EasyMock.replay(mockPeon, coordinator, druidServer, druidDataSource);
druidCluster = new DruidCluster( druidCluster = DruidClusterBuilder
null,
ImmutableMap.of(
"normal",
Stream.of(
new ServerHolder(druidServer, mockPeon)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
));
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
.newBuilder() .newBuilder()
.withAvailableSegmentsInTest(availableSegments) .addTier("normal", new ServerHolder(druidServer, mockPeon))
.build();
DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
.newBuilder()
.withUsedSegmentsInTest(usedSegments)
.withCoordinatorStats(new CoordinatorStats()) .withCoordinatorStats(new CoordinatorStats())
.withDruidCluster(druidCluster) .withDruidCluster(druidCluster)
.withDynamicConfigs(
DruidCoordinatorRuleRunnerTest.COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS
)
.build(); .build();
druidCoordinatorCleanupOvershadowed.run(params); druidCoordinatorMarkAsUnusedOvershadowedSegments.run(params);
EasyMock.verify(coordinator, druidDataSource, druidServer); EasyMock.verify(coordinator, druidDataSource, druidServer);
} }
} }

View File

@ -21,7 +21,9 @@ package org.apache.druid.server.coordinator.helper;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Lists; 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.ClientCompactQueryTuningConfig;
import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.client.indexing.NoopIndexingServiceClient; import org.apache.druid.client.indexing.NoopIndexingServiceClient;
@ -29,6 +31,7 @@ import org.apache.druid.indexer.TaskStatusPlus;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.server.coordinator.CoordinatorCompactionConfig; 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.CoordinatorStats;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams; import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
@ -48,7 +51,6 @@ import org.junit.Test;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.function.Supplier; import java.util.function.Supplier;
@ -123,34 +125,19 @@ public class DruidCoordinatorSegmentCompactorTest
@Before @Before
public void setup() public void setup()
{ {
dataSources = new HashMap<>(); List<DataSegment> segments = new ArrayList<>();
for (int i = 0; i < 3; i++) { for (int i = 0; i < 3; i++) {
final String dataSource = DATA_SOURCE_PREFIX + i; final String dataSource = DATA_SOURCE_PREFIX + i;
for (int j : new int[] {0, 1, 2, 3, 7, 8}) {
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>(
String.CASE_INSENSITIVE_ORDER
);
for (int j = 0; j < 4; j++) {
for (int k = 0; k < 2; k++) { for (int k = 0; k < 2; k++) {
DataSegment segment = createSegment(dataSource, j, true, k); segments.add(createSegment(dataSource, j, true, k));
timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment)); segments.add(createSegment(dataSource, j, false, k));
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 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));
}
}
dataSources.put(dataSource, timeline);
} }
dataSources = DataSourcesSnapshot
.fromUsedSegments(segments, ImmutableMap.of())
.getUsedSegmentsTimelinesPerDataSource();
} }
private static DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition) private static DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition)
@ -261,9 +248,9 @@ public class DruidCoordinatorSegmentCompactorTest
private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor) private CoordinatorStats runCompactor(DruidCoordinatorSegmentCompactor compactor)
{ {
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
.newBuilder() .newBuilder()
.withDataSources(dataSources) .withUsedSegmentsTimelinesPerDataSourceInTest(dataSources)
.withCompactionConfig(CoordinatorCompactionConfig.from(createCompactionConfigs())) .withCompactionConfig(CoordinatorCompactionConfig.from(createCompactionConfigs()))
.build(); .build();
return compactor.run(params).getCoordinatorStats(); return compactor.run(params).getCoordinatorStats();
@ -291,10 +278,7 @@ public class DruidCoordinatorSegmentCompactorTest
long numDataSourceOfExpectedRemainingSegments = stats long numDataSourceOfExpectedRemainingSegments = stats
.getDataSources(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT) .getDataSources(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT)
.stream() .stream()
.mapToLong(dataSource -> stats.getDataSourceStat( .mapToLong(ds -> stats.getDataSourceStat(DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT, ds))
DruidCoordinatorSegmentCompactor.SEGMENT_SIZE_WAIT_COMPACT,
dataSource)
)
.filter(stat -> stat == expectedRemainingSegments) .filter(stat -> stat == expectedRemainingSegments)
.count(); .count();
Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments); Assert.assertEquals(i + 1, numDataSourceOfExpectedRemainingSegments);

View File

@ -25,6 +25,7 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.metadata.MetadataSegmentManager; import org.apache.druid.metadata.MetadataSegmentManager;
import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig; import org.apache.druid.server.coordinator.TestDruidCoordinatorConfig;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.joda.time.DateTime;
import org.joda.time.Duration; import org.joda.time.Duration;
import org.joda.time.Interval; import org.joda.time.Interval;
import org.junit.Assert; import org.junit.Assert;
@ -37,34 +38,34 @@ import java.util.List;
public class DruidCoordinatorSegmentKillerTest public class DruidCoordinatorSegmentKillerTest
{ {
@Test @Test
public void testFindIntervalForKillTask() public void testFindIntervalForKill()
{ {
testFindIntervalForKillTask(null, null); testFindIntervalForKill(null, null);
testFindIntervalForKillTask(ImmutableList.of(), 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")), ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2016/2017")),
Intervals.of("2014/2017") Intervals.of("2014/2017")
); );
testFindIntervalForKillTask( testFindIntervalForKill(
ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2015/2016")), ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2015/2016")),
Intervals.of("2014/2016") Intervals.of("2014/2016")
); );
testFindIntervalForKillTask( testFindIntervalForKill(
ImmutableList.of(Intervals.of("2015/2016"), Intervals.of("2014/2015")), ImmutableList.of(Intervals.of("2015/2016"), Intervals.of("2014/2015")),
Intervals.of("2014/2016") Intervals.of("2014/2016")
); );
testFindIntervalForKillTask( testFindIntervalForKill(
ImmutableList.of(Intervals.of("2015/2017"), Intervals.of("2014/2016")), ImmutableList.of(Intervals.of("2015/2017"), Intervals.of("2014/2016")),
Intervals.of("2014/2017") Intervals.of("2014/2017")
); );
testFindIntervalForKillTask( testFindIntervalForKill(
ImmutableList.of( ImmutableList.of(
Intervals.of("2015/2019"), Intervals.of("2015/2019"),
Intervals.of("2014/2016"), Intervals.of("2014/2016"),
@ -73,7 +74,7 @@ public class DruidCoordinatorSegmentKillerTest
Intervals.of("2014/2020") Intervals.of("2014/2020")
); );
testFindIntervalForKillTask( testFindIntervalForKill(
ImmutableList.of( ImmutableList.of(
Intervals.of("2015/2019"), Intervals.of("2015/2019"),
Intervals.of("2014/2016"), Intervals.of("2014/2016"),
@ -84,23 +85,21 @@ public class DruidCoordinatorSegmentKillerTest
); );
} }
private void testFindIntervalForKillTask(List<Interval> segmentManagerResult, Interval expected) private void testFindIntervalForKill(List<Interval> segmentIntervals, Interval expected)
{ {
MetadataSegmentManager segmentManager = EasyMock.createMock(MetadataSegmentManager.class); MetadataSegmentManager segmentsMetadata = EasyMock.createMock(MetadataSegmentManager.class);
EasyMock.expect( EasyMock.expect(
segmentManager.getUnusedSegmentIntervals( segmentsMetadata.getUnusedSegmentIntervals(
EasyMock.anyString(), EasyMock.anyString(),
EasyMock.anyObject(Interval.class), EasyMock.anyObject(DateTime.class),
EasyMock.anyInt() EasyMock.anyInt()
) )
).andReturn( ).andReturn(segmentIntervals);
segmentManagerResult EasyMock.replay(segmentsMetadata);
);
EasyMock.replay(segmentManager);
IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class); IndexingServiceClient indexingServiceClient = EasyMock.createMock(IndexingServiceClient.class);
DruidCoordinatorSegmentKiller coordinatorSegmentKiller = new DruidCoordinatorSegmentKiller( DruidCoordinatorSegmentKiller coordinatorSegmentKiller = new DruidCoordinatorSegmentKiller(
segmentManager, segmentsMetadata,
indexingServiceClient, indexingServiceClient,
new TestDruidCoordinatorConfig( new TestDruidCoordinatorConfig(
null, null,
@ -117,7 +116,7 @@ public class DruidCoordinatorSegmentKillerTest
Assert.assertEquals( Assert.assertEquals(
expected, expected,
coordinatorSegmentKiller.findIntervalForKillTask("test", 10000) coordinatorSegmentKiller.findIntervalForKill("test", 10000)
); );
} }
} }

View File

@ -625,10 +625,7 @@ public class NewestSegmentFirstPolicyTest
SegmentGenerateSpec... specs SegmentGenerateSpec... specs
) )
{ {
VersionedIntervalTimeline<String, DataSegment> timeline = new VersionedIntervalTimeline<>( List<DataSegment> segments = new ArrayList<>();
String.CASE_INSENSITIVE_ORDER
);
final String version = DateTimes.nowUtc().toString(); final String version = DateTimes.nowUtc().toString();
final List<SegmentGenerateSpec> orderedSpecs = Arrays.asList(specs); final List<SegmentGenerateSpec> orderedSpecs = Arrays.asList(specs);
@ -659,18 +656,14 @@ public class NewestSegmentFirstPolicyTest
0, 0,
spec.segmentSize spec.segmentSize
); );
timeline.add( segments.add(segment);
segmentInterval,
version,
shardSpec.createChunk(segment)
);
} }
remaininInterval = SegmentCompactorUtil.removeIntervalFromEnd(remaininInterval, segmentInterval); remaininInterval = SegmentCompactorUtil.removeIntervalFromEnd(remaininInterval, segmentInterval);
} }
} }
return timeline; return VersionedIntervalTimeline.forSegments(segments);
} }
private DataSourceCompactionConfig createCompactionConfig( private DataSourceCompactionConfig createCompactionConfig(

View File

@ -20,13 +20,14 @@
package org.apache.druid.server.coordinator.rules; package org.apache.druid.server.coordinator.rules;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.client.DruidServer; import org.apache.druid.client.DruidServer;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.server.coordination.ServerType; 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.CoordinatorStats;
import org.apache.druid.server.coordinator.DruidCluster; 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.DruidCoordinatorRuntimeParams;
import org.apache.druid.server.coordinator.LoadQueuePeonTester; import org.apache.druid.server.coordinator.LoadQueuePeonTester;
import org.apache.druid.server.coordinator.SegmentReplicantLookup; import org.apache.druid.server.coordinator.SegmentReplicantLookup;
@ -38,12 +39,8 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.TreeSet;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class BroadcastDistributionRuleTest public class BroadcastDistributionRuleTest
{ {
@ -241,56 +238,50 @@ public class BroadcastDistributionRuleTest
true true
); );
druidCluster = new DruidCluster( druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"hot", "hot",
Stream.of( holdersOfLargeSegments.get(0),
holdersOfLargeSegments.get(0), holderOfSmallSegment,
holderOfSmallSegment, holdersOfLargeSegments2.get(0)
holdersOfLargeSegments2.get(0) )
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))), .addTier(
DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_TIER,
Stream.of( holdersOfLargeSegments.get(1),
holdersOfLargeSegments.get(1), holdersOfLargeSegments.get(2),
holdersOfLargeSegments.get(2), holdersOfLargeSegments2.get(1)
holdersOfLargeSegments2.get(1)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
) )
); .build();
secondCluster = new DruidCluster( secondCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"tier1", "tier1",
Stream.of( activeServer,
activeServer, decommissioningServer1,
decommissioningServer1, decommissioningServer2
decommissioningServer2
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
) )
); .build();
} }
@Test @Test
public void testBroadcastToSingleDataSource() 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( CoordinatorStats stats = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() makeCoordinartorRuntimeParams(
.withDruidCluster(druidCluster) druidCluster,
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) smallSegment,
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) largeSegments.get(0),
.withAvailableSegmentsInTest( largeSegments.get(1),
smallSegment, largeSegments.get(2),
largeSegments.get(0), largeSegments2.get(0),
largeSegments.get(1), largeSegments2.get(1)
largeSegments.get(2), ),
largeSegments2.get(0),
largeSegments2.get(1)
).build(),
smallSegment smallSegment
); );
@ -310,6 +301,19 @@ public class BroadcastDistributionRuleTest
Assert.assertFalse(holderOfSmallSegment.getPeon().getSegmentsToLoad().contains(smallSegment)); 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: * Servers:
* name | segments * name | segments
@ -326,19 +330,17 @@ public class BroadcastDistributionRuleTest
@Test @Test
public void testBroadcastDecommissioning() 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( CoordinatorStats stats = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() makeCoordinartorRuntimeParams(
.withDruidCluster(secondCluster) secondCluster,
.withSegmentReplicantLookup(SegmentReplicantLookup.make(secondCluster)) smallSegment,
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) largeSegments.get(0),
.withAvailableSegmentsInTest( largeSegments.get(1)
smallSegment, ),
largeSegments.get(0),
largeSegments.get(1)
).build(),
smallSegment smallSegment
); );
@ -359,18 +361,15 @@ public class BroadcastDistributionRuleTest
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() makeCoordinartorRuntimeParams(
.withDruidCluster(druidCluster) druidCluster,
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) smallSegment,
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) largeSegments.get(0),
.withAvailableSegmentsInTest( largeSegments.get(1),
smallSegment, largeSegments.get(2),
largeSegments.get(0), largeSegments2.get(0),
largeSegments.get(1), largeSegments2.get(1)
largeSegments.get(2), ),
largeSegments2.get(0),
largeSegments2.get(1)
).build(),
smallSegment smallSegment
); );
@ -397,18 +396,15 @@ public class BroadcastDistributionRuleTest
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() makeCoordinartorRuntimeParams(
.withDruidCluster(druidCluster) druidCluster,
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) smallSegment,
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) largeSegments.get(0),
.withAvailableSegmentsInTest( largeSegments.get(1),
smallSegment, largeSegments.get(2),
largeSegments.get(0), largeSegments2.get(0),
largeSegments.get(1), largeSegments2.get(1)
largeSegments.get(2), ),
largeSegments2.get(0),
largeSegments2.get(1)
).build(),
smallSegment smallSegment
); );

View File

@ -26,6 +26,7 @@ import com.google.common.collect.ImmutableSet;
import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.client.DruidServer; import org.apache.druid.client.DruidServer;
import org.apache.druid.client.ImmutableDruidServer;
import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
@ -36,9 +37,11 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.server.coordinator.BalancerStrategy; import org.apache.druid.server.coordinator.BalancerStrategy;
import org.apache.druid.server.coordinator.CoordinatorDynamicConfig; 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.CoordinatorStats;
import org.apache.druid.server.coordinator.CostBalancerStrategyFactory; import org.apache.druid.server.coordinator.CostBalancerStrategyFactory;
import org.apache.druid.server.coordinator.DruidCluster; 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.DruidCoordinatorRuntimeParams;
import org.apache.druid.server.coordinator.LoadQueuePeon; import org.apache.druid.server.coordinator.LoadQueuePeon;
import org.apache.druid.server.coordinator.LoadQueuePeonTester; import org.apache.druid.server.coordinator.LoadQueuePeonTester;
@ -56,18 +59,13 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.Executors; import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/** /**
*/ */
@ -138,53 +136,34 @@ public class LoadRuleTest
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"hot", "hot",
Stream.of( new ServerHolder(
new ServerHolder( new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1)
new DruidServer( .toImmutableDruidServer(),
"serverHot", mockPeon
"hostHot", )
null,
1000,
ServerType.HISTORICAL,
"hot",
1
).toImmutableDruidServer(),
mockPeon
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
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())))
) )
); .addTier(
DruidServer.DEFAULT_TIER,
new ServerHolder(
new DruidServer(
"serverNorm",
"hostNorm",
null,
1000,
ServerType.HISTORICAL,
DruidServer.DEFAULT_TIER,
0
).toImmutableDruidServer(),
mockPeon
)
)
.build();
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
null,
DruidCoordinatorRuntimeParams.newBuilder()
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment
);
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot"));
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER)); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, DruidServer.DEFAULT_TIER));
@ -192,6 +171,21 @@ public class LoadRuleTest
EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); 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 @Test
public void testLoadPrimaryAssignDoesNotOverAssign() public void testLoadPrimaryAssignDoesNotOverAssign()
{ {
@ -213,47 +207,18 @@ public class LoadRuleTest
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
DruidCluster druidCluster = new DruidCluster( ImmutableDruidServer server1 =
null, new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer();
ImmutableMap.of( ImmutableDruidServer server2 =
"hot", new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 1).toImmutableDruidServer();
Stream.of( DruidCluster druidCluster = DruidClusterBuilder
new ServerHolder( .newBuilder()
new DruidServer( .addTier("hot", new ServerHolder(server1, mockPeon), new ServerHolder(server2, mockPeon))
"serverHot", .build();
"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())))
)
);
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() makeCoordinatorRuntimeParams(druidCluster, segment),
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment segment
); );
@ -264,50 +229,17 @@ public class LoadRuleTest
final LoadQueuePeon loadingPeon = createLoadingPeon(ImmutableList.of(segment)); final LoadQueuePeon loadingPeon = createLoadingPeon(ImmutableList.of(segment));
EasyMock.replay(loadingPeon); EasyMock.replay(loadingPeon);
DruidCluster afterLoad = new DruidCluster( DruidCluster afterLoad = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier("hot", new ServerHolder(server1, loadingPeon), new ServerHolder(server2, mockPeon))
"hot", .build();
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())))
)
);
CoordinatorStats statsAfterLoadPrimary = rule.run( CoordinatorStats statsAfterLoadPrimary = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() makeCoordinatorRuntimeParams(afterLoad, segment),
.withDruidCluster(afterLoad)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(afterLoad))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment segment
); );
Assert.assertEquals(0, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot")); Assert.assertEquals(0, statsAfterLoadPrimary.getTieredStat(LoadRule.ASSIGNED_COUNT, "hot"));
EasyMock.verify(throttler, mockPeon, mockBalancerStrategy); EasyMock.verify(throttler, mockPeon, mockBalancerStrategy);
@ -330,72 +262,36 @@ public class LoadRuleTest
EasyMock.replay(throttler, mockPeon1, mockPeon2, mockBalancerStrategy); EasyMock.replay(throttler, mockPeon1, mockPeon2, mockBalancerStrategy);
final LoadRule rule = createLoadRule(ImmutableMap.of( final LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 10, "tier2", 10));
"tier1", 10,
"tier2", 10
));
final DruidCluster druidCluster = new DruidCluster( final DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"tier1", "tier1",
Stream.of( new ServerHolder(
new ServerHolder( new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "tier1", 0)
new DruidServer( .toImmutableDruidServer(),
"server1", mockPeon1
"host1", )
null,
1000,
ServerType.HISTORICAL,
"tier1",
0
).toImmutableDruidServer(),
mockPeon1
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
"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())))
) )
); .addTier(
"tier2",
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 DataSegment segment = createDataSegment("foo");
final CoordinatorStats stats = rule.run( final CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
null,
DruidCoordinatorRuntimeParams.newBuilder()
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment
);
Assert.assertEquals(0L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); Assert.assertEquals(0L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1"));
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2"));
@ -421,15 +317,7 @@ public class LoadRuleTest
final DataSegment segment = createDataSegment("foo"); final DataSegment segment = createDataSegment("foo");
DruidServer server1 = new DruidServer( DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0);
"serverHot",
"hostHot",
null,
1000,
ServerType.HISTORICAL,
"hot",
0
);
server1.addDataSegment(segment); server1.addDataSegment(segment);
DruidServer server2 = new DruidServer( DruidServer server2 = new DruidServer(
"serverNorm", "serverNorm",
@ -450,41 +338,17 @@ public class LoadRuleTest
DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_TIER,
0 0
); );
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon))
"hot", .addTier(
Stream.of(
new ServerHolder(
server1.toImmutableDruidServer(),
mockPeon
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
DruidServer.DEFAULT_TIER, DruidServer.DEFAULT_TIER,
Stream.of( new ServerHolder(server2.toImmutableDruidServer(), mockPeon),
new ServerHolder( new ServerHolder(server3.toImmutableDruidServer(), mockPeon)
server2.toImmutableDruidServer(),
mockPeon
),
new ServerHolder(
server3.toImmutableDruidServer(),
mockPeon
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
) )
); .build();
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
null,
DruidCoordinatorRuntimeParams.newBuilder()
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment
);
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot"));
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", DruidServer.DEFAULT_TIER)); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", DruidServer.DEFAULT_TIER));
@ -505,43 +369,32 @@ public class LoadRuleTest
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
LoadRule rule = createLoadRule(ImmutableMap.of( LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1));
"nonExistentTier", 1,
"hot", 1
));
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"hot", "hot",
Stream.of( new ServerHolder(
new ServerHolder( new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0)
new DruidServer( .toImmutableDruidServer(),
"serverHot", mockPeon
"hostHot", )
null,
1000,
ServerType.HISTORICAL,
"hot",
0
).toImmutableDruidServer(),
mockPeon
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
) )
); .build();
final DataSegment segment = createDataSegment("foo"); final DataSegment segment = createDataSegment("foo");
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(
null, null,
DruidCoordinatorRuntimeParams.newBuilder() CoordinatorRuntimeParamsTestHelpers
.withDruidCluster(druidCluster) .newBuilder()
.withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster())) .withDruidCluster(druidCluster)
.withReplicationManager(throttler) .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
.withBalancerStrategy(mockBalancerStrategy) .withReplicationManager(throttler)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .withBalancerStrategy(mockBalancerStrategy)
.withAvailableSegmentsInTest(segment).build(), .withUsedSegmentsInTest(segment)
.build(),
segment segment
); );
@ -561,62 +414,25 @@ public class LoadRuleTest
.times(2); .times(2);
EasyMock.replay(throttler, mockPeon, mockBalancerStrategy); EasyMock.replay(throttler, mockPeon, mockBalancerStrategy);
LoadRule rule = createLoadRule(ImmutableMap.of( LoadRule rule = createLoadRule(ImmutableMap.of("nonExistentTier", 1, "hot", 1));
"nonExistentTier", 1,
"hot", 1
));
final DataSegment segment = createDataSegment("foo"); final DataSegment segment = createDataSegment("foo");
DruidServer server1 = new DruidServer( DruidServer server1 = new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0);
"serverHot", DruidServer server2 = new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0);
"hostHot",
null,
1000,
ServerType.HISTORICAL,
"hot",
0
);
DruidServer server2 = new DruidServer(
"serverHo2t",
"hostHot2",
null,
1000,
ServerType.HISTORICAL,
"hot",
0
);
server1.addDataSegment(segment); server1.addDataSegment(segment);
server2.addDataSegment(segment); server2.addDataSegment(segment);
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"hot", "hot",
Stream.of( new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
new ServerHolder( new ServerHolder(server2.toImmutableDruidServer(), mockPeon)
server1.toImmutableDruidServer(),
mockPeon
),
new ServerHolder(
server2.toImmutableDruidServer(),
mockPeon
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
) )
); .build();
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
null,
DruidCoordinatorRuntimeParams.newBuilder()
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment
);
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot")); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "hot"));
@ -634,46 +450,33 @@ public class LoadRuleTest
final LoadQueuePeonTester peon = new LoadQueuePeonTester(); final LoadQueuePeonTester peon = new LoadQueuePeonTester();
LoadRule rule = createLoadRule(ImmutableMap.of( LoadRule rule = createLoadRule(ImmutableMap.of("hot", 1));
"hot", 1
));
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier(
"hot", "hot",
Stream.of( new ServerHolder(
new ServerHolder( new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0)
new DruidServer( .toImmutableDruidServer(),
"serverHot", peon
"hostHot", )
null,
1000,
ServerType.HISTORICAL,
"hot",
0
).toImmutableDruidServer(),
peon
)
).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
) )
); .build();
DataSegment dataSegment1 = createDataSegment("ds1"); DataSegment dataSegment1 = createDataSegment("ds1");
DataSegment dataSegment2 = createDataSegment("ds2"); DataSegment dataSegment2 = createDataSegment("ds2");
DataSegment dataSegment3 = createDataSegment("ds3"); DataSegment dataSegment3 = createDataSegment("ds3");
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
DruidCoordinatorRuntimeParams .newBuilder()
.newBuilder() .withDruidCluster(druidCluster)
.withDruidCluster(druidCluster) .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) .withReplicationManager(throttler)
.withReplicationManager(throttler) .withBalancerStrategy(mockBalancerStrategy)
.withBalancerStrategy(mockBalancerStrategy) .withUsedSegmentsInTest(dataSegment1, dataSegment2, dataSegment3)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build())
.withAvailableSegmentsInTest(dataSegment1, dataSegment2, dataSegment3) .build();
.withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsInNodeLoadingQueue(2).build())
.build();
CoordinatorStats stats1 = rule.run(null, params, dataSegment1); CoordinatorStats stats1 = rule.run(null, params, dataSegment1);
CoordinatorStats stats2 = rule.run(null, params, dataSegment2); CoordinatorStats stats2 = rule.run(null, params, dataSegment2);
@ -696,10 +499,7 @@ public class LoadRuleTest
final LoadQueuePeon mockPeon1 = createEmptyPeon(); final LoadQueuePeon mockPeon1 = createEmptyPeon();
final LoadQueuePeon mockPeon2 = createOneCallPeonMock(); final LoadQueuePeon mockPeon2 = createOneCallPeonMock();
LoadRule rule = createLoadRule(ImmutableMap.of( LoadRule rule = createLoadRule(ImmutableMap.of("tier1", 1, "tier2", 1));
"tier1", 1,
"tier2", 1
));
final DataSegment segment = createDataSegment("foo"); final DataSegment segment = createDataSegment("foo");
@ -710,27 +510,13 @@ public class LoadRuleTest
EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy); EasyMock.replay(mockPeon1, mockPeon2, mockBalancerStrategy);
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null, .newBuilder()
ImmutableMap.of( .addTier("tier1", createServerHolder("tier1", mockPeon1, true))
"tier1", .addTier("tier2", createServerHolder("tier2", mockPeon2, false))
Collections.singleton(createServerHolder("tier1", mockPeon1, true)), .build();
"tier2",
Collections.singleton(createServerHolder("tier2", mockPeon2, false))
)
);
CoordinatorStats stats = rule.run( CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
null,
DruidCoordinatorRuntimeParams.newBuilder()
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster))
.withReplicationManager(throttler)
.withBalancerStrategy(mockBalancerStrategy)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
.withAvailableSegmentsInTest(segment).build(),
segment
);
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2"));
EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy); EasyMock.verify(mockPeon1, mockPeon2, mockBalancerStrategy);
@ -771,23 +557,13 @@ public class LoadRuleTest
EasyMock.replay(throttler, mockPeon1, mockPeon2, mockPeon3, mockPeon4, mockBalancerStrategy); 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( CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment), segment);
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
);
Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1")); Assert.assertEquals(1L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier1"));
Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2")); Assert.assertEquals(2L, stats.getTieredStat(LoadRule.ASSIGNED_COUNT, "tier2"));
@ -820,39 +596,20 @@ public class LoadRuleTest
DruidServer server2 = createServer("tier1"); DruidServer server2 = createServer("tier1");
server2.addDataSegment(segment2); server2.addDataSegment(segment2);
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null,
ImmutableMap.of(
"tier1",
Arrays.asList(
new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true),
new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false)
)
)
);
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
.newBuilder() .newBuilder()
.withDruidCluster(druidCluster) .addTier(
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) "tier1",
.withReplicationManager(throttler) new ServerHolder(server1.toImmutableDruidServer(), mockPeon, true),
.withBalancerStrategy(mockBalancerStrategy) new ServerHolder(server2.toImmutableDruidServer(), mockPeon, false)
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) )
.withAvailableSegmentsInTest(segment1, segment2)
.build(); .build();
CoordinatorStats stats = rule.run(
null,
params,
segment1
);
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
stats = rule.run(
null,
params,
segment2
);
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
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);
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
EasyMock.verify(throttler, mockPeon); EasyMock.verify(throttler, mockPeon);
} }
@ -885,32 +642,17 @@ public class LoadRuleTest
DruidServer server3 = createServer("tier1"); DruidServer server3 = createServer("tier1");
server3.addDataSegment(segment1); server3.addDataSegment(segment1);
DruidCluster druidCluster = new DruidCluster( DruidCluster druidCluster = DruidClusterBuilder
null,
ImmutableMap.of(
"tier1",
Arrays.asList(
new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false),
new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true),
new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false)
)
)
);
DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
.newBuilder() .newBuilder()
.withDruidCluster(druidCluster) .addTier(
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster)) "tier1",
.withReplicationManager(throttler) new ServerHolder(server1.toImmutableDruidServer(), mockPeon1, false),
.withBalancerStrategy(mockBalancerStrategy) new ServerHolder(server2.toImmutableDruidServer(), mockPeon2, true),
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01")) new ServerHolder(server3.toImmutableDruidServer(), mockPeon3, false)
.withAvailableSegmentsInTest(segment1) )
.build(); .build();
CoordinatorStats stats = rule.run(
null, CoordinatorStats stats = rule.run(null, makeCoordinatorRuntimeParams(druidCluster, segment1), segment1);
params,
segment1
);
Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1")); Assert.assertEquals(1L, stats.getTieredStat("droppedCount", "tier1"));
Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size()); Assert.assertEquals(0, mockPeon1.getSegmentsToDrop().size());
Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size()); Assert.assertEquals(1, mockPeon2.getSegmentsToDrop().size());

View File

@ -223,9 +223,10 @@ public class CoordinatorDynamicConfigTest
@Test @Test
public void testUpdate() public void testUpdate()
{ {
CoordinatorDynamicConfig current = CoordinatorDynamicConfig.builder() CoordinatorDynamicConfig current = CoordinatorDynamicConfig
.withKillDataSourceWhitelist(ImmutableSet.of("x")) .builder()
.build(); .withSpecificDataSourcesToKillUnusedSegmentsIn(ImmutableSet.of("x"))
.build();
Assert.assertEquals( Assert.assertEquals(
current, current,
@ -246,7 +247,7 @@ public class CoordinatorDynamicConfigTest
private void assertConfig( private void assertConfig(
CoordinatorDynamicConfig config, CoordinatorDynamicConfig config,
long expectedMillisToWaitBeforeDeleting, long expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
long expectedMergeBytesLimit, long expectedMergeBytesLimit,
int expectedMergeSegmentsLimit, int expectedMergeSegmentsLimit,
int expectedMaxSegmentsToMove, int expectedMaxSegmentsToMove,
@ -254,14 +255,17 @@ public class CoordinatorDynamicConfigTest
int expectedReplicationThrottleLimit, int expectedReplicationThrottleLimit,
int expectedBalancerComputeThreads, int expectedBalancerComputeThreads,
boolean expectedEmitingBalancingStats, boolean expectedEmitingBalancingStats,
Set<String> expectedKillableDatasources, Set<String> expectedSpecificDataSourcesToKillUnusedSegmentsIn,
boolean expectedKillAllDataSources, boolean expectedKillUnusedSegmentsInAllDataSources,
int expectedMaxSegmentsInNodeLoadingQueue, int expectedMaxSegmentsInNodeLoadingQueue,
Set<String> decommissioning, Set<String> decommissioningNodes,
int decommissioningMaxPercentOfMaxSegmentsToMove int decommissioningMaxPercentOfMaxSegmentsToMove
) )
{ {
Assert.assertEquals(expectedMillisToWaitBeforeDeleting, config.getMillisToWaitBeforeDeleting()); Assert.assertEquals(
expectedLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments,
config.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
);
Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit()); Assert.assertEquals(expectedMergeBytesLimit, config.getMergeBytesLimit());
Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit()); Assert.assertEquals(expectedMergeSegmentsLimit, config.getMergeSegmentsLimit());
Assert.assertEquals(expectedMaxSegmentsToMove, config.getMaxSegmentsToMove()); Assert.assertEquals(expectedMaxSegmentsToMove, config.getMaxSegmentsToMove());
@ -269,10 +273,16 @@ public class CoordinatorDynamicConfigTest
Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit()); Assert.assertEquals(expectedReplicationThrottleLimit, config.getReplicationThrottleLimit());
Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads()); Assert.assertEquals(expectedBalancerComputeThreads, config.getBalancerComputeThreads());
Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats()); Assert.assertEquals(expectedEmitingBalancingStats, config.emitBalancingStats());
Assert.assertEquals(expectedKillableDatasources, config.getKillableDataSources()); Assert.assertEquals(
Assert.assertEquals(expectedKillAllDataSources, config.isKillAllDataSources()); expectedSpecificDataSourcesToKillUnusedSegmentsIn,
config.getSpecificDataSourcesToKillUnusedSegmentsIn()
);
Assert.assertEquals(expectedKillUnusedSegmentsInAllDataSources, config.isKillUnusedSegmentsInAllDataSources());
Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue()); Assert.assertEquals(expectedMaxSegmentsInNodeLoadingQueue, config.getMaxSegmentsInNodeLoadingQueue());
Assert.assertEquals(decommissioning, config.getDecommissioningNodes()); Assert.assertEquals(decommissioningNodes, config.getDecommissioningNodes());
Assert.assertEquals(decommissioningMaxPercentOfMaxSegmentsToMove, config.getDecommissioningMaxPercentOfMaxSegmentsToMove()); Assert.assertEquals(
decommissioningMaxPercentOfMaxSegmentsToMove,
config.getDecommissioningMaxPercentOfMaxSegmentsToMove()
);
} }
} }

View File

@ -555,7 +555,7 @@ public class SystemSchema extends AbstractSchema
for (ImmutableDruidServer druidServer : druidServers) { for (ImmutableDruidServer druidServer : druidServers) {
final Iterable<DataSegment> authorizedServerSegments = AuthorizationUtils.filterAuthorizedResources( final Iterable<DataSegment> authorizedServerSegments = AuthorizationUtils.filterAuthorizedResources(
authenticationResult, authenticationResult,
druidServer.getLazyAllSegments(), druidServer.iterateAllSegments(),
SEGMENT_RA_GENERATOR, SEGMENT_RA_GENERATOR,
authorizerMapper authorizerMapper
); );

View File

@ -275,7 +275,8 @@ public class DruidSchemaTest extends CalciteTestBase
final Pair<ImmutableDruidServer, DataSegment> pair = druidServers final Pair<ImmutableDruidServer, DataSegment> pair = druidServers
.stream() .stream()
.flatMap(druidServer -> druidServer .flatMap(druidServer -> druidServer
.getLazyAllSegments().stream() .iterateAllSegments()
.stream()
.filter(segment -> segment.getId().equals(existingSegment.getId())) .filter(segment -> segment.getId().equals(existingSegment.getId()))
.map(segment -> Pair.of(druidServer, segment)) .map(segment -> Pair.of(druidServer, segment))
) )

View File

@ -105,11 +105,10 @@ import java.net.URL;
import java.nio.charset.StandardCharsets; import java.nio.charset.StandardCharsets;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collections; import java.util.Collections;
import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
public class SystemSchemaTest extends CalciteTestBase public class SystemSchemaTest extends CalciteTestBase
{ {
@ -484,19 +483,18 @@ public class SystemSchemaTest extends CalciteTestBase
@Test @Test
public void testSegmentsTable() public void testSegmentsTable()
{ {
final SystemSchema.SegmentsTable segmentsTable = EasyMock final SystemSchema.SegmentsTable segmentsTable = EasyMock
.createMockBuilder(SystemSchema.SegmentsTable.class) .createMockBuilder(SystemSchema.SegmentsTable.class)
.withConstructor(druidSchema, metadataView, mapper, authMapper) .withConstructor(druidSchema, metadataView, mapper, authMapper)
.createMock(); .createMock();
EasyMock.replay(segmentsTable); EasyMock.replay(segmentsTable);
final Set<SegmentWithOvershadowedStatus> publishedSegments = Stream.of( final Set<SegmentWithOvershadowedStatus> publishedSegments = new HashSet<>(Arrays.asList(
new SegmentWithOvershadowedStatus(publishedSegment1, true), new SegmentWithOvershadowedStatus(publishedSegment1, true),
new SegmentWithOvershadowedStatus(publishedSegment2, false), new SegmentWithOvershadowedStatus(publishedSegment2, false),
new SegmentWithOvershadowedStatus(publishedSegment3, false), new SegmentWithOvershadowedStatus(publishedSegment3, false),
new SegmentWithOvershadowedStatus(segment1, true), new SegmentWithOvershadowedStatus(segment1, true),
new SegmentWithOvershadowedStatus(segment2, false) new SegmentWithOvershadowedStatus(segment2, false)
).collect(Collectors.toSet()); ));
EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once(); EasyMock.expect(metadataView.getPublishedSegments()).andReturn(publishedSegments.iterator()).once();

View File

@ -78,9 +78,8 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C
) )
{ {
final Segment segment = new QueryableIndexSegment(index, descriptor.getId()); final Segment segment = new QueryableIndexSegment(index, descriptor.getId());
timelines.computeIfAbsent(descriptor.getDataSource(), datasource -> new VersionedIntervalTimeline<>(Ordering.natural())); final VersionedIntervalTimeline<String, Segment> timeline = timelines
.computeIfAbsent(descriptor.getDataSource(), dsName -> new VersionedIntervalTimeline<>(Ordering.natural()));
final VersionedIntervalTimeline<String, Segment> timeline = timelines.get(descriptor.getDataSource());
timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment)); timeline.add(descriptor.getInterval(), descriptor.getVersion(), descriptor.getShardSpec().createChunk(segment));
segments.add(descriptor); segments.add(descriptor);
closeables.add(index); closeables.add(index);

View File

@ -59,7 +59,7 @@ POST /druid/indexer/v1/worker
GET /druid/indexer/v1/workers GET /druid/indexer/v1/workers
GET /druid/coordinator/v1/loadqueue?simple GET /druid/coordinator/v1/loadqueue?simple
GET /druid/coordinator/v1/config 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/rules
GET /druid/coordinator/v1/config/compaction GET /druid/coordinator/v1/config/compaction
GET /druid/coordinator/v1/tiers GET /druid/coordinator/v1/tiers