mirror of https://github.com/apache/druid.git
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:
parent
b80f20f769
commit
ceb969903f
|
@ -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">
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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 +
|
||||||
'}';
|
'}';
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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;
|
|
@ -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());
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
{
|
{
|
||||||
|
|
|
@ -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}`
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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
|
||||||
{
|
{
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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 +
|
||||||
|
|
|
@ -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 +
|
||||||
|
|
|
@ -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
|
||||||
{
|
{
|
||||||
|
|
|
@ -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 = {
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -33,4 +33,9 @@ public class MetadataSegmentManagerConfig
|
||||||
{
|
{
|
||||||
return pollDuration;
|
return pollDuration;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void setPollDuration(Period pollDuration)
|
||||||
|
{
|
||||||
|
this.pollDuration = pollDuration;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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)
|
||||||
|
|
|
@ -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()) {
|
||||||
|
|
|
@ -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
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)) {
|
||||||
|
|
|
@ -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())
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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()) {
|
||||||
|
|
|
@ -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();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
*/
|
*/
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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
|
||||||
)
|
)
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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<>(
|
||||||
|
|
|
@ -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()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
}
|
|
@ -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());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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) {
|
||||||
|
|
|
@ -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()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
}
|
|
@ -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));
|
||||||
|
|
|
@ -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())
|
||||||
|
|
|
@ -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));
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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();
|
||||||
|
|
|
@ -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)
|
||||||
{
|
{
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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(
|
||||||
|
|
|
@ -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
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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
|
||||||
);
|
);
|
||||||
|
|
|
@ -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))
|
||||||
)
|
)
|
||||||
|
|
|
@ -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();
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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
|
||||||
|
|
Loading…
Reference in New Issue