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

* Refactor SQLMetadataSegmentManager; Change contract of REST methods in DataSourcesResource

* Style fixes

* Unused imports

* Fix tests

* Fix style

* Comments

* Comment fix

* Remove unresolvable Javadoc references; address comments

* Add comments to ImmutableDruidDataSource

* Merge with master

* Fix bad web-console merge

* Fixes in api-reference.md

* Rename in DruidCoordinatorRuntimeParams

* Fix compilation

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

View File

@ -311,6 +311,11 @@
<constraint name="k" within="" contains="" />
<constraint name="v" within="" contains="" />
</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 class="SimplifyStreamApiCallChains" enabled="true" level="ERROR" enabled_by_default="true" />
<inspection_tool class="SpellCheckingInspection" enabled="false" level="TYPO" enabled_by_default="false">

View File

@ -20,6 +20,8 @@
package org.apache.druid.server.coordinator;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator;
import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy;
@ -42,6 +44,7 @@ import org.openjdk.jmh.annotations.Setup;
import org.openjdk.jmh.annotations.State;
import org.openjdk.jmh.infra.Blackhole;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@ -102,14 +105,10 @@ public class NewestSegmentFirstPolicyBenchmark
);
}
dataSources = new HashMap<>();
List<DataSegment> segments = new ArrayList<>();
for (int i = 0; i < numDataSources; 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);
DateTime date = DateTimes.of(startYear, 1, 1, 0, 0);
@ -127,12 +126,11 @@ public class NewestSegmentFirstPolicyBenchmark
0,
segmentSizeBytes
);
timeline.add(segment.getInterval(), segment.getVersion(), shardSpec.createChunk(segment));
segments.add(segment);
}
}
dataSources.put(dataSource, timeline);
}
dataSources = DataSourcesSnapshot.fromUsedSegments(segments, ImmutableMap.of()).getUsedSegmentsTimelinesPerDataSource();
}
@Benchmark

View File

@ -123,6 +123,11 @@ public class Logger
log.error(StringUtils.nonStrictFormat(message, formatArgs), t);
}
public void assertionError(String message, Object... formatArgs)
{
log.error("ASSERTION_ERROR: " + message, formatArgs);
}
public void wtf(String message, Object... formatArgs)
{
log.error(StringUtils.nonStrictFormat("WTF?!: " + message, formatArgs), new Exception());

View File

@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.AlertBuilder;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import javax.annotation.Nullable;
import java.io.PrintWriter;
import java.io.StringWriter;
@ -61,15 +62,22 @@ public class EmittingLogger extends Logger
return makeAlert(null, message, objects);
}
public AlertBuilder makeAlert(Throwable t, String message, Object... objects)
public AlertBuilder makeAlert(@Nullable Throwable t, String message, Object... objects)
{
if (emitter == null) {
final String errorMessage = StringUtils.format(
"Emitter not initialized! Cannot alert. Please make sure to call %s.registerEmitter()", this.getClass()
"Emitter not initialized! Cannot alert. Please make sure to call %s.registerEmitter()\n"
+ "Message: %s",
this.getClass(),
StringUtils.nonStrictFormat(message, objects)
);
error(errorMessage);
throw new ISE(errorMessage);
ISE e = new ISE(errorMessage);
if (t != null) {
e.addSuppressed(t);
}
throw e;
}
final AlertBuilder retVal = new EmittingAlertBuilder(t, StringUtils.format(message, objects), emitter)

View File

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

View File

@ -23,16 +23,18 @@ import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterators;
import com.google.common.collect.Ordering;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.timeline.partition.ImmutablePartitionHolder;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.PartitionHolder;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
@ -44,7 +46,9 @@ import java.util.NavigableMap;
import java.util.Objects;
import java.util.Set;
import java.util.TreeMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import java.util.stream.StreamSupport;
/**
* VersionedIntervalTimeline is a data structure that manages objects on a specific timeline.
@ -75,12 +79,11 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
Comparators.intervalsByStartThenEnd()
);
private final Map<Interval, TreeMap<VersionType, TimelineEntry>> allTimelineEntries = new HashMap<>();
private final AtomicInteger numObjects = new AtomicInteger();
private final Comparator<? super VersionType> versionComparator;
public VersionedIntervalTimeline(
Comparator<? super VersionType> versionComparator
)
public VersionedIntervalTimeline(Comparator<? super VersionType> versionComparator)
{
this.versionComparator = versionComparator;
}
@ -92,7 +95,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
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);
return timeline;
}
@ -115,6 +119,28 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
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)
{
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);
versionEntry.put(version, entry);
allTimelineEntries.put(interval, versionEntry);
numObjects.incrementAndGet();
} else {
entry = exists.get(version);
if (entry == null) {
entry = new TimelineEntry(interval, version, new PartitionHolder<>(object));
exists.put(version, entry);
numObjects.incrementAndGet();
} else {
PartitionHolder<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)
{
try {
@ -189,7 +220,11 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
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()) {
versionEntries.remove(version);
if (versionEntries.isEmpty()) {
@ -201,7 +236,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
remove(completePartitionsTimeline, interval, entry, false);
return retVal;
return removedChunk;
}
finally {
lock.writeLock().unlock();
@ -209,7 +244,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
}
@Override
public PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version)
public @Nullable PartitionHolder<ObjectType> findEntry(Interval interval, VersionType version)
{
try {
lock.readLock().lock();
@ -217,9 +252,7 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
if (entry.getKey().equals(interval) || entry.getKey().contains(interval)) {
TimelineEntry foundEntry = entry.getValue().get(version);
if (foundEntry != null) {
return new ImmutablePartitionHolder<ObjectType>(
foundEntry.getPartitionHolder()
);
return new ImmutablePartitionHolder<>(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()
{
try {
@ -315,8 +352,8 @@ public class VersionedIntervalTimeline<VersionType, ObjectType> implements Timel
Map<Interval, Map<VersionType, TimelineEntry>> overShadowed = new HashMap<>();
for (Map.Entry<Interval, TreeMap<VersionType, TimelineEntry>> versionEntry : allTimelineEntries.entrySet()) {
Map<VersionType, TimelineEntry> versionCopy = new HashMap<>();
versionCopy.putAll(versionEntry.getValue());
@SuppressWarnings("unchecked")
Map<VersionType, TimelineEntry> versionCopy = (TreeMap) versionEntry.getValue().clone();
overShadowed.put(versionEntry.getKey(), versionCopy);
}

View File

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

View File

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

View File

@ -19,13 +19,16 @@
package org.apache.druid.utils;
import com.google.common.collect.Iterables;
import com.google.common.collect.Maps;
import java.util.AbstractCollection;
import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator;
import java.util.Map;
import java.util.Spliterator;
import java.util.TreeSet;
import java.util.function.Function;
import java.util.function.Supplier;
import java.util.stream.Stream;
@ -72,6 +75,13 @@ public 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
* function.

View File

@ -0,0 +1,23 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
@EverythingIsNonnullByDefault
package org.apache.druid.utils;
import org.apache.druid.annotations.EverythingIsNonnullByDefault;

View File

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

View File

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

View File

@ -113,15 +113,17 @@ Returns the serialized JSON of segments to load and drop for each Historical pro
* `/druid/coordinator/v1/metadata/datasources`
Returns a list of the names of enabled datasources in the cluster.
Returns a list of the names of data sources with at least one used segment in the cluster.
* `/druid/coordinator/v1/metadata/datasources?includeDisabled`
* `/druid/coordinator/v1/metadata/datasources?includeUnused`
Returns a list of the names of enabled and disabled datasources in the cluster.
Returns a list of the names of data sources, regardless of whether there are used segments belonging to those data
sources in the cluster or not.
* `/druid/coordinator/v1/metadata/datasources?full`
Returns a list of all enabled datasources with all metadata about those datasources as stored in the metadata store.
Returns a list of all data sources with at least one used segment in the cluster. Returns all metadata about those data
sources as stored in the metadata store.
* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}`
@ -229,11 +231,15 @@ Caution : Avoid using indexing or kill tasks and these API's at the same time fo
* `/druid/coordinator/v1/datasources/{dataSourceName}`
Enables all segments of datasource which are not overshadowed by others.
Marks as used all segments belonging to a data source. Returns a JSON object of the form
`{"numChangedSegments": <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}`
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`
@ -259,7 +265,9 @@ JSON Request Payload:
* `/druid/coordinator/v1/datasources/{dataSourceName}`
Disables a datasource.
Marks as unused all segments belonging to a data source. Returns a JSON object of the form
`{"numChangedSegments": <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}`
* `@Deprecated. /druid/coordinator/v1/datasources/{dataSourceName}?kill=true&interval={myInterval}`
@ -268,7 +276,9 @@ Runs a [Kill task](../ingestion/tasks.html) for a given interval and datasource.
* `/druid/coordinator/v1/datasources/{dataSourceName}/segments/{segmentId}`
Disables a segment.
Marks as unused a segment of a data source. Returns a JSON object of the form `{"segmentStateChanged": <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
@ -595,7 +605,7 @@ Note that all _interval_ URL parameters are ISO 8601 strings delimited by a `_`
* `/druid/indexer/v1/worker`
Retreives current overlord dynamic configuration.
Retrieves current overlord dynamic configuration.
* `/druid/indexer/v1/worker/history?interval={interval}&counter={count}`

View File

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

View File

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

View File

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

View File

@ -73,7 +73,9 @@ public class SegmentListActionsTest
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)

View File

@ -19,11 +19,13 @@
package org.apache.druid.client;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Ordering;
import com.google.common.collect.Maps;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -31,40 +33,88 @@ import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
/**
* An immutable snapshot of fields from {@link org.apache.druid.metadata.SQLMetadataSegmentManager} (dataSources and
* overshadowedSegments). Getters of {@link org.apache.druid.metadata.MetadataSegmentManager} should use this snapshot
* to return dataSources and overshadowedSegments.
* An immutable snapshot information about used segments and overshadowed segments for
* {@link org.apache.druid.metadata.SQLMetadataSegmentManager}.
*/
public class DataSourcesSnapshot
{
private final Map<String, ImmutableDruidDataSource> dataSources;
private final ImmutableSet<SegmentId> overshadowedSegments;
public DataSourcesSnapshot(
Map<String, ImmutableDruidDataSource> dataSources
public static DataSourcesSnapshot fromUsedSegments(
Iterable<DataSegment> segments,
ImmutableMap<String, String> dataSourceProperties
)
{
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());
}
public Collection<ImmutableDruidDataSource> getDataSources()
public Collection<ImmutableDruidDataSource> getDataSourcesWithAllUsedSegments()
{
return dataSources.values();
return dataSourcesWithAllUsedSegments.values();
}
public Map<String, ImmutableDruidDataSource> getDataSourcesMap()
{
return dataSources;
return dataSourcesWithAllUsedSegments;
}
@Nullable
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()
@ -72,40 +122,48 @@ public class DataSourcesSnapshot
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 null;
}
return () -> dataSources.values().stream()
.flatMap(dataSource -> dataSource.getSegments().stream())
.iterator();
return () -> dataSourcesWithAllUsedSegments
.values()
.stream()
.flatMap(dataSource -> dataSource.getSegments().stream())
.iterator();
}
/**
* This method builds timelines from all dataSources and finds the overshadowed segments list
* This method builds timelines from all data sources and finds the overshadowed segments list
*
* This method should be deduplicated with {@link VersionedIntervalTimeline#findOvershadowed()}: see
* https://github.com/apache/incubator-druid/issues/8070.
*
* @return overshadowed segment Ids list
*/
private List<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
// a small fraction of the segments in the cluster are expected to be overshadowed,
// so building this collection shouldn't generate a lot of garbage.
final List<SegmentId> overshadowedSegments = new ArrayList<>();
for (DataSegment dataSegment : segments) {
final VersionedIntervalTimeline<String, DataSegment> timeline = timelines.get(dataSegment.getDataSource());
if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) {
overshadowedSegments.add(dataSegment.getId());
for (ImmutableDruidDataSource dataSource : dataSourcesWithAllUsedSegments.values()) {
VersionedIntervalTimeline<String, DataSegment> usedSegmentsTimeline =
usedSegmentsTimelinesPerDataSource.get(dataSource.getName());
for (DataSegment segment : dataSource.getSegments()) {
if (usedSegmentsTimeline.isOvershadowed(segment.getInterval(), segment.getVersion())) {
overshadowedSegments.add(segment.getId());
}
}
}
return overshadowedSegments;

View File

@ -30,12 +30,13 @@ import java.util.Collections;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
import java.util.function.Predicate;
/**
* A mutable collection of metadata of segments ({@link DataSegment} objects), belonging to a particular data source.
*
* Concurrency: could be updated concurrently via {@link #addSegment} and {@link #removeSegment}, and accessed
* concurrently (e. g. via {@link #getSegments}) as well.
* Concurrency: could be updated concurrently via {@link #addSegment}, {@link #removeSegment}, and {@link
* #removeSegmentsIf}, and accessed concurrently (e. g. via {@link #getSegments}) as well.
*
* @see ImmutableDruidDataSource - an immutable counterpart of this class
*/
@ -44,10 +45,7 @@ public class DruidDataSource
private final String name;
private final Map<String, String> properties;
/**
* This map needs to be concurrent because it should be possible to iterate the segments of the data source
* (indirectly via {@link #getSegments} or in {@link #toString}) concurrently updates via {@link #addSegment} or
* {@link #removeSegment}. Concurrent updates are also supported incidentally, though this is not needed for the use
* cases of DruidDataSource.
* This map needs to be concurrent to support concurrent iteration and updates.
*/
private final ConcurrentMap<SegmentId, DataSegment> idToSegmentMap = new ConcurrentHashMap<>();
@ -80,6 +78,14 @@ public class DruidDataSource
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)
{
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
* DruidDataSource.
*/
public DataSegment removeSegment(SegmentId segmentId)
public @Nullable DataSegment removeSegment(SegmentId segmentId)
{
return idToSegmentMap.remove(segmentId);
}
@ -126,7 +132,6 @@ public class DruidDataSource
@Override
public boolean equals(Object o)
{
//noinspection Contract
throw new UnsupportedOperationException("Use ImmutableDruidDataSource instead");
}

View File

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

View File

@ -123,6 +123,8 @@ public class ImmutableDruidDataSource
@Override
public boolean equals(Object o)
{
// Note: this method is not well-defined. It should instead just throw UnsupportedOperationsException.
// See https://github.com/apache/incubator-druid/issues/7858.
if (this == o) {
return true;
}
@ -146,6 +148,8 @@ public class ImmutableDruidDataSource
@Override
public int hashCode()
{
return Objects.hash(name, properties, idToSegments);
// Note: this method is not well-defined. It should instead just throw UnsupportedOperationsException.
// See https://github.com/apache/incubator-druid/issues/7858.
return Objects.hash(name, properties);
}
}

View File

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

View File

@ -29,6 +29,10 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* Client representation of org.apache.druid.indexing.common.task.CompactionTask. JSON serialization fields of
* this class must correspond to those of org.apache.druid.indexing.common.task.CompactionTask.
*/
public class ClientCompactQuery implements ClientQuery
{
private final String dataSource;
@ -136,7 +140,7 @@ public class ClientCompactQuery implements ClientQuery
@Override
public String toString()
{
return "ClientCompactQuery{" +
return getClass().getSimpleName() + "{" +
"dataSource='" + dataSource + '\'' +
", segments=" + segments +
", interval=" + interval +

View File

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

View File

@ -24,6 +24,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
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
{

View File

@ -24,7 +24,12 @@ import com.fasterxml.jackson.annotation.JsonSubTypes.Type;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
/**
* org.apache.druid.indexing.common.task.Task representation for clients
* org.apache.druid.indexing.common.task.Task representations for clients. The magic conversion happens right
* at the moment of making a REST query: {@link HttpIndexingServiceClient#runTask} serializes ClientTaskQuery
* objects and org.apache.druid.indexing.overlord.http.OverlordResource.taskPost() deserializes
* org.apache.druid.indexing.common.task.Task objects from the same bytes. Therefore JSON serialization fields of
* ClientTaskQuery objects must match with those of the corresponding
* org.apache.druid.indexing.common.task.Task objects.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {

View File

@ -24,6 +24,7 @@ import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.client.ImmutableDruidDataSource;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -32,89 +33,105 @@ import java.util.List;
import java.util.Set;
/**
* The difference between this class and org.apache.druid.sql.calcite.schema.MetadataSegmentView is that this
* class resides in Coordinator's memory, while org.apache.druid.sql.calcite.schema.MetadataSegmentView resides
* in Broker's memory.
*/
public interface MetadataSegmentManager
{
void start();
void startPollingDatabasePeriodically();
void stop();
void stopPollingDatabasePeriodically();
boolean isPollingDatabasePeriodically();
/**
* Enables all segments for a dataSource which will not be overshadowed.
* Returns the number of segment entries in the database whose state was changed as the result of this call (that is,
* the segments were marked as used). If the call results in a database error, an exception is relayed to the caller.
*/
boolean enableDataSource(String dataSource);
int markAsUsedAllNonOvershadowedSegmentsInDataSource(String dataSource);
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);
long disableSegments(String dataSource, Collection<String> segmentIds);
int disableSegments(String dataSource, Interval interval);
boolean isStarted();
@Nullable
ImmutableDruidDataSource getDataSource(String dataSourceName);
boolean markSegmentAsUnused(String segmentId);
/**
* Returns a collection of known datasources.
*
* Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has
* not yet been polled.)
* If there are used segments belonging to the given data source this method returns them as an {@link
* ImmutableDruidDataSource} object. If there are no used segments belonging to the given data source this method
* returns null.
*/
@Nullable
Collection<ImmutableDruidDataSource> getDataSources();
@Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSource);
/**
* 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
* unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try
* (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than
* several times.
*
* Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has
* not yet been polled.)
*/
@Nullable
Iterable<DataSegment> iterateAllSegments();
Collection<String> getAllDataSourceNames();
Iterable<DataSegment> iterateAllUsedSegments();
/**
* Returns a set of overshadowed segment Ids
* Retrieves all data source names for which there are segment in the database, regardless of whether those segments
* are used or not. Data source names in the returned collection are unique. If there are no segments in the database,
* returns an empty collection.
*
* Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has
* not yet been polled.)
* Performance warning: this method makes a query into the database.
*
* This method might return a different set of data source names than may be observed via {@link
* #getImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there
* are no used segments belonging to it, while {@link #getImmutableDataSourcesWithAllUsedSegments} won't return
* such a data source.
*/
@Nullable
Set<SegmentId> getOvershadowedSegments();
Collection<String> retrieveAllDataSourceNames();
/**
* 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
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);
List<Interval> getUnusedSegmentIntervals(String dataSource, DateTime maxEndTime, int limit);
@VisibleForTesting
void poll();

View File

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

View File

@ -200,7 +200,7 @@ public class SQLMetadataRuleManager implements MetadataRuleManager
{
try {
// poll() is synchronized together with start() and stop() to ensure that when stop() exits, poll()
// won't actually run anymore after that (it could only enter the syncrhonized section and exit
// won't actually run anymore after that (it could only enter the synchronized section and exit
// immediately because the localStartedOrder doesn't match the new currentStartOrder). It's needed
// to avoid flakiness in SQLMetadataRuleManagerTest.
// See https://github.com/apache/incubator-druid/issues/6028

View File

@ -19,13 +19,23 @@
package org.apache.druid.metadata;
import java.util.Collection;
/**
* Exception thrown by MetadataSegmentManager when an segment id is unknown.
* Exception thrown by {@link MetadataSegmentManager} when a segment id is unknown.
*/
public class UnknownSegmentIdException extends RuntimeException
public class UnknownSegmentIdException extends Exception
{
public UnknownSegmentIdException(String message)
private final Collection<String> unknownSegmentIds;
UnknownSegmentIdException(Collection<String> segmentIds)
{
super(message);
super("Cannot find segment ids " + segmentIds);
this.unknownSegmentIds = segmentIds;
}
public Collection<String> getUnknownSegmentIds()
{
return unknownSegmentIds;
}
}

View File

@ -19,10 +19,16 @@
package org.apache.druid.server;
import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
import javax.ws.rs.core.MultivaluedMap;
import javax.ws.rs.core.UriInfo;
public class JettyUtils
{
private static final Logger log = new Logger(JettyUtils.class);
/**
* Concatenate URI parts, in a way that is useful for proxy servlets.
*
@ -46,4 +52,38 @@ public class JettyUtils
return url.toString();
}
/**
* Returns the value of the query parameter of the given name. If not found, but there is a value corresponding to
* the parameter of the given compatiblityName it is returned instead and a warning is logged suggestion to make
* queries using the new parameter name.
*
* This method is useful for renaming query parameters (from name to compatiblityName) while preserving backward
* compatibility of the REST API.
*/
@Nullable
public static String getQueryParam(UriInfo uriInfo, String name, String compatiblityName)
{
MultivaluedMap<String, String> queryParameters = uriInfo.getQueryParameters();
// Returning the first value, according to the @QueryParam spec:
// https://docs.oracle.com/javaee/7/api/javax/ws/rs/QueryParam.html:
// "If the type is not one of the collection types listed in 5 above and the query parameter is represented by
// multiple values then the first value (lexically) of the parameter is used."
String paramValue = queryParameters.getFirst(name);
if (paramValue != null) {
return paramValue;
}
String compatibilityParamValue = queryParameters.getFirst(compatiblityName);
if (compatibilityParamValue != null) {
log.warn(
"Parameter %s in %s query has been renamed to %s. Use the new parameter name.",
compatiblityName,
uriInfo.getPath(),
name
);
return compatibilityParamValue;
}
// Not found neither name nor compatiblityName
return null;
}
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -76,7 +76,8 @@ public class DruidCoordinatorSegmentCompactor implements DruidCoordinatorHelper
final CoordinatorStats stats = new CoordinatorStats();
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();
if (compactionConfigList != null && !compactionConfigList.isEmpty()) {

View File

@ -19,14 +19,12 @@
package org.apache.druid.server.coordinator.helper;
import com.google.common.collect.Iterables;
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.coordinator.DruidCoordinator;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.apache.druid.timeline.DataSegment;
import java.util.TreeSet;
public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper
{
private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class);
@ -41,50 +39,27 @@ public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper
@Override
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();
if (dataSegments == null) {
log.info("Metadata store not polled yet, canceling this run.");
return null;
DataSourcesSnapshot dataSourcesSnapshot = params.getDataSourcesSnapshot();
for (DataSegment segment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) {
if (segment.getSize() < 0) {
log.makeAlert("No size on a segment")
.addData("segment", segment)
.emit();
}
}
// The following transform() call doesn't actually transform the iterable. It only checks the sizes of the segments
// and emits alerts if segments with negative sizes are encountered. In other words, semantically it's similar to
// Stream.peek(). It works as long as DruidCoordinatorRuntimeParams.createAvailableSegmentsSet() (which is called
// below) guarantees to go over the passed iterable exactly once.
//
// An iterable returned from iterateAvailableDataSegments() is not simply iterated (with size checks) before passing
// into DruidCoordinatorRuntimeParams.createAvailableSegmentsSet() because iterateAvailableDataSegments()'s
// documentation says to strive to avoid iterating the result more than once.
//
//noinspection StaticPseudoFunctionalStyleMethod: https://youtrack.jetbrains.com/issue/IDEA-153047
Iterable<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
// Log info about all used segments
if (log.isDebugEnabled()) {
log.debug("Available DataSegments");
for (DataSegment dataSegment : availableSegments) {
log.debug("Used Segments");
for (DataSegment dataSegment : dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot()) {
log.debug(" %s", dataSegment);
}
}
log.info("Found [%,d] available segments.", availableSegments.size());
log.info("Found [%,d] used segments.", params.getUsedSegments().size());
return params.buildFromExisting()
.setAvailableSegments(availableSegments)
.build();
return params;
}
}

View File

@ -31,10 +31,15 @@ import org.apache.druid.server.coordinator.DruidCoordinatorConfig;
import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.util.Collection;
import java.util.List;
/**
* Completely removes information about unused segments whose end time is older than {@link #retainDuration} from now
* from the metadata store. This action is called "to kill a segment".
*
* See org.apache.druid.indexing.common.task.KillTask
*/
public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
{
@ -46,12 +51,12 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
private long lastKillTime = 0;
private final MetadataSegmentManager segmentManager;
private final MetadataSegmentManager segmentsMetadata;
private final IndexingServiceClient indexingServiceClient;
@Inject
public DruidCoordinatorSegmentKiller(
MetadataSegmentManager segmentManager,
MetadataSegmentManager segmentsMetadata,
IndexingServiceClient indexingServiceClient,
DruidCoordinatorConfig config
)
@ -75,30 +80,36 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
this.maxSegmentsToKill
);
this.segmentManager = segmentManager;
this.segmentsMetadata = segmentsMetadata;
this.indexingServiceClient = indexingServiceClient;
}
@Override
public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
{
boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillAllDataSources();
Collection<String> whitelist = params.getCoordinatorDynamicConfig().getKillableDataSources();
boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources();
Collection<String> specificDataSourcesToKill =
params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn();
if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) {
log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled.");
if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) {
log.error(
"killAllDataSources can't be true when specificDataSourcesToKill is non-empty. No kill tasks are scheduled."
);
return params;
}
Collection<String> dataSourcesToKill = specificDataSourcesToKill;
if (killAllDataSources) {
whitelist = segmentManager.getAllDataSourceNames();
dataSourcesToKill = segmentsMetadata.retrieveAllDataSourceNames();
}
if (whitelist != null && whitelist.size() > 0 && (lastKillTime + period) < System.currentTimeMillis()) {
if (dataSourcesToKill != null &&
dataSourcesToKill.size() > 0 &&
(lastKillTime + period) < System.currentTimeMillis()) {
lastKillTime = System.currentTimeMillis();
for (String dataSource : whitelist) {
final Interval intervalToKill = findIntervalForKillTask(dataSource, maxSegmentsToKill);
for (String dataSource : dataSourcesToKill) {
final Interval intervalToKill = findIntervalForKill(dataSource, maxSegmentsToKill);
if (intervalToKill != null) {
try {
indexingServiceClient.killSegments(dataSource, intervalToKill);
@ -117,13 +128,11 @@ public class DruidCoordinatorSegmentKiller implements DruidCoordinatorHelper
}
@VisibleForTesting
Interval findIntervalForKillTask(String dataSource, int limit)
@Nullable
Interval findIntervalForKill(String dataSource, int limit)
{
List<Interval> unusedSegmentIntervals = segmentManager.getUnusedSegmentIntervals(
dataSource,
new Interval(DateTimes.EPOCH, DateTimes.nowUtc().minus(retainDuration)),
limit
);
List<Interval> unusedSegmentIntervals =
segmentsMetadata.getUnusedSegmentIntervals(dataSource, DateTimes.nowUtc().minus(retainDuration), limit);
if (unusedSegmentIntervals != null && unusedSegmentIntervals.size() > 0) {
return JodaUtils.umbrellaInterval(unusedSegmentIntervals);

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,39 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.client;
import org.apache.druid.timeline.DataSegment;
import org.easymock.EasyMock;
import java.util.Collection;
public final class ImmutableDruidServerTests
{
public static void expectSegments(ImmutableDruidServer mockServer, Collection<DataSegment> segments)
{
EasyMock.expect(mockServer.iterateAllSegments()).andReturn(segments).anyTimes();
EasyMock.expect(mockServer.getNumSegments()).andReturn(segments.size()).anyTimes();
}
private ImmutableDruidServerTests()
{
}
}

View File

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

View File

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

View File

@ -0,0 +1,44 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.server.coordinator;
import org.apache.druid.java.util.common.DateTimes;
public class CoordinatorRuntimeParamsTestHelpers
{
public static DruidCoordinatorRuntimeParams.Builder newBuilder()
{
return DruidCoordinatorRuntimeParams
.newBuilder()
.withStartTimeNanos(System.nanoTime())
.withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"));
}
public static DruidCoordinatorRuntimeParams.Builder newBuilder(DruidCluster druidCluster)
{
return newBuilder()
.withDruidCluster(druidCluster)
.withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster));
}
private CoordinatorRuntimeParamsTestHelpers()
{
}
}

View File

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

View File

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

View File

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

View File

@ -0,0 +1,61 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.server.coordinator;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
public final class DruidClusterBuilder
{
public static DruidClusterBuilder newBuilder()
{
return new DruidClusterBuilder();
}
private @Nullable Set<ServerHolder> realtimes = null;
private final Map<String, Iterable<ServerHolder>> historicals = new HashMap<>();
private DruidClusterBuilder()
{
}
public DruidClusterBuilder withRealtimes(ServerHolder... realtimes)
{
this.realtimes = new HashSet<>(Arrays.asList(realtimes));
return this;
}
public DruidClusterBuilder addTier(String tierName, ServerHolder... historicals)
{
if (this.historicals.putIfAbsent(tierName, Arrays.asList(historicals)) != null) {
throw new IllegalArgumentException("Duplicate tier: " + tierName);
}
return this;
}
public DruidCluster build()
{
return DruidCluster.createDruidClusterFromBuilderInTest(realtimes, historicals);
}
}

View File

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

View File

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

View File

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

View File

@ -25,17 +25,8 @@ import org.apache.druid.server.coordinator.helper.DruidCoordinatorBalancer;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import java.util.Comparator;
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)
{

View File

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

View File

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

View File

@ -115,7 +115,7 @@ public class CachingCostBalancerStrategyTest
{
ClusterCostCache.Builder builder = ClusterCostCache.builder();
serverHolders.forEach(
s -> s.getServer().getLazyAllSegments().forEach(segment -> builder.addSegment(s.getServer().getName(), segment))
s -> s.getServer().iterateAllSegments().forEach(segment -> builder.addSegment(s.getServer().getName(), segment))
);
return new CachingCostBalancerStrategy(builder.build(), listeningExecutorService);
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -59,7 +59,7 @@ POST /druid/indexer/v1/worker
GET /druid/indexer/v1/workers
GET /druid/coordinator/v1/loadqueue?simple
GET /druid/coordinator/v1/config
GET /druid/coordinator/v1/metadata/datasources?includeDisabled
GET /druid/coordinator/v1/metadata/datasources?includeUnused
GET /druid/coordinator/v1/rules
GET /druid/coordinator/v1/config/compaction
GET /druid/coordinator/v1/tiers