You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by le...@apache.org on 2019/07/17 14:19:01 UTC

[incubator-druid] branch master updated: Refactor SQLMetadataSegmentManager; Change contract of REST met… (#7653)

This is an automated email from the ASF dual-hosted git repository.

leventov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-druid.git


The following commit(s) were added to refs/heads/master by this push:
     new ceb9699  Refactor SQLMetadataSegmentManager; Change contract of REST met… (#7653)
ceb9699 is described below

commit ceb969903f06a72d669bb55ab83223bddb5355ee
Author: Roman Leventov <le...@gmail.com>
AuthorDate: Wed Jul 17 17:18:48 2019 +0300

    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
---
 .idea/inspectionProfiles/Druid.xml                 |    5 +
 .../NewestSegmentFirstPolicyBenchmark.java         |   14 +-
 .../druid/java/util/common/logger/Logger.java      |    5 +
 .../druid/java/util/emitter/EmittingLogger.java    |   14 +-
 .../org/apache/druid/timeline/TimelineLookup.java  |    3 +-
 .../druid/timeline/VersionedIntervalTimeline.java  |   65 +-
 .../partition/ImmutablePartitionHolder.java        |    4 +-
 .../druid/timeline/partition/PartitionHolder.java  |   52 +-
 .../org/apache/druid/utils/CollectionUtils.java    |   10 +
 .../java/org/apache/druid/utils/package-info.java  |   14 +-
 .../timeline/VersionedIntervalTimelineTest.java    |   32 +-
 docs/content/ingestion/delete-data.md              |    3 +-
 docs/content/operations/api-reference.md           |   28 +-
 .../MaterializedViewSupervisor.java                |    4 +-
 .../druid/indexing/common/task/IndexTask.java      |    4 +-
 .../druid/indexing/common/task/KillTask.java       |    4 +
 .../common/actions/SegmentListActionsTest.java     |    4 +-
 .../apache/druid/client/DataSourcesSnapshot.java   |  126 ++-
 .../org/apache/druid/client/DruidDataSource.java   |   21 +-
 .../java/org/apache/druid/client/DruidServer.java  |    2 -
 .../druid/client/ImmutableDruidDataSource.java     |    6 +-
 .../apache/druid/client/ImmutableDruidServer.java  |   21 +-
 .../druid/client/indexing/ClientCompactQuery.java  |    6 +-
 .../indexing/ClientCompactQueryTuningConfig.java   |   14 +-
 .../druid/client/indexing/ClientKillQuery.java     |    3 +
 .../apache/druid/client/indexing/ClientQuery.java  |    7 +-
 .../druid/metadata/MetadataSegmentManager.java     |  105 +-
 .../metadata/MetadataSegmentManagerConfig.java     |    5 +
 .../druid/metadata/SQLMetadataRuleManager.java     |    2 +-
 .../druid/metadata/SQLMetadataSegmentManager.java  | 1010 +++++++++++--------
 .../druid/metadata/UnknownSegmentIdException.java  |   18 +-
 .../java/org/apache/druid/server/JettyUtils.java   |   40 +
 .../coordinator/CoordinatorDynamicConfig.java      |  207 ++--
 .../server/coordinator/CostBalancerStrategy.java   |   24 +-
 .../DiskNormalizedCostBalancerStrategy.java        |    4 +-
 .../druid/server/coordinator/DruidCluster.java     |   29 +-
 .../druid/server/coordinator/DruidCoordinator.java |  133 +--
 .../DruidCoordinatorCleanupPendingSegments.java    |   12 +-
 .../coordinator/DruidCoordinatorRuntimeParams.java |  179 ++--
 .../coordinator/ReservoirSegmentSampler.java       |    2 +-
 .../server/coordinator/SegmentReplicantLookup.java |    2 +-
 .../helper/DruidCoordinatorBalancer.java           |   20 +-
 .../DruidCoordinatorCleanupOvershadowed.java       |   53 +-
 .../helper/DruidCoordinatorCleanupUnneeded.java    |   13 +-
 .../coordinator/helper/DruidCoordinatorLogger.java |   13 +-
 .../helper/DruidCoordinatorRuleRunner.java         |   27 +-
 .../helper/DruidCoordinatorSegmentCompactor.java   |    3 +-
 .../helper/DruidCoordinatorSegmentInfoLoader.java  |   53 +-
 .../helper/DruidCoordinatorSegmentKiller.java      |   45 +-
 .../druid/server/coordinator/rules/DropRule.java   |    2 +-
 .../druid/server/coordinator/rules/Rule.java       |    8 +-
 .../druid/server/http/CoordinatorResource.java     |    2 +-
 .../druid/server/http/DataSourcesResource.java     |  437 ++++-----
 .../apache/druid/server/http/MetadataResource.java |  173 ++--
 .../druid/client/CachingClusteredClientTest.java   |    3 +-
 .../druid/client/ImmutableDruidServerTests.java    |   25 +-
 .../druid/client/indexing/ClientKillQueryTest.java |   12 +-
 .../metadata/SQLMetadataSegmentManagerTest.java    | 1025 +++++++-------------
 .../appenderator/TestUsedSegmentChecker.java       |    8 +-
 .../CoordinatorRuntimeParamsTestHelpers.java       |   26 +-
 .../coordinator/CostBalancerStrategyTest.java      |    3 +-
 .../coordinator/CuratorDruidCoordinatorTest.java   |   35 +-
 .../DiskNormalizedCostBalancerStrategyTest.java    |    3 +-
 .../server/coordinator/DruidClusterBuilder.java    |   61 ++
 .../druid/server/coordinator/DruidClusterTest.java |   34 +-
 .../DruidCoordinatorBalancerProfiler.java          |  139 +--
 .../coordinator/DruidCoordinatorBalancerTest.java  |   28 +-
 .../DruidCoordinatorBalancerTester.java            |    9 -
 .../DruidCoordinatorRuleRunnerTest.java            |  967 ++++++------------
 .../server/coordinator/DruidCoordinatorTest.java   |   72 +-
 .../coordinator/ReservoirSegmentSamplerTest.java   |    9 +-
 .../cost/CachingCostBalancerStrategyTest.java      |    2 +-
 .../DruidCoordinatorCleanupOvershadowedTest.java   |   41 +-
 .../DruidCoordinatorSegmentCompactorTest.java      |   42 +-
 .../helper/DruidCoordinatorSegmentKillerTest.java  |   39 +-
 .../helper/NewestSegmentFirstPolicyTest.java       |   13 +-
 .../rules/BroadcastDistributionRuleTest.java       |  146 ++-
 .../server/coordinator/rules/LoadRuleTest.java     |  590 ++++-------
 .../server/http/CoordinatorDynamicConfigTest.java  |   34 +-
 .../druid/server/http/DataSourcesResourceTest.java |  760 ++++++---------
 .../druid/sql/calcite/schema/SystemSchema.java     |    2 +-
 .../druid/sql/calcite/schema/DruidSchemaTest.java  |    3 +-
 .../druid/sql/calcite/schema/SystemSchemaTest.java |    8 +-
 .../util/SpecificSegmentsQuerySegmentWalker.java   |    5 +-
 web-console/README.md                              |    2 +-
 85 files changed, 3269 insertions(+), 3969 deletions(-)

diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml
index 4de1e05..bba3782 100644
--- a/.idea/inspectionProfiles/Druid.xml
+++ b/.idea/inspectionProfiles/Druid.xml
@@ -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">
diff --git a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java
index d06125a..dfef128 100644
--- a/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java
+++ b/benchmarks/src/main/java/org/apache/druid/server/coordinator/NewestSegmentFirstPolicyBenchmark.java
@@ -20,6 +20,8 @@
 package org.apache.druid.server.coordinator;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import org.apache.druid.client.DataSourcesSnapshot;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator;
 import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy;
@@ -42,6 +44,7 @@ import org.openjdk.jmh.annotations.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
diff --git a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java
index d5aa17a..569537a 100644
--- a/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java
+++ b/core/src/main/java/org/apache/druid/java/util/common/logger/Logger.java
@@ -123,6 +123,11 @@ public 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());
diff --git a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java
index 8e685c6..ef5d47a 100644
--- a/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java
+++ b/core/src/main/java/org/apache/druid/java/util/emitter/EmittingLogger.java
@@ -28,6 +28,7 @@ import org.apache.druid.java.util.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)
diff --git a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java
index af75440..6bdab5c 100644
--- a/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java
+++ b/core/src/main/java/org/apache/druid/timeline/TimelineLookup.java
@@ -22,6 +22,7 @@ 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);
 }
diff --git a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
index 36c177d..8deed36 100644
--- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
+++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
@@ -23,16 +23,18 @@ import com.google.common.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);
       }
 
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
index ec8f7d8..5003f65 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
@@ -23,7 +23,7 @@ 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();
   }
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java
index 396b4f3..dcf29ae 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java
@@ -22,72 +22,62 @@ 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 +
            '}';
   }
 }
diff --git a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java
index af3cf07..deb4eaf 100644
--- a/core/src/main/java/org/apache/druid/utils/CollectionUtils.java
+++ b/core/src/main/java/org/apache/druid/utils/CollectionUtils.java
@@ -19,13 +19,16 @@
 
 package org.apache.druid.utils;
 
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Maps;
 
 import java.util.AbstractCollection;
 import java.util.Collection;
+import java.util.Comparator;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Spliterator;
+import java.util.TreeSet;
 import java.util.function.Function;
 import java.util.function.Supplier;
 import java.util.stream.Stream;
@@ -72,6 +75,13 @@ public 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.
diff --git a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java b/core/src/main/java/org/apache/druid/utils/package-info.java
similarity index 75%
copy from server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java
copy to core/src/main/java/org/apache/druid/utils/package-info.java
index cca37b9..f41e226 100644
--- a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java
+++ b/core/src/main/java/org/apache/druid/utils/package-info.java
@@ -17,15 +17,7 @@
  * under the License.
  */
 
-package org.apache.druid.metadata;
+@EverythingIsNonnullByDefault
+package org.apache.druid.utils;
 
-/**
- * Exception thrown by MetadataSegmentManager when an segment id is unknown.
- */
-public class UnknownSegmentIdException extends RuntimeException
-{
-  public UnknownSegmentIdException(String message)
-  {
-    super(message);
-  }
-}
+import org.apache.druid.annotations.EverythingIsNonnullByDefault;
diff --git a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
index 681f6a6..3e66bf5 100644
--- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
+++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
@@ -240,22 +240,22 @@ public 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());
   }
 
 }
diff --git a/docs/content/ingestion/delete-data.md b/docs/content/ingestion/delete-data.md
index 7e21e99..181fddb 100644
--- a/docs/content/ingestion/delete-data.md
+++ b/docs/content/ingestion/delete-data.md
@@ -37,7 +37,8 @@ A data deletion tutorial is available at [Tutorial: Deleting data](../tutorials/
 
 ## Kill Task
 
-Kill tasks delete all information about a segment and removes it from deep storage. Killable segments must be disabled (used==0) in the Druid segment table. The available grammar is:
+Kill tasks delete all information about a segment and removes it from deep storage. Segments to kill must be unused
+(used==0) in the Druid segment table. The available grammar is:
 
 ```json
 {
diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md
index 3b49b12..716d5e3 100644
--- a/docs/content/operations/api-reference.md
+++ b/docs/content/operations/api-reference.md
@@ -113,15 +113,17 @@ Returns the serialized JSON of segments to load and drop for each Historical pro
 
 * `/druid/coordinator/v1/metadata/datasources`
 
-Returns a list of the names of enabled datasources in the cluster.
+Returns a list of the names of data sources with at least one used segment in the cluster.
 
-* `/druid/coordinator/v1/metadata/datasources?includeDisabled`
+* `/druid/coordinator/v1/metadata/datasources?includeUnused`
 
-Returns a list of the names of enabled and disabled datasources in the cluster.
+Returns a list of the names of data sources, regardless of whether there are used segments belonging to those data
+sources in the cluster or not.
 
 * `/druid/coordinator/v1/metadata/datasources?full`
 
-Returns a list of all enabled datasources with all metadata about those datasources as stored in the metadata store.
+Returns a list of all data sources with at least one used segment in the cluster. Returns all metadata about those data
+sources as stored in the metadata store.
 
 * `/druid/coordinator/v1/metadata/datasources/{dataSourceName}`
 
@@ -229,11 +231,15 @@ Caution : Avoid using indexing or kill tasks and these API's at the same time fo
 
 * `/druid/coordinator/v1/datasources/{dataSourceName}`
 
-Enables all segments of datasource which are not overshadowed by others.
+Marks as used all segments belonging to a data source. Returns a JSON object of the form
+`{"numChangedSegments": <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}`
 
diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
index 76883ea..1ed80ca 100644
--- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
+++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisor.java
@@ -390,7 +390,7 @@ 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);
   }
 
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
index e766eff..07a507a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/IndexTask.java
@@ -91,6 +91,7 @@ import org.apache.druid.timeline.partition.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()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
index 44c90bc..ac15e67 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
@@ -23,6 +23,7 @@ import com.fasterxml.jackson.annotation.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
 {
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java
index fa30dde..c5f6771 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java
@@ -73,7 +73,9 @@ public 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)
diff --git a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java
index 8417662..865b8bd 100644
--- a/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java
+++ b/server/src/main/java/org/apache/druid/client/DataSourcesSnapshot.java
@@ -19,11 +19,13 @@
 
 package org.apache.druid.client;
 
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Ordering;
+import com.google.common.collect.Maps;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 import org.apache.druid.timeline.VersionedIntervalTimeline;
+import org.apache.druid.utils.CollectionUtils;
 
 import javax.annotation.Nullable;
 import java.util.ArrayList;
@@ -31,40 +33,88 @@ import java.util.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;
+  public static DataSourcesSnapshot fromUsedSegments(
+      Iterable<DataSegment> segments,
+      ImmutableMap<String, String> dataSourceProperties
+  )
+  {
+    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> dataSources
+  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.dataSources = dataSources;
+    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;
diff --git a/server/src/main/java/org/apache/druid/client/DruidDataSource.java b/server/src/main/java/org/apache/druid/client/DruidDataSource.java
index 7678dad..c293523 100644
--- a/server/src/main/java/org/apache/druid/client/DruidDataSource.java
+++ b/server/src/main/java/org/apache/druid/client/DruidDataSource.java
@@ -30,12 +30,13 @@ import java.util.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");
   }
 
diff --git a/server/src/main/java/org/apache/druid/client/DruidServer.java b/server/src/main/java/org/apache/druid/client/DruidServer.java
index ac73d63..d3bcb0d 100644
--- a/server/src/main/java/org/apache/druid/client/DruidServer.java
+++ b/server/src/main/java/org/apache/druid/client/DruidServer.java
@@ -232,7 +232,6 @@ public 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;
         }
     );
diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java
index 5953944..7ce3fd6 100644
--- a/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java
+++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidDataSource.java
@@ -123,6 +123,8 @@ public 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);
   }
 }
diff --git a/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java b/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java
index d01ad96..605b2de 100644
--- a/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java
+++ b/server/src/main/java/org/apache/druid/client/ImmutableDruidServer.java
@@ -42,19 +42,19 @@ public class ImmutableDruidServer
   private final DruidServerMetadata metadata;
   private final long currSize;
   private final ImmutableMap<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) {
diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java
index 695297d..1a09c45 100644
--- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java
+++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQuery.java
@@ -29,6 +29,10 @@ import java.util.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 +
diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java
index 0680169..14cf45b 100644
--- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java
+++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactQueryTuningConfig.java
@@ -43,17 +43,17 @@ public class ClientCompactQueryTuningConfig
   private final Long pushTimeout;
 
   public static ClientCompactQueryTuningConfig from(
-      @Nullable UserCompactTuningConfig userCompactTuningConfig,
+      @Nullable UserCompactTuningConfig userCompactionTaskQueryTuningConfig,
       @Nullable Integer maxRowsPerSegment
   )
   {
     return new ClientCompactQueryTuningConfig(
         maxRowsPerSegment,
-        userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxRowsInMemory(),
-        userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxTotalRows(),
-        userCompactTuningConfig == null ? null : userCompactTuningConfig.getIndexSpec(),
-        userCompactTuningConfig == null ? null : userCompactTuningConfig.getMaxPendingPersists(),
-        userCompactTuningConfig == null ? null : userCompactTuningConfig.getPushTimeout()
+        userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxRowsInMemory(),
+        userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxTotalRows(),
+        userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getIndexSpec(),
+        userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getMaxPendingPersists(),
+        userCompactionTaskQueryTuningConfig == null ? null : userCompactionTaskQueryTuningConfig.getPushTimeout()
     );
   }
 
@@ -150,7 +150,7 @@ public class ClientCompactQueryTuningConfig
   @Override
   public String toString()
   {
-    return "ClientCompactQueryTuningConfig{" +
+    return getClass().getSimpleName() + "{" +
            "maxRowsPerSegment=" + maxRowsPerSegment +
            ", maxRowsInMemory=" + maxRowsInMemory +
            ", maxTotalRows=" + maxTotalRows +
diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java
index 06d88f9..583dee5 100644
--- a/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java
+++ b/server/src/main/java/org/apache/druid/client/indexing/ClientKillQuery.java
@@ -24,6 +24,9 @@ import 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
 {
diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java
index aaa8b5c..306d6e7 100644
--- a/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java
+++ b/server/src/main/java/org/apache/druid/client/indexing/ClientQuery.java
@@ -24,7 +24,12 @@ import com.fasterxml.jackson.annotation.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 = {
diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java
index db1fbef..550dd49 100644
--- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java
+++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java
@@ -24,6 +24,7 @@ import org.apache.druid.client.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);
+
+  int markAsUsedNonOvershadowedSegmentsInInterval(String dataSource, Interval interval);
 
-  boolean enableSegment(String segmentId);
+  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);
+
+  int markAsUnusedSegmentsInInterval(String dataSource, Interval interval);
 
-  boolean removeDataSource(String dataSource);
+  int markSegmentsAsUnused(String dataSource, Set<String> segmentIds);
 
   /**
-   * Removes the given segmentId from metadata store. Returns true if one or more rows were affected.
+   * Returns true if the state of the segment entry is changed in the database as the result of this call (that is, the
+   * segment was marked as unused), false otherwise. If the call results in a database error, an exception is relayed to
+   * the caller.
    */
-  boolean removeSegment(String segmentId);
+  boolean markSegmentAsUnused(String segmentId);
 
-  long disableSegments(String dataSource, Collection<String> segmentIds);
-
-  int disableSegments(String dataSource, Interval interval);
+  /**
+   * If there are used segments belonging to the given data source this method returns them as an {@link
+   * ImmutableDruidDataSource} object. If there are no used segments belonging to the given data source this method
+   * returns null.
+   */
+  @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSource);
 
-  boolean isStarted();
+  /**
+   * Returns a set of {@link ImmutableDruidDataSource} objects containing information about all used segments. {@link
+   * ImmutableDruidDataSource} objects in the returned collection are unique. If there are no used segments, this method
+   * returns an empty collection.
+   */
+  Collection<ImmutableDruidDataSource> getImmutableDataSourcesWithAllUsedSegments();
 
-  @Nullable
-  ImmutableDruidDataSource getDataSource(String dataSourceName);
+  /**
+   * Returns a set of overshadowed segment ids.
+   */
+  Set<SegmentId> getOvershadowedSegments();
 
   /**
-   * Returns a collection of known datasources.
-   *
-   * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has
-   * not yet been polled.)
+   * Returns a snapshot of DruidDataSources and overshadowed segments
    */
-  @Nullable
-  Collection<ImmutableDruidDataSource> getDataSources();
+  DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments();
 
   /**
    * Returns an iterable to go over all segments in all data sources. The order in which segments are iterated is
    * unspecified. Note: the iteration may not be as trivially cheap as, for example, iteration over an ArrayList. Try
    * (to some reasonable extent) to organize the code so that it iterates the returned iterable only once rather than
    * several times.
-   *
-   * Will return null if we do not have a valid snapshot of segments yet (perhaps the underlying metadata store has
-   * not yet been polled.)
    */
-  @Nullable
-  Iterable<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.)
-   */
-  @Nullable
-  Set<SegmentId> getOvershadowedSegments();
-
-  /**
-   * Returns a snapshot of DruidDataSources and overshadowed segments
+   * Performance warning: this method makes a query into the database.
    *
+   * This method might return a different set of data source names than may be observed via {@link
+   * #getImmutableDataSourcesWithAllUsedSegments} method. This method will include a data source name even if there
+   * are no used segments belonging to it, while {@link #getImmutableDataSourcesWithAllUsedSegments} won't return
+   * such a data source.
    */
-  @Nullable
-  DataSourcesSnapshot getDataSourcesSnapshot();
+  Collection<String> retrieveAllDataSourceNames();
 
   /**
-   * Returns top N unused segment intervals in given interval when ordered by segment start time, end time.
+   * Returns top N unused segment intervals with the end time no later than the specified maxEndTime when ordered by
+   * segment start time, end time.
    */
-  List<Interval> getUnusedSegmentIntervals(String dataSource, Interval interval, int limit);
+  List<Interval> getUnusedSegmentIntervals(String dataSource, DateTime maxEndTime, int limit);
 
   @VisibleForTesting
   void poll();
diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java
index 3f58cfa..d6c8811 100644
--- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java
+++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManagerConfig.java
@@ -33,4 +33,9 @@ public class MetadataSegmentManagerConfig
   {
     return pollDuration;
   }
+
+  public void setPollDuration(Period pollDuration)
+  {
+    this.pollDuration = pollDuration;
+  }
 }
diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java
index be95669..1f47e44 100644
--- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java
+++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataRuleManager.java
@@ -200,7 +200,7 @@ public 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
diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java
index bb85142..db1d19e 100644
--- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java
+++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java
@@ -20,16 +20,20 @@
 package org.apache.druid.metadata;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Supplier;
+import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Futures;
+import com.google.errorprone.annotations.concurrent.GuardedBy;
 import com.google.inject.Inject;
 import org.apache.druid.client.DataSourcesSnapshot;
-import org.apache.druid.client.DruidDataSource;
 import org.apache.druid.client.ImmutableDruidDataSource;
 import org.apache.druid.guice.ManageLifecycle;
 import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.JodaUtils;
 import org.apache.druid.java.util.common.MapUtils;
 import org.apache.druid.java.util.common.Pair;
@@ -41,18 +45,18 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.SegmentId;
 import org.apache.druid.timeline.VersionedIntervalTimeline;
-import org.apache.druid.utils.CollectionUtils;
+import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
+import org.joda.time.DateTime;
 import org.joda.time.Duration;
 import org.joda.time.Interval;
 import org.skife.jdbi.v2.BaseResultSetMapper;
 import org.skife.jdbi.v2.Batch;
 import org.skife.jdbi.v2.FoldController;
-import org.skife.jdbi.v2.Folder3;
 import org.skife.jdbi.v2.Handle;
+import org.skife.jdbi.v2.Query;
 import org.skife.jdbi.v2.StatementContext;
 import org.skife.jdbi.v2.TransactionCallback;
 import org.skife.jdbi.v2.TransactionStatus;
-import org.skife.jdbi.v2.tweak.HandleCallback;
 import org.skife.jdbi.v2.tweak.ResultSetMapper;
 
 import javax.annotation.Nullable;
@@ -60,19 +64,19 @@ import java.io.IOException;
 import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
-import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 import java.util.stream.Collectors;
+import java.util.stream.StreamSupport;
 
 /**
  *
@@ -83,11 +87,48 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
   private static final EmittingLogger log = new EmittingLogger(SQLMetadataSegmentManager.class);
 
   /**
-   * Use to synchronize {@link #start()}, {@link #stop()}, {@link #poll()}, and {@link #isStarted()}. These methods
-   * should be synchronized to prevent from being called at the same time if two different threads are calling them.
-   * This might be possible if a druid coordinator gets and drops leadership repeatedly in quick succession.
+   * Marker interface for objects stored in {@link #latestDatabasePoll}. See the comment for that field for details.
    */
-  private final ReentrantReadWriteLock startStopLock = new ReentrantReadWriteLock();
+  private interface DatabasePoll
+  {}
+
+  /** Represents periodic {@link #poll}s happening from {@link #exec}. */
+  private static class PeriodicDatabasePoll implements DatabasePoll
+  {
+    /**
+     * This future allows to wait until {@link #dataSourcesSnapshot} is initialized in the first {@link #poll()}
+     * happening since {@link #startPollingDatabasePeriodically()} is called for the first time, or since the last
+     * visible (in happens-before terms) call to {@link #startPollingDatabasePeriodically()} in case of Coordinator's
+     * leadership changes.
+     */
+    final CompletableFuture<Void> firstPollCompletionFuture = new CompletableFuture<>();
+  }
+
+  /**
+   * Represents on-demand {@link #poll} initiated at periods of time when SqlSegmentsMetadata doesn't poll the database
+   * periodically.
+   */
+  private static class OnDemandDatabasePoll implements DatabasePoll
+  {
+    final long initiationTimeNanos = System.nanoTime();
+    final CompletableFuture<Void> pollCompletionFuture = new CompletableFuture<>();
+
+    long nanosElapsedFromInitiation()
+    {
+      return System.nanoTime() - initiationTimeNanos;
+    }
+  }
+
+  /**
+   * Use to synchronize {@link #startPollingDatabasePeriodically}, {@link #stopPollingDatabasePeriodically}, {@link
+   * #poll}, and {@link #isPollingDatabasePeriodically}. These methods should be synchronized to prevent from being
+   * called at the same time if two different threads are calling them. This might be possible if Coordinator gets and
+   * drops leadership repeatedly in quick succession.
+   *
+   * This lock is also used to synchronize {@link #awaitOrPerformDatabasePoll} for times when SqlSegmentsMetadata
+   * is not polling the database periodically (in other words, when the Coordinator is not the leader).
+   */
+  private final ReentrantReadWriteLock startStopPollLock = new ReentrantReadWriteLock();
 
   /**
    * Used to ensure that {@link #poll()} is never run concurrently. It should already be so (at least in production
@@ -95,41 +136,91 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
    * scheduled in a single-threaded {@link #exec}, so this lock is an additional safety net in case there are bugs in
    * the code, and for tests, where {@link #poll()} is called from the outside code.
    *
-   * Not using {@link #startStopLock}.writeLock() in order to still be able to run {@link #poll()} concurrently with
-   * {@link #isStarted()}.
+   * Not using {@link #startStopPollLock}.writeLock() in order to still be able to run {@link #poll()} concurrently
+   * with {@link #isPollingDatabasePeriodically()}.
    */
   private final Object pollLock = new Object();
 
   private final ObjectMapper jsonMapper;
-  private final Supplier<MetadataSegmentManagerConfig> config;
+  private final Duration periodicPollDelay;
   private final Supplier<MetadataStorageTablesConfig> dbTables;
   private final SQLMetadataConnector connector;
 
-  // Volatile since this reference is reassigned in "poll" and then read from in other threads.
-  // Starts null so we can differentiate "never polled" (null) from "polled, but empty" (empty dataSources map and
-  // empty overshadowedSegments set).
-  // Note that this is not simply a lazy-initialized variable: it starts off as null, and may transition between
-  // null and nonnull multiple times as stop() and start() are called.
-  @Nullable
-  private volatile DataSourcesSnapshot dataSourcesSnapshot = null;
+  /**
+   * This field is made volatile to avoid "ghost secondary reads" that may result in NPE, see
+   * https://github.com/code-review-checklists/java-concurrency#safe-local-dcl (note that dataSourcesSnapshot resembles
+   * a lazily initialized field). Alternative is to always read the field in a snapshot local variable, but it's too
+   * easy to forget to do.
+   *
+   * This field may be updated from {@link #exec}, or from whatever thread calling {@link #doOnDemandPoll} via {@link
+   * #awaitOrPerformDatabasePoll()} via one of the public methods of SqlSegmentsMetadata.
+   */
+  private volatile @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = null;
 
   /**
-   * The number of times this SQLMetadataSegmentManager was started.
+   * The latest {@link DatabasePoll} represent {@link #poll()} calls which update {@link #dataSourcesSnapshot}, either
+   * periodically (see {@link PeriodicDatabasePoll}, {@link #startPollingDatabasePeriodically}, {@link
+   * #stopPollingDatabasePeriodically}) or "on demand" (see {@link OnDemandDatabasePoll}), when one of the methods that
+   * accesses {@link #dataSourcesSnapshot}'s state (such as {@link #getImmutableDataSourceWithUsedSegments}) is
+   * called when the Coordinator is not the leader and therefore SqlSegmentsMetadata isn't polling the database
+   * periodically.
+   *
+   * Note that if there is a happens-before relationship between a call to {@link #startPollingDatabasePeriodically()}
+   * (on Coordinators' leadership change) and one of the methods accessing the {@link #dataSourcesSnapshot}'s state in
+   * this class the latter is guaranteed to await for the initiated periodic poll. This is because when the latter
+   * method calls to {@link #awaitLatestDatabasePoll()} via {@link #awaitOrPerformDatabasePoll}, they will
+   * see the latest {@link PeriodicDatabasePoll} value (stored in this field, latestDatabasePoll, in {@link
+   * #startPollingDatabasePeriodically()}) and to await on its {@link PeriodicDatabasePoll#firstPollCompletionFuture}.
+   *
+   * However, the guarantee explained above doesn't make any actual semantic difference, because on both periodic and
+   * on-demand database polls the same invariant is maintained that the results not older than {@link
+   * #periodicPollDelay} are used. The main difference is in performance: since on-demand polls are irregular and happen
+   * in the context of the thread wanting to access the {@link #dataSourcesSnapshot}, that may cause delays in the
+   * logic. On the other hand, periodic polls are decoupled into {@link #exec} and {@link
+   * #dataSourcesSnapshot}-accessing methods should be generally "wait free" for database polls.
+   *
+   * The notion and the complexity of "on demand" database polls was introduced to simplify the interface of {@link
+   * MetadataSegmentManager} and guarantee that it always returns consistent and relatively up-to-date data from methods
+   * like {@link #getImmutableDataSourceWithUsedSegments}, while avoiding excessive repetitive polls. The last part
+   * is achieved via "hooking on" other polls by awaiting on {@link PeriodicDatabasePoll#firstPollCompletionFuture} or
+   * {@link OnDemandDatabasePoll#pollCompletionFuture}, see {@link #awaitOrPerformDatabasePoll} method
+   * implementation for details.
+   *
+   * Note: the overall implementation of periodic/on-demand polls is not completely optimal: for example, when the
+   * Coordinator just stopped leading, the latest periodic {@link #poll} (which is still "fresh") is not considered
+   * and a new on-demand poll is always initiated. This is done to simplify the implementation, while the efficiency
+   * during Coordinator leadership switches is not a priority.
+   *
+   * This field is {@code volatile} because it's checked and updated in a double-checked locking manner in {@link
+   * #awaitOrPerformDatabasePoll()}.
    */
-  private long startCount = 0;
+  private volatile @Nullable DatabasePoll latestDatabasePoll = null;
+
+  /** Used to cancel periodic poll task in {@link #stopPollingDatabasePeriodically}. */
+  @GuardedBy("startStopPollLock")
+  private @Nullable Future<?> periodicPollTaskFuture = null;
+
+  /** The number of times {@link #startPollingDatabasePeriodically} was called. */
+  @GuardedBy("startStopPollLock")
+  private long startPollingCount = 0;
+
   /**
-   * Equal to the current {@link #startCount} value, if the SQLMetadataSegmentManager is currently started; -1 if
+   * Equal to the current {@link #startPollingCount} value if the SqlSegmentsMetadata is currently started; -1 if
    * currently stopped.
    *
    * This field is used to implement a simple stamp mechanism instead of just a boolean "started" flag to prevent
-   * the theoretical situation of two or more tasks scheduled in {@link #start()} calling {@link #isStarted()} and
-   * {@link #poll()} concurrently, if the sequence of {@link #start()} - {@link #stop()} - {@link #start()} actions
-   * occurs quickly.
+   * the theoretical situation of two or more tasks scheduled in {@link #startPollingDatabasePeriodically()} calling
+   * {@link #isPollingDatabasePeriodically()} and {@link #poll()} concurrently, if the sequence of {@link
+   * #startPollingDatabasePeriodically()} - {@link #stopPollingDatabasePeriodically()} - {@link
+   * #startPollingDatabasePeriodically()} actions occurs quickly.
    *
-   * {@link SQLMetadataRuleManager} also have a similar issue.
+   * {@link SQLMetadataRuleManager} also has a similar issue.
    */
-  private long currentStartOrder = -1;
-  private ScheduledExecutorService exec = null;
+  @GuardedBy("startStopPollLock")
+  private long currentStartPollingOrder = -1;
+
+  @GuardedBy("startStopPollLock")
+  private @Nullable ScheduledExecutorService exec = null;
 
   @Inject
   public SQLMetadataSegmentManager(
@@ -140,33 +231,73 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
   )
   {
     this.jsonMapper = jsonMapper;
-    this.config = config;
+    this.periodicPollDelay = config.get().getPollDuration().toStandardDuration();
     this.dbTables = dbTables;
     this.connector = connector;
   }
 
-  @Override
+  /**
+   * Don't confuse this method with {@link #startPollingDatabasePeriodically}. This is a lifecycle starting method to
+   * be executed just once for an instance of SqlSegmentsMetadata.
+   */
   @LifecycleStart
   public void start()
   {
-    ReentrantReadWriteLock.WriteLock lock = startStopLock.writeLock();
+    ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock();
+    lock.lock();
+    try {
+      if (exec != null) {
+        return; // Already started
+      }
+      exec = Execs.scheduledSingleThreaded(getClass().getName() + "-Exec--%d");
+    }
+    finally {
+      lock.unlock();
+    }
+  }
+
+  /**
+   * Don't confuse this method with {@link #stopPollingDatabasePeriodically}. This is a lifecycle stopping method to
+   * be executed just once for an instance of SqlSegmentsMetadata.
+   */
+  @LifecycleStop
+  public void stop()
+  {
+    ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock();
+    lock.lock();
+    try {
+      exec.shutdownNow();
+      exec = null;
+    }
+    finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void startPollingDatabasePeriodically()
+  {
+    ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock();
     lock.lock();
     try {
-      if (isStarted()) {
+      if (exec == null) {
+        throw new IllegalStateException(getClass().getName() + " is not started");
+      }
+      if (isPollingDatabasePeriodically()) {
         return;
       }
 
-      startCount++;
-      currentStartOrder = startCount;
-      final long localStartOrder = currentStartOrder;
+      PeriodicDatabasePoll periodicPollUpdate = new PeriodicDatabasePoll();
+      latestDatabasePoll = periodicPollUpdate;
 
-      exec = Execs.scheduledSingleThreaded("DatabaseSegmentManager-Exec--%d");
+      startPollingCount++;
+      currentStartPollingOrder = startPollingCount;
+      final long localStartOrder = currentStartPollingOrder;
 
-      final Duration delay = config.get().getPollDuration().toStandardDuration();
-      exec.scheduleWithFixedDelay(
-          createPollTaskForStartOrder(localStartOrder),
+      periodicPollTaskFuture = exec.scheduleWithFixedDelay(
+          createPollTaskForStartOrder(localStartOrder, periodicPollUpdate),
           0,
-          delay.getMillis(),
+          periodicPollDelay.getMillis(),
           TimeUnit.MILLISECONDS
       );
     }
@@ -175,24 +306,33 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
     }
   }
 
-  private Runnable createPollTaskForStartOrder(long startOrder)
+  private Runnable createPollTaskForStartOrder(long startOrder, PeriodicDatabasePoll periodicPollUpdate)
   {
     return () -> {
-      // poll() is synchronized together with start(), stop() and isStarted() to ensure that when stop() exits, poll()
-      // won't actually run anymore after that (it could only enter the syncrhonized section and exit immediately
-      // because the localStartedOrder doesn't match the new currentStartOrder). It's needed to avoid flakiness in
-      // SQLMetadataSegmentManagerTest. See https://github.com/apache/incubator-druid/issues/6028
-      ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock();
+      // poll() is synchronized together with startPollingDatabasePeriodically(), stopPollingDatabasePeriodically() and
+      // isPollingDatabasePeriodically() to ensure that when stopPollingDatabasePeriodically() exits, poll() won't
+      // actually run anymore after that (it could only enter the synchronized section and exit immediately because the
+      // localStartedOrder doesn't match the new currentStartPollingOrder). It's needed to avoid flakiness in
+      // SqlSegmentsMetadataTest. See https://github.com/apache/incubator-druid/issues/6028
+      ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock();
       lock.lock();
       try {
-        if (startOrder == currentStartOrder) {
+        if (startOrder == currentStartPollingOrder) {
           poll();
+          periodicPollUpdate.firstPollCompletionFuture.complete(null);
         } else {
-          log.debug("startOrder = currentStartOrder = %d, skipping poll()", startOrder);
+          log.debug("startOrder = currentStartPollingOrder = %d, skipping poll()", startOrder);
         }
       }
-      catch (Exception e) {
-        log.makeAlert(e, "uncaught exception in segment manager polling thread").emit();
+      catch (Throwable t) {
+        log.makeAlert(t, "Uncaught exception in %s's polling thread", SQLMetadataSegmentManager.class).emit();
+        // Swallow the exception, so that scheduled polling goes on. Leave firstPollFutureSinceLastStart uncompleted
+        // for now, so that it may be completed during the next poll.
+        if (!(t instanceof Exception)) {
+          // Don't try to swallow a Throwable which is not an Exception (that is, a Error).
+          periodicPollUpdate.firstPollCompletionFuture.completeExceptionally(t);
+          throw t;
+        }
       }
       finally {
         lock.unlock();
@@ -201,320 +341,435 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
   }
 
   @Override
-  @LifecycleStop
-  public void stop()
+  public boolean isPollingDatabasePeriodically()
+  {
+    // isPollingDatabasePeriodically() is synchronized together with startPollingDatabasePeriodically(),
+    // stopPollingDatabasePeriodically() and poll() to ensure that the latest currentStartPollingOrder is always
+    // visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator.
+    ReentrantReadWriteLock.ReadLock lock = startStopPollLock.readLock();
+    lock.lock();
+    try {
+      return currentStartPollingOrder >= 0;
+    }
+    finally {
+      lock.unlock();
+    }
+  }
+
+  @Override
+  public void stopPollingDatabasePeriodically()
   {
-    ReentrantReadWriteLock.WriteLock lock = startStopLock.writeLock();
+    ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock();
     lock.lock();
     try {
-      if (!isStarted()) {
+      if (!isPollingDatabasePeriodically()) {
         return;
       }
-      dataSourcesSnapshot = null;
-      currentStartOrder = -1;
-      exec.shutdownNow();
-      exec = null;
+
+      periodicPollTaskFuture.cancel(false);
+      latestDatabasePoll = null;
+
+      // NOT nulling dataSourcesSnapshot, allowing to query the latest polled data even when this SegmentsMetadata
+      // object is stopped.
+
+      currentStartPollingOrder = -1;
     }
     finally {
       lock.unlock();
     }
   }
 
-  private Pair<DataSegment, Boolean> usedPayloadMapper(
-      final int index,
-      final ResultSet resultSet,
-      final StatementContext context
-  ) throws SQLException
+  private void awaitOrPerformDatabasePoll()
   {
+    // Double-checked locking with awaitLatestDatabasePoll() call playing the role of the "check".
+    if (awaitLatestDatabasePoll()) {
+      return;
+    }
+    ReentrantReadWriteLock.WriteLock lock = startStopPollLock.writeLock();
+    lock.lock();
     try {
-      return new Pair<>(
-          jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class),
-          resultSet.getBoolean("used")
-      );
+      if (awaitLatestDatabasePoll()) {
+        return;
+      }
+      OnDemandDatabasePoll newOnDemandUpdate = new OnDemandDatabasePoll();
+      this.latestDatabasePoll = newOnDemandUpdate;
+      doOnDemandPoll(newOnDemandUpdate);
     }
-    catch (IOException e) {
-      throw new RuntimeException(e);
+    finally {
+      lock.unlock();
     }
   }
 
   /**
-   * Gets a list of all datasegments that overlap the provided interval along with thier used status.
+   * If the latest {@link DatabasePoll} is a {@link PeriodicDatabasePoll}, or an {@link OnDemandDatabasePoll} that is
+   * made not longer than {@link #periodicPollDelay} from now, awaits for it and returns true; returns false otherwise,
+   * meaning that a new on-demand database poll should be initiated.
    */
-  private List<Pair<DataSegment, Boolean>> getDataSegmentsOverlappingInterval(
-      final String dataSource,
-      final Interval interval
-  )
-  {
-    return connector.inReadOnlyTransaction(
-        (handle, status) -> handle.createQuery(
-            StringUtils.format(
-                "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start",
-                getSegmentsTable(),
-                connector.getQuoteString()
-            )
-        )
-        .setFetchSize(connector.getStreamingFetchSize())
-        .bind("dataSource", dataSource)
-        .bind("start", interval.getStart().toString())
-        .bind("end", interval.getEnd().toString())
-        .map(this::usedPayloadMapper)
-        .list()
-    );
-  }
-
-  private List<Pair<DataSegment, Boolean>> getDataSegments(
-      final String dataSource,
-      final Collection<String> segmentIds,
-      final Handle handle
-  )
+  private boolean awaitLatestDatabasePoll()
   {
-    return segmentIds.stream().map(
-        segmentId -> Optional.ofNullable(
-            handle.createQuery(
-                StringUtils.format(
-                    "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND id = :id",
-                    getSegmentsTable()
-                )
-            )
-            .bind("dataSource", dataSource)
-            .bind("id", segmentId)
-            .map(this::usedPayloadMapper)
-            .first()
-        )
-        .orElseThrow(() -> new UnknownSegmentIdException(StringUtils.format("Cannot find segment id [%s]", segmentId)))
-    )
-    .collect(Collectors.toList());
+    DatabasePoll latestDatabasePoll = this.latestDatabasePoll;
+    if (latestDatabasePoll instanceof PeriodicDatabasePoll) {
+      Futures.getUnchecked(((PeriodicDatabasePoll) latestDatabasePoll).firstPollCompletionFuture);
+      return true;
+    }
+    if (latestDatabasePoll instanceof OnDemandDatabasePoll) {
+      long periodicPollDelayNanos = TimeUnit.MILLISECONDS.toNanos(periodicPollDelay.getMillis());
+      OnDemandDatabasePoll latestOnDemandPoll = (OnDemandDatabasePoll) latestDatabasePoll;
+      boolean latestUpdateIsFresh = latestOnDemandPoll.nanosElapsedFromInitiation() < periodicPollDelayNanos;
+      if (latestUpdateIsFresh) {
+        Futures.getUnchecked(latestOnDemandPoll.pollCompletionFuture);
+        return true;
+      }
+      // Latest on-demand update is not fresh. Fall through to return false from this method.
+    } else {
+      assert latestDatabasePoll == null;
+      // No periodic updates and no on-demand database poll have been done yet, nothing to await for.
+    }
+    return false;
   }
 
-  /**
-   * Builds a VersionedIntervalTimeline containing used segments that overlap the intervals passed.
-   */
-  private VersionedIntervalTimeline<String, DataSegment> buildVersionedIntervalTimeline(
-      final String dataSource,
-      final Collection<Interval> intervals,
-      final Handle handle
-  )
+  private void doOnDemandPoll(OnDemandDatabasePoll onDemandPoll)
   {
-    return VersionedIntervalTimeline.forSegments(intervals
-        .stream()
-        .flatMap(interval -> handle.createQuery(
-                StringUtils.format(
-                    "SELECT payload FROM %1$s WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start AND used = true",
-                    getSegmentsTable(),
-                    connector.getQuoteString()
-                )
-            )
-            .setFetchSize(connector.getStreamingFetchSize())
-            .bind("dataSource", dataSource)
-            .bind("start", interval.getStart().toString())
-            .bind("end", interval.getEnd().toString())
-            .map((i, resultSet, context) -> {
-              try {
-                return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class);
-              }
-              catch (IOException e) {
-                throw new RuntimeException(e);
-              }
-            })
-            .list()
-            .stream()
-        )
-        .iterator()
-    );
+    try {
+      poll();
+      onDemandPoll.pollCompletionFuture.complete(null);
+    }
+    catch (Throwable t) {
+      onDemandPoll.pollCompletionFuture.completeExceptionally(t);
+      throw t;
+    }
   }
 
   @Override
-  public boolean enableDataSource(final String dataSource)
+  public boolean markSegmentAsUsed(final String segmentId)
   {
     try {
-      return enableSegments(dataSource, Intervals.ETERNITY) != 0;
+      int numUpdatedDatabaseEntries = connector.getDBI().withHandle(
+          (Handle handle) -> handle
+              .createStatement(StringUtils.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable()))
+              .bind("id", segmentId)
+              .execute()
+      );
+      // Unlike bulk markAsUsed methods: markAsUsedAllNonOvershadowedSegmentsInDataSource(),
+      // markAsUsedNonOvershadowedSegmentsInInterval(), and markAsUsedNonOvershadowedSegments() we don't put the marked
+      // segment into the respective data source, because we don't have it fetched from the database. It's probably not
+      // worth complicating the implementation and making two database queries just to add the segment because it will
+      // be anyway fetched during the next poll(). Segment putting that is done in the bulk markAsUsed methods is a nice
+      // to have thing, but doesn't formally affects the external guarantees of SegmentsMetadata class.
+      return numUpdatedDatabaseEntries > 0;
     }
-    catch (Exception e) {
-      log.error(e, "Exception enabling datasource %s", dataSource);
-      return false;
+    catch (RuntimeException e) {
+      log.error(e, "Exception marking segment %s as used", segmentId);
+      throw e;
     }
   }
 
   @Override
-  public int enableSegments(final String dataSource, final Interval interval)
+  public int markAsUsedAllNonOvershadowedSegmentsInDataSource(final String dataSource)
   {
-    List<Pair<DataSegment, Boolean>> segments = getDataSegmentsOverlappingInterval(dataSource, interval);
-    List<DataSegment> segmentsToEnable = segments.stream()
-        .filter(segment -> !segment.rhs && interval.contains(segment.lhs.getInterval()))
-        .map(segment -> segment.lhs)
-        .collect(Collectors.toList());
-
-    VersionedIntervalTimeline<String, DataSegment> versionedIntervalTimeline = VersionedIntervalTimeline.forSegments(
-        segments.stream().filter(segment -> segment.rhs).map(segment -> segment.lhs).iterator()
-    );
-    VersionedIntervalTimeline.addSegments(versionedIntervalTimeline, segmentsToEnable.iterator());
-
-    return enableSegments(
-        segmentsToEnable,
-        versionedIntervalTimeline
-    );
+    return doMarkAsUsedNonOvershadowedSegments(dataSource, null);
   }
 
   @Override
-  public int enableSegments(final String dataSource, final Collection<String> segmentIds)
+  public int markAsUsedNonOvershadowedSegmentsInInterval(final String dataSource, final Interval interval)
   {
-    Pair<List<DataSegment>, VersionedIntervalTimeline<String, DataSegment>> data = connector.inReadOnlyTransaction(
-        (handle, status) -> {
-          List<DataSegment> segments = getDataSegments(dataSource, segmentIds, handle)
-              .stream()
-              .filter(pair -> !pair.rhs)
-              .map(pair -> pair.lhs)
-              .collect(Collectors.toList());
-
-          VersionedIntervalTimeline<String, DataSegment> versionedIntervalTimeline = buildVersionedIntervalTimeline(
-              dataSource,
-              JodaUtils.condenseIntervals(segments.stream().map(segment -> segment.getInterval()).collect(Collectors.toList())),
-              handle
-          );
-          VersionedIntervalTimeline.addSegments(versionedIntervalTimeline, segments.iterator());
+    Preconditions.checkNotNull(interval);
+    return doMarkAsUsedNonOvershadowedSegments(dataSource, interval);
+  }
 
-          return new Pair<>(
-              segments,
-              versionedIntervalTimeline
-          );
+  /**
+   * Implementation for both {@link #markAsUsedAllNonOvershadowedSegmentsInDataSource} (if the given interval is null)
+   * and {@link #markAsUsedNonOvershadowedSegmentsInInterval}.
+   */
+  private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable Interval interval)
+  {
+    List<DataSegment> usedSegmentsOverlappingInterval = new ArrayList<>();
+    List<DataSegment> unusedSegmentsInInterval = new ArrayList<>();
+    connector.inReadOnlyTransaction(
+        (handle, status) -> {
+          String queryString =
+              StringUtils.format("SELECT used, payload FROM %1$s WHERE dataSource = :dataSource", getSegmentsTable());
+          if (interval != null) {
+            queryString += StringUtils.format(" AND start < :end AND %1$send%1$s > :start", connector.getQuoteString());
+          }
+          Query<?> query = handle
+              .createQuery(queryString)
+              .setFetchSize(connector.getStreamingFetchSize())
+              .bind("dataSource", dataSourceName);
+          if (interval != null) {
+            query = query
+                .bind("start", interval.getStart().toString())
+                .bind("end", interval.getEnd().toString());
+          }
+          query = query
+              .map((int index, ResultSet resultSet, StatementContext context) -> {
+                try {
+                  DataSegment segment = jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class);
+                  if (resultSet.getBoolean("used")) {
+                    usedSegmentsOverlappingInterval.add(segment);
+                  } else {
+                    if (interval == null || interval.contains(segment.getInterval())) {
+                      unusedSegmentsInInterval.add(segment);
+                    }
+                  }
+                  return null;
+                }
+                catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+              });
+          // Consume the query results to ensure usedSegmentsOverlappingInterval and unusedSegmentsInInterval are
+          // populated.
+          consume(query.iterator());
+          return null;
         }
     );
 
-    return enableSegments(
-        data.lhs,
-        data.rhs
+    VersionedIntervalTimeline<String, DataSegment> versionedIntervalTimeline = VersionedIntervalTimeline.forSegments(
+        Iterators.concat(usedSegmentsOverlappingInterval.iterator(), unusedSegmentsInInterval.iterator())
     );
+
+    return markNonOvershadowedSegmentsAsUsed(unusedSegmentsInInterval, versionedIntervalTimeline);
   }
 
-  private int enableSegments(
-      final Collection<DataSegment> segments,
-      final VersionedIntervalTimeline<String, DataSegment> versionedIntervalTimeline
+  private static void consume(Iterator<?> iterator)
+  {
+    while (iterator.hasNext()) {
+      iterator.next();
+    }
+  }
+
+  private int markNonOvershadowedSegmentsAsUsed(
+      List<DataSegment> unusedSegments,
+      VersionedIntervalTimeline<String, DataSegment> timeline
   )
   {
-    if (segments.isEmpty()) {
-      log.warn("No segments found to update!");
-      return 0;
+    List<String> segmentIdsToMarkAsUsed = new ArrayList<>();
+    for (DataSegment segment : unusedSegments) {
+      if (timeline.isOvershadowed(segment.getInterval(), segment.getVersion())) {
+        continue;
+      }
+      segmentIdsToMarkAsUsed.add(segment.getId().toString());
     }
 
-    return connector.getDBI().withHandle(handle -> {
-      Batch batch = handle.createBatch();
-      segments
-          .stream()
-          .map(segment -> segment.getId())
-          .filter(segmentId -> !versionedIntervalTimeline.isOvershadowed(
-              segmentId.getInterval(),
-              segmentId.getVersion()
-          ))
-          .forEach(segmentId -> batch.add(
-              StringUtils.format(
-                  "UPDATE %s SET used=true WHERE id = '%s'",
-                  getSegmentsTable(),
-                  segmentId
-              )
-          ));
-      return batch.execute().length;
-    });
+    return markSegmentsAsUsed(segmentIdsToMarkAsUsed);
   }
 
   @Override
-  public boolean enableSegment(final String segmentId)
+  public int markAsUsedNonOvershadowedSegments(final String dataSource, final Set<String> segmentIds)
+      throws UnknownSegmentIdException
   {
     try {
-      connector.getDBI().withHandle(
-          new HandleCallback<Void>()
-          {
-            @Override
-            public Void withHandle(Handle handle)
-            {
-              handle.createStatement(StringUtils.format("UPDATE %s SET used=true WHERE id = :id", getSegmentsTable()))
-                    .bind("id", segmentId)
-                    .execute();
-              return null;
-            }
-          }
-      );
+      Pair<List<DataSegment>, VersionedIntervalTimeline<String, DataSegment>> unusedSegmentsAndTimeline = connector
+          .inReadOnlyTransaction(
+              (handle, status) -> {
+                List<DataSegment> unusedSegments = retrieveUnusedSegments(dataSource, segmentIds, handle);
+                List<Interval> unusedSegmentsIntervals = JodaUtils.condenseIntervals(
+                    unusedSegments.stream().map(DataSegment::getInterval).collect(Collectors.toList())
+                );
+                Iterator<DataSegment> usedSegmentsOverlappingUnusedSegmentsIntervals =
+                    retrieveUsedSegmentsOverlappingIntervals(dataSource, unusedSegmentsIntervals, handle);
+                VersionedIntervalTimeline<String, DataSegment> timeline = VersionedIntervalTimeline.forSegments(
+                    Iterators.concat(usedSegmentsOverlappingUnusedSegmentsIntervals, unusedSegments.iterator())
+                );
+                return new Pair<>(unusedSegments, timeline);
+              }
+          );
+
+      List<DataSegment> unusedSegments = unusedSegmentsAndTimeline.lhs;
+      VersionedIntervalTimeline<String, DataSegment> timeline = unusedSegmentsAndTimeline.rhs;
+      return markNonOvershadowedSegmentsAsUsed(unusedSegments, timeline);
     }
     catch (Exception e) {
-      log.error(e, "Exception enabling segment %s", segmentId);
-      return false;
+      Throwable rootCause = Throwables.getRootCause(e);
+      if (rootCause instanceof UnknownSegmentIdException) {
+        throw (UnknownSegmentIdException) rootCause;
+      } else {
+        throw e;
+      }
+    }
+  }
+
+  private List<DataSegment> retrieveUnusedSegments(
+      final String dataSource,
+      final Set<String> segmentIds,
+      final Handle handle
+  ) throws UnknownSegmentIdException
+  {
+    List<String> unknownSegmentIds = new ArrayList<>();
+    List<DataSegment> segments = segmentIds
+        .stream()
+        .map(
+            segmentId -> {
+              Iterator<DataSegment> segmentResultIterator = handle
+                  .createQuery(
+                      StringUtils.format(
+                          "SELECT used, payload FROM %1$s WHERE dataSource = :dataSource AND id = :id",
+                          getSegmentsTable()
+                      )
+                  )
+                  .bind("dataSource", dataSource)
+                  .bind("id", segmentId)
+                  .map((int index, ResultSet resultSet, StatementContext context) -> {
+                    try {
+                      if (!resultSet.getBoolean("used")) {
+                        return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class);
+                      } else {
+                        // We emit nulls for used segments. They are filtered out below in this method.
+                        return null;
+                      }
+                    }
+                    catch (IOException e) {
+                      throw new RuntimeException(e);
+                    }
+                  })
+                  .iterator();
+              if (!segmentResultIterator.hasNext()) {
+                unknownSegmentIds.add(segmentId);
+                return null;
+              } else {
+                @Nullable DataSegment segment = segmentResultIterator.next();
+                if (segmentResultIterator.hasNext()) {
+                  log.error(
+                      "There is more than one row corresponding to segment id [%s] in data source [%s] in the database",
+                      segmentId,
+                      dataSource
+                  );
+                }
+                return segment;
+              }
+            }
+        )
+        .filter(Objects::nonNull) // Filter nulls corresponding to used segments.
+        .collect(Collectors.toList());
+    if (!unknownSegmentIds.isEmpty()) {
+      throw new UnknownSegmentIdException(unknownSegmentIds);
+    }
+    return segments;
+  }
+
+  private Iterator<DataSegment> retrieveUsedSegmentsOverlappingIntervals(
+      final String dataSource,
+      final Collection<Interval> intervals,
+      final Handle handle
+  )
+  {
+    return intervals
+        .stream()
+        .flatMap(interval -> {
+          Iterable<DataSegment> segmentResultIterable = () -> handle
+              .createQuery(
+                  StringUtils.format(
+                      "SELECT payload FROM %1$s "
+                      + "WHERE dataSource = :dataSource AND start < :end AND %2$send%2$s > :start AND used = true",
+                      getSegmentsTable(),
+                      connector.getQuoteString()
+                  )
+              )
+              .setFetchSize(connector.getStreamingFetchSize())
+              .bind("dataSource", dataSource)
+              .bind("start", interval.getStart().toString())
+              .bind("end", interval.getEnd().toString())
+              .map((int index, ResultSet resultSet, StatementContext context) -> {
+                try {
+                  return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class);
+                }
+                catch (IOException e) {
+                  throw new RuntimeException(e);
+                }
+              })
+              .iterator();
+          return StreamSupport.stream(segmentResultIterable.spliterator(), false);
+        })
+        .iterator();
+  }
+
+  private int markSegmentsAsUsed(final List<String> segmentIds)
+  {
+    if (segmentIds.isEmpty()) {
+      log.info("No segments found to update!");
+      return 0;
     }
 
-    return true;
+    return connector.getDBI().withHandle(handle -> {
+      Batch batch = handle.createBatch();
+      segmentIds.forEach(segmentId -> batch.add(
+          StringUtils.format("UPDATE %s SET used=true WHERE id = '%s'", getSegmentsTable(), segmentId)
+      ));
+      int[] segmentChanges = batch.execute();
+      return computeNumChangedSegments(segmentIds, segmentChanges);
+    });
   }
 
   @Override
-  public boolean removeDataSource(final String dataSource)
+  public int markAsUnusedAllSegmentsInDataSource(final String dataSource)
   {
     try {
-      final int removed = connector.getDBI().withHandle(
-          handle -> handle.createStatement(
-              StringUtils.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable())
-          ).bind("dataSource", dataSource).execute()
+      final int numUpdatedDatabaseEntries = connector.getDBI().withHandle(
+          (Handle handle) -> handle
+              .createStatement(
+                  StringUtils.format("UPDATE %s SET used=false WHERE dataSource = :dataSource", getSegmentsTable())
+              )
+              .bind("dataSource", dataSource)
+              .execute()
       );
 
-      if (removed == 0) {
-        return false;
-      }
+      return numUpdatedDatabaseEntries;
     }
-    catch (Exception e) {
-      log.error(e, "Error removing datasource %s", dataSource);
-      return false;
+    catch (RuntimeException e) {
+      log.error(e, "Exception marking all segments as unused in data source [%s]", dataSource);
+      throw e;
     }
-
-    return true;
   }
 
   /**
-   * This method does not update {@code dataSourcesSnapshot}, see the comments in {@code doPoll()} about
-   * snapshot update. The segment removal will be reflected after next poll cyccle runs.
+   * This method does not update {@link #dataSourcesSnapshot}, see the comments in {@link #doPoll()} about
+   * snapshot update. The update of the segment's state will be reflected after the next {@link DatabasePoll}.
    */
   @Override
-  public boolean removeSegment(String segmentId)
+  public boolean markSegmentAsUnused(final String segmentId)
   {
     try {
-      return removeSegmentFromTable(segmentId);
+      return markSegmentAsUnusedInDatabase(segmentId);
     }
-    catch (Exception e) {
-      log.error(e, e.toString());
-      return false;
+    catch (RuntimeException e) {
+      log.error(e, "Exception marking segment [%s] as unused", segmentId);
+      throw e;
     }
   }
 
   @Override
-  public long disableSegments(String dataSource, Collection<String> segmentIds)
+  public int markSegmentsAsUnused(String dataSourceName, Set<String> segmentIds)
   {
     if (segmentIds.isEmpty()) {
       return 0;
     }
-    final long[] result = new long[1];
+    final List<String> segmentIdList = new ArrayList<>(segmentIds);
     try {
-      connector.getDBI().withHandle(handle -> {
+      return connector.getDBI().withHandle(handle -> {
         Batch batch = handle.createBatch();
-        segmentIds
-            .forEach(segmentId -> batch.add(
-                StringUtils.format(
-                    "UPDATE %s SET used=false WHERE datasource = '%s' AND id = '%s' ",
-                    getSegmentsTable(),
-                    dataSource,
-                    segmentId
-                )
-            ));
-        final int[] resultArr = batch.execute();
-        result[0] = Arrays.stream(resultArr).filter(x -> x > 0).count();
-        return result[0];
+        segmentIdList.forEach(segmentId -> batch.add(
+            StringUtils.format(
+                "UPDATE %s SET used=false WHERE datasource = '%s' AND id = '%s'",
+                getSegmentsTable(),
+                dataSourceName,
+                segmentId
+            )
+        ));
+        final int[] segmentChanges = batch.execute();
+        return computeNumChangedSegments(segmentIdList, segmentChanges);
       });
     }
     catch (Exception e) {
       throw new RuntimeException(e);
     }
-    return result[0];
   }
 
   @Override
-  public int disableSegments(String dataSource, Interval interval)
+  public int markAsUnusedSegmentsInInterval(String dataSourceName, Interval interval)
   {
     try {
-      return connector.getDBI().withHandle(
+      Integer numUpdatedDatabaseEntries = connector.getDBI().withHandle(
           handle -> handle
               .createStatement(
                   StringUtils
@@ -524,116 +779,121 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
                           getSegmentsTable(),
                           connector.getQuoteString()
                       ))
-              .bind("datasource", dataSource)
+              .bind("datasource", dataSourceName)
               .bind("start", interval.getStart().toString())
               .bind("end", interval.getEnd().toString())
               .execute()
       );
+      return numUpdatedDatabaseEntries;
     }
     catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
-  private boolean removeSegmentFromTable(String segmentId)
+  private boolean markSegmentAsUnusedInDatabase(String segmentId)
   {
-    final int removed = connector.getDBI().withHandle(
+    final int numUpdatedRows = connector.getDBI().withHandle(
         handle -> handle
             .createStatement(StringUtils.format("UPDATE %s SET used=false WHERE id = :segmentID", getSegmentsTable()))
             .bind("segmentID", segmentId)
             .execute()
     );
-    return removed > 0;
+    if (numUpdatedRows < 0) {
+      log.assertionError(
+          "Negative number of rows updated for segment id [%s]: %d",
+          segmentId,
+          numUpdatedRows
+      );
+    } else if (numUpdatedRows > 1) {
+      log.error(
+          "More than one row updated for segment id [%s]: %d, "
+          + "there may be more than one row for the segment id in the database",
+          segmentId,
+          numUpdatedRows
+      );
+    }
+    return numUpdatedRows > 0;
   }
 
-  @Override
-  public boolean isStarted()
+  private static int computeNumChangedSegments(List<String> segmentIds, int[] segmentChanges)
   {
-    // isStarted() is synchronized together with start(), stop() and poll() to ensure that the latest currentStartOrder
-    // is always visible. readLock should be used to avoid unexpected performance degradation of DruidCoordinator.
-    ReentrantReadWriteLock.ReadLock lock = startStopLock.readLock();
-    lock.lock();
-    try {
-      return currentStartOrder >= 0;
-    }
-    finally {
-      lock.unlock();
+    int numChangedSegments = 0;
+    for (int i = 0; i < segmentChanges.length; i++) {
+      int numUpdatedRows = segmentChanges[i];
+      if (numUpdatedRows < 0) {
+        log.assertionError(
+            "Negative number of rows updated for segment id [%s]: %d",
+            segmentIds.get(i),
+            numUpdatedRows
+        );
+      } else if (numUpdatedRows > 1) {
+        log.error(
+            "More than one row updated for segment id [%s]: %d, "
+            + "there may be more than one row for the segment id in the database",
+            segmentIds.get(i),
+            numUpdatedRows
+        );
+      }
+      if (numUpdatedRows > 0) {
+        numChangedSegments += 1;
+      }
     }
+    return numChangedSegments;
   }
 
   @Override
-  @Nullable
-  public ImmutableDruidDataSource getDataSource(String dataSourceName)
+  public @Nullable ImmutableDruidDataSource getImmutableDataSourceWithUsedSegments(String dataSourceName)
   {
-    final ImmutableDruidDataSource dataSource = Optional.ofNullable(dataSourcesSnapshot)
-                                                        .map(m -> m.getDataSourcesMap().get(dataSourceName))
-                                                        .orElse(null);
-    return dataSource == null ? null : dataSource;
+    return getSnapshotOfDataSourcesWithAllUsedSegments().getDataSource(dataSourceName);
   }
 
   @Override
-  @Nullable
-  public Collection<ImmutableDruidDataSource> getDataSources()
+  public Collection<ImmutableDruidDataSource> getImmutableDataSourcesWithAllUsedSegments()
   {
-    return Optional.ofNullable(dataSourcesSnapshot).map(m -> m.getDataSources()).orElse(null);
+    return getSnapshotOfDataSourcesWithAllUsedSegments().getDataSourcesWithAllUsedSegments();
   }
 
   @Override
-  @Nullable
-  public Iterable<DataSegment> iterateAllSegments()
+  public Set<SegmentId> getOvershadowedSegments()
   {
-    final Collection<ImmutableDruidDataSource> dataSources = Optional.ofNullable(dataSourcesSnapshot)
-                                                                     .map(m -> m.getDataSources())
-                                                                     .orElse(null);
-    if (dataSources == null) {
-      return null;
-    }
-
-    return () -> dataSources.stream()
-                            .flatMap(dataSource -> dataSource.getSegments().stream())
-                            .iterator();
+    return getSnapshotOfDataSourcesWithAllUsedSegments().getOvershadowedSegments();
   }
 
   @Override
-  @Nullable
-  public Set<SegmentId> getOvershadowedSegments()
+  public DataSourcesSnapshot getSnapshotOfDataSourcesWithAllUsedSegments()
   {
-    return Optional.ofNullable(dataSourcesSnapshot).map(m -> m.getOvershadowedSegments()).orElse(null);
+    awaitOrPerformDatabasePoll();
+    return dataSourcesSnapshot;
   }
 
-  @Nullable
   @Override
-  public DataSourcesSnapshot getDataSourcesSnapshot()
+  public Iterable<DataSegment> iterateAllUsedSegments()
   {
-    return dataSourcesSnapshot;
+    awaitOrPerformDatabasePoll();
+    return () -> dataSourcesSnapshot
+        .getDataSourcesWithAllUsedSegments()
+        .stream()
+        .flatMap(dataSource -> dataSource.getSegments().stream())
+        .iterator();
   }
 
   @Override
-  public Collection<String> getAllDataSourceNames()
+  public Collection<String> retrieveAllDataSourceNames()
   {
     return connector.getDBI().withHandle(
-        handle -> handle.createQuery(
-            StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable())
-        )
-                        .fold(
-                            new ArrayList<>(),
-                            new Folder3<List<String>, Map<String, Object>>()
-                            {
-                              @Override
-                              public List<String> fold(
-                                  List<String> druidDataSources,
-                                  Map<String, Object> stringObjectMap,
-                                  FoldController foldController,
-                                  StatementContext statementContext
-                              )
-                              {
-                                druidDataSources.add(
-                                    MapUtils.getString(stringObjectMap, "datasource")
-                                );
-                                return druidDataSources;
-                              }
-                            }
-                        )
+        handle -> handle
+            .createQuery(StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable()))
+            .fold(
+                new ArrayList<>(),
+                (List<String> druidDataSources,
+                 Map<String, Object> stringObjectMap,
+                 FoldController foldController,
+                 StatementContext statementContext) -> {
+                  druidDataSources.add(MapUtils.getString(stringObjectMap, "datasource"));
+                  return druidDataSources;
+                }
+            )
     );
   }
 
@@ -642,15 +902,12 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
   {
     // See the comment to the pollLock field, explaining this synchronized block
     synchronized (pollLock) {
-      try {
-        doPoll();
-      }
-      catch (Exception e) {
-        log.makeAlert(e, "Problem polling DB.").emit();
-      }
+      doPoll();
     }
   }
 
+  /** This method is extracted from {@link #poll()} solely to reduce code nesting. */
+  @GuardedBy("pollLock")
   private void doPoll()
   {
     log.debug("Starting polling of segment table");
@@ -681,6 +938,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
                         }
                         catch (IOException e) {
                           log.makeAlert(e, "Failed to read segment from db.").emit();
+                          // If one entry in database is corrupted doPoll() should continue to work overall. See
+                          // filter by `Objects::nonNull` below in this method.
                           return null;
                         }
                       }
@@ -692,54 +951,49 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
     );
 
     if (segments == null || segments.isEmpty()) {
-      log.warn("No segments found in the database!");
+      log.info("No segments found in the database!");
       return;
     }
 
     log.info("Polled and found %,d segments in the database", segments.size());
 
-    ConcurrentHashMap<String, DruidDataSource> newDataSources = new ConcurrentHashMap<>();
+    ImmutableMap<String, String> dataSourceProperties = createDefaultDataSourceProperties();
 
-    ImmutableMap<String, String> dataSourceProperties = ImmutableMap.of("created", DateTimes.nowUtc().toString());
-    segments
-        .stream()
-        .filter(Objects::nonNull)
-        .forEach(segment -> {
-          newDataSources
-              .computeIfAbsent(segment.getDataSource(), dsName -> new DruidDataSource(dsName, dataSourceProperties))
-              .addSegmentIfAbsent(segment);
-        });
-
-    // dataSourcesSnapshot is updated only here, please note that if datasources or segments are enabled or disabled
-    // outside of poll, the dataSourcesSnapshot can become invalid until the next poll cycle.
+    // dataSourcesSnapshot is updated only here and the DataSourcesSnapshot object is immutable. If data sources or
+    // segments are marked as used or unused directly (via markAs...() methods in MetadataSegmentManager), the
+    // dataSourcesSnapshot can become invalid until the next database poll.
     // DataSourcesSnapshot computes the overshadowed segments, which makes it an expensive operation if the
-    // snapshot is invalidated on each segment removal, especially if a user issues a lot of single segment remove
-    // calls in rapid succession. So the snapshot update is not done outside of poll at this time.
-    // Updates outside of poll(), were primarily for the user experience, so users would immediately see the effect of
-    // a segment remove call reflected in MetadataResource API calls. These updates outside of scheduled poll may be
-    // added back in removeDataSource and removeSegment methods after the on-demand polling changes from
-    // https://github.com/apache/incubator-druid/pull/7653 are in.
-    final Map<String, ImmutableDruidDataSource> updatedDataSources = CollectionUtils.mapValues(
-        newDataSources,
-        v -> v.toImmutableDruidDataSource()
+    // snapshot was invalidated on each segment mark as unused or used, especially if a user issues a lot of single
+    // segment mark calls in rapid succession. So the snapshot update is not done outside of database poll at this time.
+    // Updates outside of database polls were primarily for the user experience, so users would immediately see the
+    // effect of a segment mark call reflected in MetadataResource API calls.
+    dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments(
+        Iterables.filter(segments, Objects::nonNull), // Filter corrupted entries (see above in this method).
+        dataSourceProperties
     );
-    dataSourcesSnapshot = new DataSourcesSnapshot(updatedDataSources);
+  }
+
+  private static ImmutableMap<String, String> createDefaultDataSourceProperties()
+  {
+    return ImmutableMap.of("created", DateTimes.nowUtc().toString());
   }
 
   /**
    * For the garbage collector in Java, it's better to keep new objects short-living, but once they are old enough
    * (i. e. promoted to old generation), try to keep them alive. In {@link #poll()}, we fetch and deserialize all
-   * existing segments each time, and then replace them in {@link #dataSourcesSnapshot}. This method allows to use already
-   * existing (old) segments when possible, effectively interning them a-la {@link String#intern} or {@link
+   * existing segments each time, and then replace them in {@link #dataSourcesSnapshot}. This method allows to use
+   * already existing (old) segments when possible, effectively interning them a-la {@link String#intern} or {@link
    * com.google.common.collect.Interner}, aiming to make the majority of {@link DataSegment} objects garbage soon after
    * they are deserialized and to die in young generation. It allows to avoid fragmentation of the old generation and
    * full GCs.
    */
   private DataSegment replaceWithExistingSegmentIfPresent(DataSegment segment)
   {
-    ImmutableDruidDataSource dataSource = Optional.ofNullable(dataSourcesSnapshot)
-                                                  .map(m -> m.getDataSourcesMap().get(segment.getDataSource()))
-                                                  .orElse(null);
+    @MonotonicNonNull DataSourcesSnapshot dataSourcesSnapshot = this.dataSourcesSnapshot;
+    if (dataSourcesSnapshot == null) {
+      return segment;
+    }
+    @Nullable ImmutableDruidDataSource dataSource = dataSourcesSnapshot.getDataSource(segment.getDataSource());
     if (dataSource == null) {
       return segment;
     }
@@ -753,11 +1007,7 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
   }
 
   @Override
-  public List<Interval> getUnusedSegmentIntervals(
-      final String dataSource,
-      final Interval interval,
-      final int limit
-  )
+  public List<Interval> getUnusedSegmentIntervals(final String dataSource, final DateTime maxEndTime, final int limit)
   {
     return connector.inReadOnlyTransaction(
         new TransactionCallback<List<Interval>>()
@@ -768,7 +1018,8 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
             Iterator<Interval> iter = handle
                 .createQuery(
                     StringUtils.format(
-                        "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource and start >= :start and %2$send%2$s <= :end and used = false ORDER BY start, %2$send%2$s",
+                        "SELECT start, %2$send%2$s FROM %1$s WHERE dataSource = :dataSource AND "
+                        + "%2$send%2$s <= :end AND used = false ORDER BY start, %2$send%2$s",
                         getSegmentsTable(),
                         connector.getQuoteString()
                     )
@@ -776,8 +1027,7 @@ public class SQLMetadataSegmentManager implements MetadataSegmentManager
                 .setFetchSize(connector.getStreamingFetchSize())
                 .setMaxRows(limit)
                 .bind("dataSource", dataSource)
-                .bind("start", interval.getStart().toString())
-                .bind("end", interval.getEnd().toString())
+                .bind("end", maxEndTime.toString())
                 .map(
                     new BaseResultSetMapper<Interval>()
                     {
diff --git a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java b/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java
index cca37b9..6362077 100644
--- a/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java
+++ b/server/src/main/java/org/apache/druid/metadata/UnknownSegmentIdException.java
@@ -19,13 +19,23 @@
 
 package org.apache.druid.metadata;
 
+import java.util.Collection;
+
 /**
- * Exception thrown by MetadataSegmentManager when an segment id is unknown.
+ * Exception thrown by {@link MetadataSegmentManager} when a segment id is unknown.
  */
-public class UnknownSegmentIdException extends RuntimeException
+public class UnknownSegmentIdException extends Exception
 {
-  public UnknownSegmentIdException(String message)
+  private final Collection<String> unknownSegmentIds;
+
+  UnknownSegmentIdException(Collection<String> segmentIds)
+  {
+    super("Cannot find segment ids " + segmentIds);
+    this.unknownSegmentIds = segmentIds;
+  }
+
+  public Collection<String> getUnknownSegmentIds()
   {
-    super(message);
+    return unknownSegmentIds;
   }
 }
diff --git a/server/src/main/java/org/apache/druid/server/JettyUtils.java b/server/src/main/java/org/apache/druid/server/JettyUtils.java
index 9374fb0..717eb13 100644
--- a/server/src/main/java/org/apache/druid/server/JettyUtils.java
+++ b/server/src/main/java/org/apache/druid/server/JettyUtils.java
@@ -19,10 +19,16 @@
 
 package org.apache.druid.server;
 
+import org.apache.druid.java.util.common.logger.Logger;
+
 import javax.annotation.Nullable;
+import javax.ws.rs.core.MultivaluedMap;
+import javax.ws.rs.core.UriInfo;
 
 public class JettyUtils
 {
+  private static final Logger log = new Logger(JettyUtils.class);
+
   /**
    * Concatenate URI parts, in a way that is useful for proxy servlets.
    *
@@ -46,4 +52,38 @@ public 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;
+  }
 }
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java
index 14bf339..ada41d2 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/CoordinatorDynamicConfig.java
@@ -48,7 +48,7 @@ public class CoordinatorDynamicConfig
 {
   public static final String CONFIG_KEY = "coordinator.config";
 
-  private final long millisToWaitBeforeDeleting;
+  private final long leadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments;
   private final long mergeBytesLimit;
   private final int mergeSegmentsLimit;
   private final int maxSegmentsToMove;
@@ -56,13 +56,29 @@ public class CoordinatorDynamicConfig
   private final int replicationThrottleLimit;
   private final int balancerComputeThreads;
   private final boolean emitBalancingStats;
-  private final boolean killAllDataSources;
-  private final Set<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,
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java
index d2d3029..6f2373b 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/CostBalancerStrategy.java
@@ -117,7 +117,7 @@ public 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
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java
index dff2871..8e559b4 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategy.java
@@ -47,8 +47,8 @@ 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;
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java
index 586f92b..2001b3f 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCluster.java
@@ -22,11 +22,13 @@ 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)
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
index 878bee8..765cb30 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
@@ -19,13 +19,15 @@
 
 package org.apache.druid.server.coordinator;
 
-import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Ordering;
 import com.google.common.collect.Sets;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import com.google.inject.Inject;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMaps;
+import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
 import it.unimi.dsi.fastutil.objects.Object2LongMap;
 import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap;
 import org.apache.curator.framework.CuratorFramework;
@@ -76,7 +78,6 @@ import org.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()) {
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java
index bf4ffde..07cb037 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorCleanupPendingSegments.java
@@ -69,15 +69,15 @@ public 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
         );
       }
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java
index de75bce..d96a7a9 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinatorRuntimeParams.java
@@ -21,8 +21,8 @@ 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,14 +178,18 @@ public class DruidCoordinatorRuntimeParams
     return balancerStrategy;
   }
 
-  public DataSourcesSnapshot getDataSourcesSnapshot()
+  public boolean coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()
   {
-    return dataSourcesSnapshot;
+    long nanosElapsedSinceCoordinatorStart = System.nanoTime() - getStartTimeNanos();
+    long lagNanos = TimeUnit.MILLISECONDS.toNanos(
+        coordinatorDynamicConfig.getLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments()
+    );
+    return nanosElapsedSinceCoordinatorStart > lagNanos;
   }
 
-  public boolean hasDeletionWaitTimeElapsed()
+  public DataSourcesSnapshot getDataSourcesSnapshot()
   {
-    return (System.currentTimeMillis() - getStartTime() > coordinatorDynamicConfig.getMillisToWaitBeforeDeleting());
+    return dataSourcesSnapshot;
   }
 
   public static Builder newBuilder()
@@ -193,12 +200,12 @@ public 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;
-      return this;
-    }
-
-    public Builder withDataSources(Collection<ImmutableDruidDataSource> dataSourcesCollection)
-    {
-      dataSourcesCollection.forEach(
-          dataSource -> {
-            VersionedIntervalTimeline<String, DataSegment> timeline = dataSources.computeIfAbsent(
-                dataSource.getName(),
-                k -> new VersionedIntervalTimeline<>(String.CASE_INSENSITIVE_ORDER)
-            );
-
-            dataSource.getSegments().forEach(
-                segment -> timeline.add(
-                    segment.getInterval(),
-                    segment.getVersion(),
-                    segment.getShardSpec().createChunk(segment)
-                )
-            );
-          }
-      );
+      this.usedSegments = createUsedSegmentsSet(snapshot.iterateAllUsedSegmentsInSnapshot());
+      this.dataSourcesSnapshot = snapshot;
       return this;
     }
 
     /** This method must be used in test code only. */
     @VisibleForTesting
-    public Builder withAvailableSegmentsInTest(DataSegment... availableSegments)
+    public Builder withUsedSegmentsInTest(DataSegment... usedSegments)
     {
-      return withAvailableSegmentsInTest(Arrays.asList(availableSegments));
+      return withUsedSegmentsInTest(Arrays.asList(usedSegments));
     }
 
     /** This method must be used in test code only. */
     @VisibleForTesting
-    public Builder withAvailableSegmentsInTest(Collection<DataSegment> availableSegments)
+    public Builder withUsedSegmentsInTest(Collection<DataSegment> usedSegments)
     {
-      return setAvailableSegments(createAvailableSegmentsSet(availableSegments));
+      this.usedSegments = createUsedSegmentsSet(usedSegments);
+      this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegments(usedSegments, ImmutableMap.of());
+      return this;
     }
 
-    /**
-     * Note: unlike {@link #withAvailableSegmentsInTest(Collection)}, this method doesn't make a defensive copy of the
-     * provided set. The set passed into this method must not be modified afterwards.
-     */
-    public Builder setAvailableSegments(TreeSet<DataSegment> availableSegments)
+    /** This method must be used in test code only. */
+    @VisibleForTesting
+    public Builder withUsedSegmentsTimelinesPerDataSourceInTest(
+        Map<String, VersionedIntervalTimeline<String, DataSegment>> usedSegmentsTimelinesPerDataSource
+    )
     {
-      //noinspection ObjectEquality
-      if (availableSegments.comparator() != DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST) {
-        throw new IllegalArgumentException("Expected DruidCoordinator.SEGMENT_COMPARATOR_RECENT_FIRST");
-      }
-      this.availableSegments = availableSegments;
+      this.dataSourcesSnapshot = DataSourcesSnapshot.fromUsedSegmentsTimelines(
+          usedSegmentsTimelinesPerDataSource,
+          ImmutableMap.of()
+      );
+      usedSegments = createUsedSegmentsSet(dataSourcesSnapshot.iterateAllUsedSegmentsInSnapshot());
       return this;
     }
 
@@ -446,11 +433,5 @@ public class DruidCoordinatorRuntimeParams
       this.balancerStrategy = balancerStrategy;
       return this;
     }
-
-    public Builder withDataSourcesSnapshot(DataSourcesSnapshot snapshot)
-    {
-      this.dataSourcesSnapshot = snapshot;
-      return this;
-    }
   }
 }
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java
index e770ef7..c2c4a7a 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/ReservoirSegmentSampler.java
@@ -34,7 +34,7 @@ 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) {
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java
index ce08bfe..cd04bfd 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/SegmentReplicantLookup.java
@@ -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;
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java
index b4b900b..dc164c8 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorBalancer.java
@@ -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,
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java
index 8131e4f..bde7c15 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowed.java
@@ -46,41 +46,40 @@ public class DruidCoordinatorCleanupOvershadowed implements DruidCoordinatorHelp
   @Override
   public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
   {
-    CoordinatorStats stats = new CoordinatorStats();
+    // Mark as unused overshadowed segments only if we've had enough time to make sure we aren't flapping with old data.
+    if (!params.coordinatorIsLeadingEnoughTimeToMarkAsUnusedOvershadowedSegements()) {
+      return params;
+    }
 
-    // Delete segments that are old
-    // Unservice old partitions if we've had enough time to make sure we aren't flapping with old data
-    if (params.hasDeletionWaitTimeElapsed()) {
-      DruidCluster cluster = params.getDruidCluster();
-      Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = new HashMap<>();
+    CoordinatorStats stats = new CoordinatorStats();
 
-      for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
-        for (ServerHolder serverHolder : serverHolders) {
-          ImmutableDruidServer server = serverHolder.getServer();
+    DruidCluster cluster = params.getDruidCluster();
+    Map<String, VersionedIntervalTimeline<String, DataSegment>> timelines = new HashMap<>();
 
-          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);
-            }
+    for (SortedSet<ServerHolder> serverHolders : cluster.getSortedHistoricalsByTier()) {
+      for (ServerHolder serverHolder : serverHolders) {
+        ImmutableDruidServer server = serverHolder.getServer();
 
-            VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator());
-          }
+        for (ImmutableDruidDataSource dataSource : server.getDataSources()) {
+          VersionedIntervalTimeline<String, DataSegment> timeline = timelines
+              .computeIfAbsent(
+                  dataSource.getName(),
+                  dsName -> new VersionedIntervalTimeline<>(Comparator.naturalOrder())
+              );
+          VersionedIntervalTimeline.addSegments(timeline, dataSource.getSegments().iterator());
         }
       }
+    }
 
-      //Remove all segments in db that are overshadowed by served segments
-      for (DataSegment dataSegment : params.getAvailableSegments()) {
-        VersionedIntervalTimeline<String, DataSegment> timeline = timelines.get(dataSegment.getDataSource());
-        if (timeline != null && timeline.isOvershadowed(dataSegment.getInterval(), dataSegment.getVersion())) {
-          coordinator.removeSegment(dataSegment);
-          stats.addToGlobalStat("overShadowedCount", 1);
-        }
+    // Mark all segments as unused in db that are overshadowed by served segments
+    for (DataSegment dataSegment : params.getUsedSegments()) {
+      VersionedIntervalTimeline<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();
+
+    return params.buildFromExisting().withCoordinatorStats(stats).build();
   }
 }
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java
index 2e77577..af7ccc1 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupUnneeded.java
@@ -43,22 +43,19 @@ public class DruidCoordinatorCleanupUnneeded implements DruidCoordinatorHelper
   public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params)
   {
     CoordinatorStats stats = new CoordinatorStats();
-    Set<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)) {
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java
index a271109..6936363 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java
@@ -19,6 +19,7 @@
 
 package org.apache.druid.server.coordinator.helper;
 
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
 import it.unimi.dsi.fastutil.objects.Object2LongMap;
 import org.apache.druid.client.ImmutableDruidServer;
 import org.apache.druid.java.util.common.logger.Logger;
@@ -214,14 +215,14 @@ public 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())
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java
index 7570c81..c7b3b7a 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorRuleRunner.java
@@ -19,9 +19,7 @@
 
 package org.apache.druid.server.coordinator.helper;
 
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
-import org.apache.druid.client.DataSourcesSnapshot;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.metadata.MetadataRuleManager;
@@ -36,7 +34,6 @@ import org.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;
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java
index f023135..f2a1a9f 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactor.java
@@ -76,7 +76,8 @@ public 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()) {
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java
index 2353247..1b7beab 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentInfoLoader.java
@@ -19,14 +19,12 @@
 
 package org.apache.druid.server.coordinator.helper;
 
-import com.google.common.collect.Iterables;
+import org.apache.druid.client.DataSourcesSnapshot;
 import org.apache.druid.java.util.emitter.EmittingLogger;
 import org.apache.druid.server.coordinator.DruidCoordinator;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
 import org.apache.druid.timeline.DataSegment;
 
-import java.util.TreeSet;
-
 public class DruidCoordinatorSegmentInfoLoader implements DruidCoordinatorHelper
 {
   private static final EmittingLogger log = new EmittingLogger(DruidCoordinatorSegmentInfoLoader.class);
@@ -41,50 +39,27 @@ public 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;
   }
 }
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java
index 3f719d3..302a91d 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKiller.java
@@ -31,10 +31,15 @@ import org.apache.druid.server.coordinator.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();
-
-    if (killAllDataSources && whitelist != null && !whitelist.isEmpty()) {
-      log.error("killAllDataSources can't be true when killDataSourceWhitelist is non-empty, No kill tasks are scheduled.");
+    boolean killAllDataSources = params.getCoordinatorDynamicConfig().isKillUnusedSegmentsInAllDataSources();
+    Collection<String> specificDataSourcesToKill =
+        params.getCoordinatorDynamicConfig().getSpecificDataSourcesToKillUnusedSegmentsIn();
+
+    if (killAllDataSources && specificDataSourcesToKill != null && !specificDataSourcesToKill.isEmpty()) {
+      log.error(
+          "killAllDataSources can't be true when specificDataSourcesToKill is non-empty. No kill tasks are scheduled."
+      );
       return params;
     }
 
+    Collection<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);
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java
index fdd0178..c565df9 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/DropRule.java
@@ -33,7 +33,7 @@ public abstract class DropRule implements Rule
   public CoordinatorStats run(DruidCoordinator coordinator, DruidCoordinatorRuntimeParams params, DataSegment segment)
   {
     CoordinatorStats stats = new CoordinatorStats();
-    coordinator.removeSegment(segment);
+    coordinator.markSegmentAsUnused(segment);
     stats.addToGlobalStat("deletedCount", 1);
     return stats;
   }
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java
index 48aff51..14f791e 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/rules/Rule.java
@@ -52,13 +52,13 @@ public interface Rule
   boolean appliesTo(Interval interval, DateTime referenceTimestamp);
 
   /**
-   * {@link DruidCoordinatorRuntimeParams#getAvailableSegments()} must not be called in Rule's code, because the
-   * available segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is
-   * because {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions:
+   * {@link DruidCoordinatorRuntimeParams#getUsedSegments()} must not be called in Rule's code, because the used
+   * segments are not specified for the {@link DruidCoordinatorRuntimeParams} passed into Rule's code. This is because
+   * {@link DruidCoordinatorRuntimeParams} entangles two slightly different (nonexistent yet) abstractions:
    * "DruidCoordinatorHelperParams" and "RuleParams" which contain params that only {@link
    * org.apache.druid.server.coordinator.helper.DruidCoordinatorHelper}s and Rules need, respectively.
    * For example, {@link org.apache.druid.server.coordinator.ReplicationThrottler} needs to belong only to "RuleParams",
-   * but not "DruidCoordinatorHelperParams". The opposite for "AvailableSegments".
+   * but not "DruidCoordinatorHelperParams". The opposite for the collection of used segments.
    *
    * See https://github.com/apache/incubator-druid/issues/7228
    */
diff --git a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java
index 779e909..9522ca8 100644
--- a/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java
+++ b/server/src/main/java/org/apache/druid/server/http/CoordinatorResource.java
@@ -91,7 +91,7 @@ public class CoordinatorResource
   )
   {
     if (simple != null) {
-      return Response.ok(coordinator.getSegmentAvailability()).build();
+      return Response.ok(coordinator.computeNumsUnavailableUsedSegmentsPerDataSource()).build();
     }
 
     if (full != null) {
diff --git a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java
index a2f04ff..7d49767 100644
--- a/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java
+++ b/server/src/main/java/org/apache/druid/server/http/DataSourcesResource.java
@@ -22,10 +22,12 @@ 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)
+  {
+    MarkSegments markSegments = () -> segmentsMetadata.markAsUsedAllNonOvershadowedSegmentsInDataSource(dataSourceName);
+    return doMarkSegments("markAsUsedAllNonOvershadowedSegments", dataSourceName, markSegments);
+  }
+
+  @POST
+  @Path("/{dataSourceName}/markUsed")
+  @Produces(MediaType.APPLICATION_JSON)
+  @ResourceFilters(DatasourceResourceFilter.class)
+  public Response markAsUsedNonOvershadowedSegments(
+      @PathParam("dataSourceName") String dataSourceName,
+      MarkDataSourceSegmentsPayload payload
+  )
+  {
+    MarkSegments markSegments = () -> {
+      final Interval interval = payload.getInterval();
+      if (interval != null) {
+        return segmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(dataSourceName, interval);
+      } else {
+        final Set<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 (!databaseSegmentManager.enableDataSource(dataSourceName)) {
-      return Response.noContent().build();
+    if (payload == null || !payload.isValid()) {
+      log.warn("Invalid request payload: [%s]", payload);
+      return Response
+          .status(Response.Status.BAD_REQUEST)
+          .entity("Invalid request payload, either interval or segmentIds array must be specified")
+          .build();
+    }
+
+    final ImmutableDruidDataSource dataSource = getDataSource(dataSourceName);
+    if (dataSource == null) {
+      return logAndCreateDataSourceNotFoundResponse(dataSourceName);
     }
 
-    return Response.ok().build();
+    return doMarkSegments(method, dataSourceName, markSegments);
+  }
+
+  private static Response logAndCreateDataSourceNotFoundResponse(String dataSourceName)
+  {
+    log.warn("datasource not found [%s]", dataSourceName);
+    return Response.noContent().build();
+  }
+
+  private static Response doMarkSegments(String method, String dataSourceName, MarkSegments markSegments)
+  {
+    try {
+      int numChangedSegments = markSegments.markSegments();
+      return Response.ok(ImmutableMap.of("numChangedSegments", numChangedSegments)).build();
+    }
+    catch (UnknownSegmentIdException e) {
+      log.warn("Segment ids %s are not found", e.getUnknownSegmentIds());
+      return Response
+          .status(Response.Status.NOT_FOUND)
+          .entity(ImmutableMap.of("message", e.getMessage()))
+          .build();
+    }
+    catch (Exception e) {
+      log.error(e, "Error occurred during [%s] call, data source: [%s]", method, dataSourceName);
+      return Response
+          .serverError()
+          .entity(ImmutableMap.of("error", "Exception occurred.", "message", Throwables.getRootCause(e).toString()))
+          .build();
+    }
   }
 
-  /* When this method is removed, a new method needs to be introduced corresponding to
-    the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters).
-    Ultimately we want to have no method with kill parameter -
-    DELETE `{dataSourceName}` will be used to disable datasource and
-    DELETE `{dataSourceName}/intervals/{interval}` will be used to nuke segments
-  */
+  /**
+   * When this method is removed, a new method needs to be introduced corresponding to
+   * the end point "DELETE /druid/coordinator/v1/datasources/{dataSourceName}" (with no query parameters).
+   * Ultimately we want to have no method with kill parameter -
+   * DELETE `{dataSourceName}` will be used to mark all segments belonging to a data source as unused, and
+   * DELETE `{dataSourceName}/intervals/{interval}` will be used to kill segments within an interval
+   */
   @DELETE
   @Deprecated
   @Path("/{dataSourceName}")
   @ResourceFilters(DatasourceResourceFilter.class)
   @Produces(MediaType.APPLICATION_JSON)
-  public Response deleteDataSource(
+  public Response markAsUnusedAllSegmentsOrKillSegmentsInInterval(
       @PathParam("dataSourceName") final String dataSourceName,
       @QueryParam("kill") final String kill,
       @QueryParam("interval") final String interval
@@ -211,47 +297,20 @@ public 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.ok().build();
+      return Response
+          .serverError()
+          .entity(
+              ImmutableMap.of(
+                  "error", "Exception occurred. Are you sure you have an indexing service?",
+                  "message", e.toString()
+              )
+          )
+          .build();
+    }
   }
 
   @GET
   @Path("/{dataSourceName}/intervals")
   @Produces(MediaType.APPLICATION_JSON)
   @ResourceFilters(DatasourceResourceFilter.class)
-  public Response getSegmentDataSourceIntervals(
+  public Response getIntervalsWithServedSegmentsOrAllServedSegmentsPerIntervals(
       @PathParam("dataSourceName") String dataSourceName,
       @QueryParam("simple") String simple,
       @QueryParam("full") String full
@@ -289,21 +352,22 @@ public 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
     )
diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java
index 556ed3d..f4cbcfa 100644
--- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java
+++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java
@@ -23,15 +23,15 @@ import com.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();
@@ -130,73 +130,60 @@ 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()));
+    if (includeOvershadowedStatus != null) {
+      return getAllUsedSegmentsWithOvershadowedStatus(req, dataSources);
     }
-    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 {
+    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 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
-      );
+    final Iterable<DataSegment> authorizedSegments =
+        AuthorizationUtils.filterAuthorizedResources(req, usedSegments::iterator, raGenerator, authorizerMapper);
 
-      Response.ResponseBuilder builder = Response.status(Response.Status.OK);
-      return builder.entity(authorizedSegments).build();
-    }
+    Response.ResponseBuilder builder = Response.status(Response.Status.OK);
+    return builder.entity(authorizedSegments).build();
   }
 
-  private Iterable<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();
     }
diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
index 8170014..2a8e755 100644
--- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
+++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java
@@ -2198,7 +2198,8 @@ 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<>(
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java
similarity index 62%
copy from core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
copy to server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java
index ec8f7d8..fea252f 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/server/src/test/java/org/apache/druid/client/ImmutableDruidServerTests.java
@@ -17,26 +17,23 @@
  * under the License.
  */
 
-package org.apache.druid.timeline.partition;
+package org.apache.druid.client;
 
-/**
- */
-public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
+import org.apache.druid.timeline.DataSegment;
+import org.easymock.EasyMock;
+
+import java.util.Collection;
+
+public final class ImmutableDruidServerTests
 {
-  public ImmutablePartitionHolder(PartitionHolder partitionHolder)
-  {
-    super(partitionHolder);
-  }
 
-  @Override
-  public PartitionChunk<T> remove(PartitionChunk<T> tPartitionChunk)
+  public static void expectSegments(ImmutableDruidServer mockServer, Collection<DataSegment> segments)
   {
-    throw new UnsupportedOperationException();
+    EasyMock.expect(mockServer.iterateAllSegments()).andReturn(segments).anyTimes();
+    EasyMock.expect(mockServer.getNumSegments()).andReturn(segments.size()).anyTimes();
   }
 
-  @Override
-  public void add(PartitionChunk<T> tPartitionChunk)
+  private ImmutableDruidServerTests()
   {
-    throw new UnsupportedOperationException();
   }
 }
diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java
index ebd3b6f..0b6c235 100644
--- a/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java
+++ b/server/src/test/java/org/apache/druid/client/indexing/ClientKillQueryTest.java
@@ -32,35 +32,35 @@ public class ClientKillQueryTest
   private static final String DATA_SOURCE = "data_source";
   public static final DateTime START = DateTimes.nowUtc();
   private static final Interval INTERVAL = new Interval(START, START.plus(1));
-  ClientKillQuery clientKillQuery;
+  ClientKillQuery clientKillUnusedSegmentsQuery;
 
   @Before
   public void setUp()
   {
-    clientKillQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL);
+    clientKillUnusedSegmentsQuery = new ClientKillQuery(DATA_SOURCE, INTERVAL);
   }
 
   @After
   public void tearDown()
   {
-    clientKillQuery = null;
+    clientKillUnusedSegmentsQuery = null;
   }
 
   @Test
   public void testGetType()
   {
-    Assert.assertEquals("kill", clientKillQuery.getType());
+    Assert.assertEquals("kill", clientKillUnusedSegmentsQuery.getType());
   }
 
   @Test
   public void testGetDataSource()
   {
-    Assert.assertEquals(DATA_SOURCE, clientKillQuery.getDataSource());
+    Assert.assertEquals(DATA_SOURCE, clientKillUnusedSegmentsQuery.getDataSource());
   }
 
   @Test
   public void testGetInterval()
   {
-    Assert.assertEquals(INTERVAL, clientKillQuery.getInterval());
+    Assert.assertEquals(INTERVAL, clientKillUnusedSegmentsQuery.getInterval());
   }
 }
diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java
index a5d436b..a1852be 100644
--- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java
+++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java
@@ -25,6 +25,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Iterables;
+import org.apache.druid.client.ImmutableDruidDataSource;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.java.util.common.StringUtils;
@@ -33,14 +34,13 @@ import org.apache.druid.segment.TestHelper;
 import org.apache.druid.server.metrics.NoopServiceEmitter;
 import org.apache.druid.timeline.DataSegment;
 import org.apache.druid.timeline.partition.NoneShardSpec;
-import org.hamcrest.core.IsInstanceOf;
 import org.joda.time.Interval;
+import org.joda.time.Period;
 import org.junit.After;
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.ExpectedException;
 
 import java.io.IOException;
 import java.util.stream.Collectors;
@@ -48,57 +48,64 @@ import java.util.stream.Collectors;
 
 public class SQLMetadataSegmentManagerTest
 {
-  @Rule
-  public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule();
+  private static DataSegment createSegment(
+      String dataSource,
+      String interval,
+      String version,
+      String bucketKey,
+      int binaryVersion
+  )
+  {
+    return new DataSegment(
+        dataSource,
+        Intervals.of(interval),
+        version,
+        ImmutableMap.of(
+            "type", "s3_zip",
+            "bucket", "test",
+            "key", dataSource + "/" + bucketKey
+        ),
+        ImmutableList.of("dim1", "dim2", "dim3"),
+        ImmutableList.of("count", "value"),
+        NoneShardSpec.instance(),
+        binaryVersion,
+        1234L
+    );
+  }
 
   @Rule
-  public ExpectedException thrown = ExpectedException.none();
+  public final TestDerbyConnector.DerbyConnectorRule derbyConnectorRule = new TestDerbyConnector.DerbyConnectorRule();
 
-  private SQLMetadataSegmentManager manager;
+  private SQLMetadataSegmentManager sqlSegmentsMetadata;
   private SQLMetadataSegmentPublisher publisher;
   private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
 
-  private final DataSegment segment1 = new DataSegment(
+  private final DataSegment segment1 = createSegment(
       "wikipedia",
-      Intervals.of("2012-03-15T00:00:00.000/2012-03-16T00:00:00.000"),
+      "2012-03-15T00:00:00.000/2012-03-16T00:00:00.000",
       "2012-03-16T00:36:30.848Z",
-      ImmutableMap.of(
-          "type", "s3_zip",
-          "bucket", "test",
-          "key", "wikipedia/index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip"
-      ),
-      ImmutableList.of("dim1", "dim2", "dim3"),
-      ImmutableList.of("count", "value"),
-      NoneShardSpec.instance(),
-      0,
-      1234L
+      "index/y=2012/m=03/d=15/2012-03-16T00:36:30.848Z/0/index.zip",
+      0
   );
 
-  private final DataSegment segment2 = new DataSegment(
+  private final DataSegment segment2 = createSegment(
       "wikipedia",
-      Intervals.of("2012-01-05T00:00:00.000/2012-01-06T00:00:00.000"),
+      "2012-01-05T00:00:00.000/2012-01-06T00:00:00.000",
       "2012-01-06T22:19:12.565Z",
-      ImmutableMap.of(
-          "type", "s3_zip",
-          "bucket", "test",
-          "key", "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip"
-      ),
-      ImmutableList.of("dim1", "dim2", "dim3"),
-      ImmutableList.of("count", "value"),
-      NoneShardSpec.instance(),
-      0,
-      1234L
+      "wikipedia/index/y=2012/m=01/d=05/2012-01-06T22:19:12.565Z/0/index.zip",
+      0
   );
 
   private void publish(DataSegment segment, boolean used) throws IOException
   {
+    boolean partitioned = !(segment.getShardSpec() instanceof NoneShardSpec);
     publisher.publishSegment(
         segment.getId().toString(),
         segment.getDataSource(),
         DateTimes.nowUtc().toString(),
         segment.getInterval().getStart().toString(),
         segment.getInterval().getEnd().toString(),
-        (segment.getShardSpec() instanceof NoneShardSpec) ? false : true,
+        partitioned,
         segment.getVersion(),
         used,
         jsonMapper.writeValueAsBytes(segment)
@@ -109,12 +116,15 @@ public class SQLMetadataSegmentManagerTest
   public void setUp() throws Exception
   {
     TestDerbyConnector connector = derbyConnectorRule.getConnector();
-    manager = new SQLMetadataSegmentManager(
+    MetadataSegmentManagerConfig config = new MetadataSegmentManagerConfig();
+    config.setPollDuration(Period.seconds(1));
+    sqlSegmentsMetadata = new SQLMetadataSegmentManager(
         jsonMapper,
-        Suppliers.ofInstance(new MetadataSegmentManagerConfig()),
+        Suppliers.ofInstance(config),
         derbyConnectorRule.metadataTablesConfigSupplier(),
         connector
     );
+    sqlSegmentsMetadata.start();
 
     publisher = new SQLMetadataSegmentPublisher(
         jsonMapper,
@@ -131,67 +141,104 @@ public class SQLMetadataSegmentManagerTest
   @After
   public void teardown()
   {
-    if (manager.isStarted()) {
-      manager.stop();
+    if (sqlSegmentsMetadata.isPollingDatabasePeriodically()) {
+      sqlSegmentsMetadata.stopPollingDatabasePeriodically();
     }
+    sqlSegmentsMetadata.stop();
   }
 
   @Test
   public void testPoll()
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
     Assert.assertEquals(
         ImmutableList.of("wikipedia"),
-        manager.getAllDataSourceNames()
+        sqlSegmentsMetadata.retrieveAllDataSourceNames()
     );
     Assert.assertEquals(
         ImmutableList.of("wikipedia"),
-        manager.getDataSources().stream().map(d -> d.getName()).collect(Collectors.toList())
+        sqlSegmentsMetadata
+            .getImmutableDataSourcesWithAllUsedSegments()
+            .stream()
+            .map(ImmutableDruidDataSource::getName)
+            .collect(Collectors.toList())
     );
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.getDataSource("wikipedia").getSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia").getSegments())
     );
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
   @Test
-  public void testNoPoll()
+  public void testPrepareImmutableDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOException
   {
-    manager.start();
-    Assert.assertTrue(manager.isStarted());
+    DataSegment newSegment = pollThenStopThenStartIntro();
     Assert.assertEquals(
-        ImmutableList.of("wikipedia"),
-        manager.getAllDataSourceNames()
+        ImmutableSet.of(newSegment),
+        ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia2").getSegments())
+    );
+  }
+
+  @Test
+  public void testGetDataSourceWithUsedSegmentsAwaitsPollOnRestart() throws IOException
+  {
+    DataSegment newSegment = pollThenStopThenStartIntro();
+    Assert.assertEquals(
+        ImmutableSet.of(newSegment),
+        ImmutableSet.copyOf(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments("wikipedia2").getSegments())
+    );
+  }
+
+  @Test
+  public void testPrepareImmutableDataSourcesWithAllUsedSegmentsAwaitsPollOnRestart() throws IOException
+  {
+    DataSegment newSegment = pollThenStopThenStartIntro();
+    Assert.assertEquals(
+        ImmutableSet.of(segment1, segment2, newSegment),
+        ImmutableSet.copyOf(
+            sqlSegmentsMetadata
+                .getImmutableDataSourcesWithAllUsedSegments()
+                .stream()
+                .flatMap((ImmutableDruidDataSource dataSource) -> dataSource.getSegments().stream())
+                .iterator()
+        )
     );
-    Assert.assertNull(manager.getDataSources());
-    Assert.assertNull(manager.getDataSource("wikipedia"));
-    Assert.assertNull(manager.iterateAllSegments());
   }
 
   @Test
-  public void testPollThenStop()
+  public void testIterateAllUsedSegmentsAwaitsPollOnRestart() throws IOException
+  {
+    DataSegment newSegment = pollThenStopThenStartIntro();
+    Assert.assertEquals(
+        ImmutableSet.of(segment1, segment2, newSegment),
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
+    );
+  }
+
+  private DataSegment pollThenStopThenStartIntro() throws IOException
   {
-    manager.start();
-    manager.poll();
-    manager.stop();
-    Assert.assertFalse(manager.isStarted());
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    sqlSegmentsMetadata.stopPollingDatabasePeriodically();
+    Assert.assertFalse(sqlSegmentsMetadata.isPollingDatabasePeriodically());
     Assert.assertEquals(
         ImmutableList.of("wikipedia"),
-        manager.getAllDataSourceNames()
+        sqlSegmentsMetadata.retrieveAllDataSourceNames()
     );
-    Assert.assertNull(manager.getDataSources());
-    Assert.assertNull(manager.getDataSource("wikipedia"));
-    Assert.assertNull(manager.iterateAllSegments());
+    DataSegment newSegment = createNewSegment1("wikipedia2");
+    publisher.publishSegment(newSegment);
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    return newSegment;
   }
 
   @Test
-  public void testPollWithCurroptedSegment()
+  public void testPollWithCorruptedSegment()
   {
     //create a corrupted segment entry in segments table, which tests
     //that overall loading of segments from database continues to work
@@ -209,355 +256,246 @@ public class SQLMetadataSegmentManagerTest
     );
 
     EmittingLogger.registerEmitter(new NoopServiceEmitter());
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
     Assert.assertEquals(
-        "wikipedia", Iterables.getOnlyElement(manager.getDataSources()).getName()
+        "wikipedia",
+        Iterables.getOnlyElement(sqlSegmentsMetadata.getImmutableDataSourcesWithAllUsedSegments()).getName()
     );
   }
 
   @Test
-  public void testGetUnusedSegmentsForInterval()
+  public void testGetUnusedSegmentIntervals()
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-    Assert.assertTrue(manager.removeDataSource("wikipedia"));
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
+    int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource("wikipedia");
+    Assert.assertEquals(2, numChangedSegments);
 
     Assert.assertEquals(
         ImmutableList.of(segment2.getInterval()),
-        manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 1)
+        sqlSegmentsMetadata.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 1)
     );
 
     Assert.assertEquals(
         ImmutableList.of(segment2.getInterval(), segment1.getInterval()),
-        manager.getUnusedSegmentIntervals("wikipedia", Intervals.of("1970/3000"), 5)
+        sqlSegmentsMetadata.getUnusedSegmentIntervals("wikipedia", DateTimes.of("3000"), 5)
     );
   }
 
-  @Test
-  public void testRemoveDataSource() throws IOException
+  @Test(timeout = 60_000)
+  public void testMarkAsUnusedAllSegmentsInDataSource() throws IOException, InterruptedException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
     final String newDataSource = "wikipedia2";
-    final DataSegment newSegment = new DataSegment(
+    final DataSegment newSegment = createNewSegment1(newDataSource);
+
+    publisher.publishSegment(newSegment);
+
+    awaitDataSourceAppeared(newDataSource);
+    int numChangedSegments = sqlSegmentsMetadata.markAsUnusedAllSegmentsInDataSource(newDataSource);
+    Assert.assertEquals(1, numChangedSegments);
+    awaitDataSourceDisappeared(newDataSource);
+    Assert.assertNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource));
+  }
+
+  private static DataSegment createNewSegment1(String newDataSource)
+  {
+    return createSegment(
         newDataSource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
+        "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
+  }
 
-    publisher.publishSegment(newSegment);
-
-    Assert.assertNull(manager.getDataSource(newDataSource));
-    Assert.assertTrue(manager.removeDataSource(newDataSource));
+  private static DataSegment createNewSegment2(String newDataSource)
+  {
+    return createSegment(
+        newDataSource,
+        "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000",
+        "2017-10-15T20:19:12.565Z",
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
+    );
   }
 
-  @Test
-  public void testRemoveDataSegment() throws IOException
+  @Test(timeout = 60_000)
+  public void testMarkSegmentAsUnused() throws IOException, InterruptedException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
     final String newDataSource = "wikipedia2";
-    final DataSegment newSegment = new DataSegment(
+    final DataSegment newSegment = createSegment(
         newDataSource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
+        "2017-10-15T00:00:00.000/2017-10-16T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
     publisher.publishSegment(newSegment);
+    awaitDataSourceAppeared(newDataSource);
+    Assert.assertNotNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource));
+
+    Assert.assertTrue(sqlSegmentsMetadata.markSegmentAsUnused(newSegment.getId().toString()));
+    awaitDataSourceDisappeared(newDataSource);
+    Assert.assertNull(sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource));
+  }
 
-    Assert.assertNull(manager.getDataSource(newDataSource));
-    Assert.assertTrue(manager.removeSegment(newSegment.getId().toString()));
+  private void awaitDataSourceAppeared(String newDataSource) throws InterruptedException
+  {
+    while (sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(newDataSource) == null) {
+      Thread.sleep(1000);
+    }
+  }
+
+  private void awaitDataSourceDisappeared(String dataSource) throws InterruptedException
+  {
+    while (sqlSegmentsMetadata.getImmutableDataSourceWithUsedSegments(dataSource) != null) {
+      Thread.sleep(1000);
+    }
   }
 
   @Test
-  public void testEnableSegmentsWithSegmentIds() throws IOException
+  public void testMarkAsUsedNonOvershadowedSegments() throws Exception
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"),
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
+
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createSegment(
+        newDataSource,
+        "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
+    final DataSegment newSegment2 = createSegment(
+        newDataSource,
+        "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000",
         "2017-10-16T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        1,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        1
     );
 
     // Overshadowed by newSegment2
-    final DataSegment newSegment3 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
+    final DataSegment newSegment3 = createSegment(
+        newDataSource,
+        "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        1,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        1
     );
 
     publish(newSegment1, false);
     publish(newSegment2, false);
     publish(newSegment3, false);
-    final ImmutableList<String> segmentIds = ImmutableList.of(
+    final ImmutableSet<String> segmentIds = ImmutableSet.of(
         newSegment1.getId().toString(),
         newSegment2.getId().toString(),
         newSegment3.getId().toString()
     );
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
-    Assert.assertEquals(2, manager.enableSegments(datasource, segmentIds));
-    manager.poll();
+    Assert.assertEquals(2, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments(newDataSource, segmentIds));
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2, newSegment1, newSegment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
-  @Test
-  public void testEnableSegmentsWithSegmentIdsInvalidDatasource() throws IOException
+  @Test(expected = UnknownSegmentIdException.class)
+  public void testMarkAsUsedNonOvershadowedSegmentsInvalidDataSource() throws Exception
   {
-    thrown.expectCause(IsInstanceOf.instanceOf(UnknownSegmentIdException.class));
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
+
+    final DataSegment newSegment2 = createNewSegment1(newDataSource);
 
     publish(newSegment1, false);
     publish(newSegment2, false);
-    final ImmutableList<String> segmentIds = ImmutableList.of(
-        newSegment1.getId().toString(),
-        newSegment2.getId().toString()
-    );
-    manager.poll();
+    final ImmutableSet<String> segmentIds =
+        ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString());
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
-    // none of the segments are in datasource
-    Assert.assertEquals(0, manager.enableSegments("wrongDataSource", segmentIds));
+    // none of the segments are in data source
+    Assert.assertEquals(0, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments("wrongDataSource", segmentIds));
   }
 
-  @Test
-  public void testEnableSegmentsWithInvalidSegmentIds()
+  @Test(expected = UnknownSegmentIdException.class)
+  public void testMarkAsUsedNonOvershadowedSegmentsWithInvalidSegmentIds() throws UnknownSegmentIdException
   {
-    thrown.expectCause(IsInstanceOf.instanceOf(UnknownSegmentIdException.class));
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
 
-    final ImmutableList<String> segmentIds = ImmutableList.of(
-        newSegment1.getId().toString(),
-        newSegment2.getId().toString()
-    );
-    manager.poll();
+    final DataSegment newSegment2 = createNewSegment1(newDataSource);
+
+    final ImmutableSet<String> segmentIds =
+        ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString());
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
-    // none of the segments are in datasource
-    Assert.assertEquals(0, manager.enableSegments(datasource, segmentIds));
+    // none of the segments are in data source
+    Assert.assertEquals(0, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegments(newDataSource, segmentIds));
   }
 
   @Test
-  public void testEnableSegmentsWithInterval() throws IOException
+  public void testMarkAsUsedNonOvershadowedSegmentsInInterval() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
+
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
+    final DataSegment newSegment2 = createSegment(
+        newDataSource,
+        "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000",
         "2017-10-16T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        1,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        1
     );
 
-    final DataSegment newSegment3 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-19T00:00:00.000/2017-10-20T00:00:00.000"),
+    final DataSegment newSegment3 = createSegment(
+        newDataSource,
+        "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
     // Overshadowed by newSegment2
-    final DataSegment newSegment4 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final DataSegment newSegment4 = createNewSegment2(newDataSource);
 
     publish(newSegment1, false);
     publish(newSegment2, false);
@@ -565,141 +503,75 @@ public class SQLMetadataSegmentManagerTest
     publish(newSegment4, false);
     final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000");
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
 
     // 2 out of 3 segments match the interval
-    Assert.assertEquals(2, manager.enableSegments(datasource, theInterval));
+    Assert.assertEquals(2, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval));
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2, newSegment1, newSegment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
   @Test(expected = IllegalArgumentException.class)
-  public void testEnableSegmentsWithInvalidInterval() throws IOException
+  public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithInvalidInterval() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
+
+    final DataSegment newSegment2 = createNewSegment2(newDataSource);
 
     publish(newSegment1, false);
     publish(newSegment2, false);
     // invalid interval start > end
     final Interval theInterval = Intervals.of("2017-10-22T00:00:00.000/2017-10-02T00:00:00.000");
-    manager.enableSegments(datasource, theInterval);
+    sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval);
   }
 
   @Test
-  public void testEnableSegmentsWithOverlappingInterval() throws IOException
+  public void testMarkAsUsedNonOvershadowedSegmentsInIntervalWithOverlappingInterval() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"),
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
+
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createSegment(
+        newDataSource,
+        "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
+    final DataSegment newSegment2 = createSegment(
+        newDataSource,
+        "2017-10-17T00:00:00.000/2017-10-18T00:00:00.000",
         "2017-10-16T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        1,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        1
     );
 
-    final DataSegment newSegment3 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-19T00:00:00.000/2017-10-22T00:00:00.000"),
+    final DataSegment newSegment3 = createSegment(
+        newDataSource,
+        "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
     // Overshadowed by newSegment2
-    final DataSegment newSegment4 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final DataSegment newSegment4 = createNewSegment2(newDataSource);
 
     publish(newSegment1, false);
     publish(newSegment2, false);
@@ -707,183 +579,90 @@ public class SQLMetadataSegmentManagerTest
     publish(newSegment4, false);
     final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000");
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
 
-    // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be disabled
-    Assert.assertEquals(1, manager.enableSegments(datasource, theInterval));
+    // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused
+    Assert.assertEquals(1, sqlSegmentsMetadata.markAsUsedNonOvershadowedSegmentsInInterval(newDataSource, theInterval));
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2, newSegment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
   @Test
-  public void testDisableSegmentsWithSegmentIds() throws IOException
+  public void testMarkSegmentsAsUnused() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
+
+    final DataSegment newSegment2 = createNewSegment1(newDataSource);
 
     publisher.publishSegment(newSegment1);
     publisher.publishSegment(newSegment2);
-    final ImmutableList<String> segmentIds = ImmutableList.of(newSegment1.getId().toString(), newSegment1.getId().toString());
+    final ImmutableSet<String> segmentIds =
+        ImmutableSet.of(newSegment1.getId().toString(), newSegment1.getId().toString());
 
-    Assert.assertEquals(segmentIds.size(), manager.disableSegments(datasource, segmentIds));
-    manager.poll();
+    Assert.assertEquals(segmentIds.size(), sqlSegmentsMetadata.markSegmentsAsUnused(newDataSource, segmentIds));
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
   @Test
-  public void testDisableSegmentsWithSegmentIdsInvalidDatasource() throws IOException
+  public void testMarkSegmentsAsUnusedInvalidDataSource() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
+
+    final DataSegment newSegment2 = createNewSegment1(newDataSource);
 
     publisher.publishSegment(newSegment1);
     publisher.publishSegment(newSegment2);
-    final ImmutableList<String> segmentIds = ImmutableList.of(
-        newSegment1.getId().toString(),
-        newSegment2.getId().toString()
-    );
-    // none of the segments are in datasource
-    Assert.assertEquals(0, manager.disableSegments("wrongDataSource", segmentIds));
-    manager.poll();
+    final ImmutableSet<String> segmentIds =
+        ImmutableSet.of(newSegment1.getId().toString(), newSegment2.getId().toString());
+    // none of the segments are in data source
+    Assert.assertEquals(0, sqlSegmentsMetadata.markSegmentsAsUnused("wrongDataSource", segmentIds));
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2, newSegment1, newSegment2),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
   @Test
-  public void testDisableSegmentsWithInterval() throws IOException
+  public void testMarkAsUnusedSegmentsInInterval() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
 
-    final DataSegment newSegment3 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-19T00:00:00.000/2017-10-20T00:00:00.000"),
+    final DataSegment newSegment2 = createNewSegment2(newDataSource);
+
+    final DataSegment newSegment3 = createSegment(
+        newDataSource,
+        "2017-10-19T00:00:00.000/2017-10-20T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
     publisher.publishSegment(newSegment1);
@@ -892,116 +671,58 @@ public class SQLMetadataSegmentManagerTest
     final Interval theInterval = Intervals.of("2017-10-15T00:00:00.000/2017-10-18T00:00:00.000");
 
     // 2 out of 3 segments match the interval
-    Assert.assertEquals(2, manager.disableSegments(datasource, theInterval));
+    Assert.assertEquals(2, sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval));
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2, newSegment3),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
   @Test(expected = IllegalArgumentException.class)
-  public void testDisableSegmentsWithInvalidInterval() throws IOException
+  public void testMarkAsUnusedSegmentsInIntervalWithInvalidInterval() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-16T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createNewSegment1(newDataSource);
+
+    final DataSegment newSegment2 = createNewSegment2(newDataSource);
 
     publisher.publishSegment(newSegment1);
     publisher.publishSegment(newSegment2);
     // invalid interval start > end
     final Interval theInterval = Intervals.of("2017-10-22T00:00:00.000/2017-10-02T00:00:00.000");
-    manager.disableSegments(datasource, theInterval);
+    sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval);
   }
 
   @Test
-  public void testDisableSegmentsWithOverlappingInterval() throws IOException
+  public void testMarkAsUnusedSegmentsInIntervalWithOverlappingInterval() throws IOException
   {
-    manager.start();
-    manager.poll();
-    Assert.assertTrue(manager.isStarted());
-
-    final String datasource = "wikipedia2";
-    final DataSegment newSegment1 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-15T00:00:00.000/2017-10-17T00:00:00.000"),
-        "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
-    );
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.poll();
+    Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically());
 
-    final DataSegment newSegment2 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-17T00:00:00.000/2017-10-18T00:00:00.000"),
+    final String newDataSource = "wikipedia2";
+    final DataSegment newSegment1 = createSegment(
+        newDataSource,
+        "2017-10-15T00:00:00.000/2017-10-17T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
-    final DataSegment newSegment3 = new DataSegment(
-        datasource,
-        Intervals.of("2017-10-19T00:00:00.000/2017-10-22T00:00:00.000"),
+    final DataSegment newSegment2 = createNewSegment2(newDataSource);
+
+    final DataSegment newSegment3 = createSegment(
+        newDataSource,
+        "2017-10-19T00:00:00.000/2017-10-22T00:00:00.000",
         "2017-10-15T20:19:12.565Z",
-        ImmutableMap.of(
-            "type", "s3_zip",
-            "bucket", "test",
-            "key", "wikipedia2/index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip"
-        ),
-        ImmutableList.of("dim1", "dim2", "dim3"),
-        ImmutableList.of("count", "value"),
-        NoneShardSpec.instance(),
-        0,
-        1234L
+        "index/y=2017/m=10/d=15/2017-10-16T20:19:12.565Z/0/index.zip",
+        0
     );
 
     publisher.publishSegment(newSegment1);
@@ -1009,13 +730,13 @@ public class SQLMetadataSegmentManagerTest
     publisher.publishSegment(newSegment3);
     final Interval theInterval = Intervals.of("2017-10-16T00:00:00.000/2017-10-20T00:00:00.000");
 
-    // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be disabled
-    Assert.assertEquals(1, manager.disableSegments(datasource, theInterval));
+    // 1 out of 3 segments match the interval, other 2 overlap, only the segment fully contained will be marked unused
+    Assert.assertEquals(1, sqlSegmentsMetadata.markAsUnusedSegmentsInInterval(newDataSource, theInterval));
 
-    manager.poll();
+    sqlSegmentsMetadata.poll();
     Assert.assertEquals(
         ImmutableSet.of(segment1, segment2, newSegment1, newSegment3),
-        ImmutableSet.copyOf(manager.iterateAllSegments())
+        ImmutableSet.copyOf(sqlSegmentsMetadata.iterateAllUsedSegments())
     );
   }
 
@@ -1023,9 +744,9 @@ public class SQLMetadataSegmentManagerTest
   public void testStopAndStart()
   {
     // Simulate successive losing and getting the coordinator leadership
-    manager.start();
-    manager.stop();
-    manager.start();
-    manager.stop();
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.stopPollingDatabasePeriodically();
+    sqlSegmentsMetadata.startPollingDatabasePeriodically();
+    sqlSegmentsMetadata.stopPollingDatabasePeriodically();
   }
 }
diff --git a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java
index 4d7d110..9db997b 100644
--- a/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java
+++ b/server/src/test/java/org/apache/druid/segment/realtime/appenderator/TestUsedSegmentChecker.java
@@ -41,13 +41,7 @@ public 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) {
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java
similarity index 55%
copy from core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
copy to server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java
index ec8f7d8..dfc98e5 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/ImmutablePartitionHolder.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/CoordinatorRuntimeParamsTestHelpers.java
@@ -17,26 +17,28 @@
  * under the License.
  */
 
-package org.apache.druid.timeline.partition;
+package org.apache.druid.server.coordinator;
 
-/**
- */
-public class ImmutablePartitionHolder<T> extends PartitionHolder<T>
+import org.apache.druid.java.util.common.DateTimes;
+
+public class CoordinatorRuntimeParamsTestHelpers
 {
-  public ImmutablePartitionHolder(PartitionHolder partitionHolder)
+  public static DruidCoordinatorRuntimeParams.Builder newBuilder()
   {
-    super(partitionHolder);
+    return DruidCoordinatorRuntimeParams
+        .newBuilder()
+        .withStartTimeNanos(System.nanoTime())
+        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"));
   }
 
-  @Override
-  public PartitionChunk<T> remove(PartitionChunk<T> tPartitionChunk)
+  public static DruidCoordinatorRuntimeParams.Builder newBuilder(DruidCluster druidCluster)
   {
-    throw new UnsupportedOperationException();
+    return newBuilder()
+        .withDruidCluster(druidCluster)
+        .withSegmentReplicantLookup(SegmentReplicantLookup.make(druidCluster));
   }
 
-  @Override
-  public void add(PartitionChunk<T> tPartitionChunk)
+  private CoordinatorRuntimeParamsTestHelpers()
   {
-    throw new UnsupportedOperationException();
   }
 }
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java
index e6aa17e..3074fb2 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/CostBalancerStrategyTest.java
@@ -23,6 +23,7 @@ import com.google.common.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));
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java
index a8faeac..eb3b25b 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/CuratorDruidCoordinatorTest.java
@@ -83,7 +83,10 @@ 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())
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java
index 0bbe46d..bd2aa6e 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DiskNormalizedCostBalancerStrategyTest.java
@@ -23,6 +23,7 @@ import com.google.common.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));
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java
new file mode 100644
index 0000000..772b7ae
--- /dev/null
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterBuilder.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator;
+
+import javax.annotation.Nullable;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+public final class DruidClusterBuilder
+{
+  public static DruidClusterBuilder newBuilder()
+  {
+    return new DruidClusterBuilder();
+  }
+
+  private @Nullable Set<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);
+  }
+}
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java
index c07830e..798c891 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidClusterTest.java
@@ -21,7 +21,6 @@ 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(
+        )
+        .addTier(
             "tier1",
-            Stream.of(
-                new ServerHolder(
-                    new ImmutableDruidServer(
-                        new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0),
-                        0L,
-                        ImmutableMap.of("src1", dataSources.get("src1")),
-                        1
-                    ),
-                    new LoadQueuePeonTester()
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new ImmutableDruidServer(
+                    new DruidServerMetadata("name1", "host1", null, 100L, ServerType.HISTORICAL, "tier1", 0),
+                    0L,
+                    ImmutableMap.of("src1", dataSources.get("src1")),
+                    1
+                ),
+                new LoadQueuePeonTester()
+            )
         )
-    );
+        .build();
   }
 
   @Test
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java
index ed4f600..fa0954f 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerProfiler.java
@@ -23,6 +23,7 @@ import com.google.common.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);
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java
index e094c3a..585a061 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTest.java
@@ -20,11 +20,11 @@
 package org.apache.druid.server.coordinator;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
 import org.apache.druid.client.ImmutableDruidServer;
+import org.apache.druid.client.ImmutableDruidServerTests;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.server.coordination.ServerType;
 import org.apache.druid.timeline.DataSegment;
@@ -520,19 +520,19 @@ 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();
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java
index fad85d0..24b8de3 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorBalancerTester.java
@@ -25,17 +25,8 @@ import org.apache.druid.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)
   {
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java
index 7d38fa0..2b79753 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorRuleRunnerTest.java
@@ -20,11 +20,9 @@
 package org.apache.druid.server.coordinator;
 
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.ListeningExecutorService;
 import com.google.common.util.concurrent.MoreExecutors;
-import org.apache.druid.client.DataSourcesSnapshot;
 import org.apache.druid.client.DruidServer;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.Intervals;
@@ -55,23 +53,22 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.concurrent.Executors;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
 
 /**
  */
 public class DruidCoordinatorRuleRunnerTest
 {
+  public static final CoordinatorDynamicConfig COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS =
+      CoordinatorDynamicConfig.builder().withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L).build();
+
   private DruidCoordinator coordinator;
   private LoadQueuePeon mockPeon;
-  private List<DataSegment> availableSegments;
+  private List<DataSegment> usedSegments;
   private DruidCoordinatorRuleRunner ruleRunner;
   private ServiceEmitter emitter;
   private MetadataRuleManager databaseRuleManager;
-  private MetadataSegmentManager databaseSegmentManager;
-  private DataSourcesSnapshot dataSourcesSnapshot;
+  private MetadataSegmentManager segmentsMetadata;
 
   @Before
   public void setUp()
@@ -81,13 +78,12 @@ public class DruidCoordinatorRuleRunnerTest
     emitter = EasyMock.createMock(ServiceEmitter.class);
     EmittingLogger.registerEmitter(emitter);
     databaseRuleManager = EasyMock.createMock(MetadataRuleManager.class);
-    databaseSegmentManager = EasyMock.createNiceMock(MetadataSegmentManager.class);
-    dataSourcesSnapshot = EasyMock.createNiceMock(DataSourcesSnapshot.class);
+    segmentsMetadata = EasyMock.createNiceMock(MetadataSegmentManager.class);
 
     DateTime start = DateTimes.of("2012-01-01");
-    availableSegments = new ArrayList<>();
+    usedSegments = new ArrayList<>();
     for (int i = 0; i < 24; i++) {
-      availableSegments.add(
+      usedSegments.add(
           new DataSegment(
               "test",
               new Interval(start, start.plusHours(1)),
@@ -118,8 +114,6 @@ public class DruidCoordinatorRuleRunnerTest
    * hot - 1 replicant
    * normal - 1 replicant
    * cold - 1 replicant
-   *
-   * @throws Exception
    */
   @Test
   public void testRunThreeTiersOneReplicant()
@@ -146,72 +140,40 @@ public class DruidCoordinatorRuleRunnerTest
         )).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "hot",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot",
-                        "hostHot",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
+            new ServerHolder(
+                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
+        )
+        .addTier(
             "normal",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverNorm",
-                        "hostNorm",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "normal",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
+            new ServerHolder(
+                new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
+        )
+        .addTier(
             "cold",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverCold",
-                        "hostCold",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "cold",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build())
-            .build();
+    DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy)
+        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build())
+        .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
@@ -226,12 +188,29 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.verify(mockPeon);
   }
 
+  private DruidCoordinatorRuntimeParams.Builder makeCoordinatorRuntimeParams(
+      DruidCluster druidCluster,
+      BalancerStrategy balancerStrategy
+  )
+  {
+    return createCoordinatorRuntimeParams(druidCluster)
+        .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
+        .withBalancerStrategy(balancerStrategy);
+  }
+
+  private DruidCoordinatorRuntimeParams.Builder createCoordinatorRuntimeParams(DruidCluster druidCluster)
+  {
+    return CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
+        .withDruidCluster(druidCluster)
+        .withUsedSegmentsInTest(usedSegments)
+        .withDatabaseRuleManager(databaseRuleManager);
+  }
+
   /**
    * Nodes:
    * hot - 2 replicants
    * cold - 1 replicant
-   *
-   * @throws Exception
    */
   @Test
   public void testRunTwoTiersTwoReplicants()
@@ -255,68 +234,35 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "hot",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot",
-                        "hostHot",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                ),
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot2",
-                        "hostHot2",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
+            new ServerHolder(
+                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            ),
+            new ServerHolder(
+                new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
+        )
+        .addTier(
             "cold",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverCold",
-                        "hostCold",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "cold",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer("serverCold", "hostCold", null, 1000, ServerType.HISTORICAL, "cold", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .build();
+    DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
@@ -334,8 +280,6 @@ public class DruidCoordinatorRuleRunnerTest
    * Nodes:
    * hot - 1 replicant
    * normal - 1 replicant
-   *
-   * @throws Exception
    */
   @Test
   public void testRunTwoTiersWithExistingSegments()
@@ -359,63 +303,33 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidServer normServer = new DruidServer(
-        "serverNorm",
-        "hostNorm",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment availableSegment : availableSegments) {
-      normServer.addDataSegment(availableSegment);
+    DruidServer normServer = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : usedSegments) {
+      normServer.addDataSegment(segment);
     }
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "hot",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot",
-                        "hostHot",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
-            "normal",
-            Stream.of(
-                new ServerHolder(
-                    normServer.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .addTier("normal", new ServerHolder(normServer.toImmutableDruidServer(), mockPeon))
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(segmentReplicantLookup)
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .build();
+    DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster)
+        .withSegmentReplicantLookup(segmentReplicantLookup)
+        .withBalancerStrategy(balancerStrategy)
+        .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
@@ -455,42 +369,24 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "normal",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverNorm",
-                        "hostNorm",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "normal",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withEmitter(emitter)
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .build();
+    DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy)
+        .withEmitter(emitter)
+        .build();
 
     ruleRunner.run(params);
 
@@ -522,35 +418,22 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock.expect(mockPeon.getLoadQueueSize()).andReturn(0L).anyTimes();
     EasyMock.replay(databaseRuleManager, mockPeon);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "normal",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverNorm",
-                        "hostNorm",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "normal",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withEmitter(emitter)
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .build();
+    DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster)
+        .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
+        .withEmitter(emitter)
+        .build();
 
     ruleRunner.run(params);
 
@@ -565,7 +448,7 @@ public class DruidCoordinatorRuleRunnerTest
     mockEmptyPeon();
 
     EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes();
-    coordinator.removeSegment(EasyMock.anyObject());
+    coordinator.markSegmentAsUnused(EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
     EasyMock.replay(coordinator);
 
@@ -580,47 +463,25 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidServer server = new DruidServer(
-        "serverNorm",
-        "hostNorm",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment segment : availableSegments) {
+    DruidServer server = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : usedSegments) {
       server.addDataSegment(segment);
     }
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
-            "normal",
-            Stream.of(
-                new ServerHolder(
-                    server.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
-        )
-    );
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier("normal", new ServerHolder(server.toImmutableDruidServer(), mockPeon))
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder()
-        .withDruidCluster(druidCluster)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build())
-        .withAvailableSegmentsInTest(availableSegments)
-        .withDatabaseRuleManager(databaseRuleManager)
+    DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster)
+        .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS)
         .withSegmentReplicantLookup(segmentReplicantLookup)
         .withBalancerStrategy(balancerStrategy)
-        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
@@ -651,62 +512,37 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidServer server1 = new DruidServer(
-        "serverNorm",
-        "hostNorm",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    server1.addDataSegment(availableSegments.get(0));
-
-    DruidServer server2 = new DruidServer(
-        "serverNorm2",
-        "hostNorm2",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment segment : availableSegments) {
+    DruidServer server1 = new DruidServer("serverNorm", "hostNorm", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    server1.addDataSegment(usedSegments.get(0));
+
+    DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : usedSegments) {
       server2.addDataSegment(segment);
     }
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "normal",
-            Stream.of(
-                new ServerHolder(
-                    server1.toImmutableDruidServer(),
-                    mockPeon
-                ),
-                new ServerHolder(
-                    server2.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
+            new ServerHolder(server2.toImmutableDruidServer(), mockPeon)
         )
-    );
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder()
-        .withDruidCluster(druidCluster)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build())
-        .withAvailableSegmentsInTest(availableSegments)
-        .withDatabaseRuleManager(databaseRuleManager)
+    DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster)
+        .withDynamicConfigs(
+            CoordinatorDynamicConfig
+                .builder()
+                .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0L)
+                .build()
+        )
         .withSegmentReplicantLookup(segmentReplicantLookup)
         .withBalancerStrategy(balancerStrategy)
-        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
@@ -740,64 +576,28 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidServer server1 = new DruidServer(
-        "server1",
-        "host1",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "hot",
-        0
-    );
-    server1.addDataSegment(availableSegments.get(0));
-    DruidServer server2 = new DruidServer(
-        "serverNorm2",
-        "hostNorm2",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment segment : availableSegments) {
+    DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0);
+    server1.addDataSegment(usedSegments.get(0));
+    DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : usedSegments) {
       server2.addDataSegment(segment);
     }
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
-            "hot",
-            Stream.of(
-                    new ServerHolder(
-                        server1.toImmutableDruidServer(),
-                        mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
-            "normal",
-            Stream.of(
-                new ServerHolder(
-                    server2.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
-        )
-    );
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon))
+        .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon))
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder()
-        .withDruidCluster(druidCluster)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build())
-        .withAvailableSegmentsInTest(availableSegments)
-        .withDatabaseRuleManager(databaseRuleManager)
+    DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster)
+        .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS)
         .withSegmentReplicantLookup(segmentReplicantLookup)
         .withBalancerStrategy(balancerStrategy)
-        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
@@ -829,54 +629,27 @@ public class DruidCoordinatorRuleRunnerTest
     ).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidServer server1 = new DruidServer(
-        "server1",
-        "host1",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "hot",
-        0
-    );
-    DruidServer server2 = new DruidServer(
-        "serverNorm2",
-        "hostNorm2",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment segment : availableSegments) {
+    DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "hot", 0);
+    DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : usedSegments) {
       server2.addDataSegment(segment);
     }
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
-            "hot",
-            Stream.of(new ServerHolder(server1.toImmutableDruidServer(), mockPeon))
-                  .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
-            "normal",
-            Stream.of(new ServerHolder(server2.toImmutableDruidServer(), mockPeon))
-                  .collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
-        )
-    );
+
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier("hot", new ServerHolder(server1.toImmutableDruidServer(), mockPeon))
+        .addTier("normal", new ServerHolder(server2.toImmutableDruidServer(), mockPeon))
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder()
-        .withDruidCluster(druidCluster)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build())
-        .withAvailableSegmentsInTest(availableSegments)
-        .withDatabaseRuleManager(databaseRuleManager)
+    DruidCoordinatorRuntimeParams params = createCoordinatorRuntimeParams(druidCluster)
+        .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS)
         .withSegmentReplicantLookup(segmentReplicantLookup)
         .withBalancerStrategy(balancerStrategy)
-        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
@@ -906,37 +679,13 @@ public class DruidCoordinatorRuleRunnerTest
         .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidServer server1 = new DruidServer(
-        "server1",
-        "host1",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    server1.addDataSegment(availableSegments.get(0));
-    DruidServer server2 = new DruidServer(
-        "serverNorm2",
-        "hostNorm2",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    server2.addDataSegment(availableSegments.get(1));
-    DruidServer server3 = new DruidServer(
-        "serverNorm3",
-        "hostNorm3",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    server3.addDataSegment(availableSegments.get(1));
-    server3.addDataSegment(availableSegments.get(2));
+    DruidServer server1 = new DruidServer("server1", "host1", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    server1.addDataSegment(usedSegments.get(0));
+    DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    server2.addDataSegment(usedSegments.get(1));
+    DruidServer server3 = new DruidServer("serverNorm3", "hostNorm3", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    server3.addDataSegment(usedSegments.get(1));
+    server3.addDataSegment(usedSegments.get(2));
 
     mockPeon.dropSegment(EasyMock.anyObject(), EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
@@ -948,33 +697,29 @@ public class DruidCoordinatorRuleRunnerTest
 
     EasyMock.replay(anotherMockPeon);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "normal",
-            Stream.of(
-                new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false),
-                new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false),
-                new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false)
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(server1.toImmutableDruidServer(), mockPeon, false),
+            new ServerHolder(server2.toImmutableDruidServer(), anotherMockPeon, false),
+            new ServerHolder(server3.toImmutableDruidServer(), anotherMockPeon, false)
         )
-    );
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder()
+    DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
         .withDruidCluster(druidCluster)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build())
-        .withAvailableSegmentsInTest(availableSegments)
+        .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS)
+        .withUsedSegmentsInTest(usedSegments)
         .withDatabaseRuleManager(databaseRuleManager)
         .withSegmentReplicantLookup(segmentReplicantLookup)
         .withBalancerStrategy(balancerStrategy)
-        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
@@ -990,15 +735,11 @@ public class DruidCoordinatorRuleRunnerTest
   /**
    * Nodes:
    * hot - 2 replicants
-   *
-   * @throws Exception
    */
   @Test
   public void testReplicantThrottle()
   {
-    EasyMock.expect(dataSourcesSnapshot.getOvershadowedSegments()).andReturn(ImmutableSet.of()).anyTimes();
-    EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes();
-    EasyMock.replay(coordinator, databaseSegmentManager, dataSourcesSnapshot);
+    mockCoordinator();
     mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
     EasyMock.expectLastCall().atLeastOnce();
     mockEmptyPeon();
@@ -1006,7 +747,8 @@ public class DruidCoordinatorRuleRunnerTest
     EasyMock
         .expect(databaseRuleManager.getRulesWithDefault(EasyMock.anyObject()))
         .andReturn(
-            Collections.singletonList(new IntervalLoadRule(
+            Collections.singletonList(
+                new IntervalLoadRule(
                     Intervals.of("2012-01-01T00:00:00.000Z/2013-01-01T00:00:00.000Z"),
                     ImmutableMap.of("hot", 2)
                 )
@@ -1015,53 +757,27 @@ public class DruidCoordinatorRuleRunnerTest
         .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "hot",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot",
-                        "hostHot",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                ),
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot2",
-                        "hostHot2",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            ),
+            new ServerHolder(
+                new DruidServer("serverHot2", "hostHot2", null, 1000, ServerType.HISTORICAL, "hot", 0)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .build();
+    DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
@@ -1083,13 +799,13 @@ public class DruidCoordinatorRuleRunnerTest
     );
 
     afterParams = ruleRunner.run(
-        new DruidCoordinatorRuntimeParams.Builder()
+        CoordinatorRuntimeParamsTestHelpers
+            .newBuilder()
             .withDruidCluster(druidCluster)
             .withEmitter(emitter)
-            .withAvailableSegmentsInTest(Collections.singletonList(overFlowSegment))
+            .withUsedSegmentsInTest(overFlowSegment)
             .withDatabaseRuleManager(databaseRuleManager)
             .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
             .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
             .build()
     );
@@ -1107,8 +823,6 @@ public class DruidCoordinatorRuleRunnerTest
    * Nodes:
    * hot - nothing loaded
    * _default_tier - 1 segment loaded
-   *
-   * @throws Exception
    */
   @Test
   public void testReplicantThrottleAcrossTiers()
@@ -1123,9 +837,7 @@ public class DruidCoordinatorRuleRunnerTest
                                     .build()
         )
         .atLeastOnce();
-    EasyMock.expect(dataSourcesSnapshot.getOvershadowedSegments()).andReturn(ImmutableSet.of()).anyTimes();
-    EasyMock.replay(dataSourcesSnapshot);
-    coordinator.removeSegment(EasyMock.anyObject());
+    coordinator.markSegmentAsUnused(EasyMock.anyObject());
     EasyMock.expectLastCall().anyTimes();
     EasyMock.replay(coordinator);
     mockPeon.loadSegment(EasyMock.anyObject(), EasyMock.anyObject());
@@ -1147,56 +859,37 @@ public class DruidCoordinatorRuleRunnerTest
         .atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "hot",
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot",
-                        "hostHot",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        "hot",
-                        1
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder()))),
+            new ServerHolder(
+                new DruidServer("serverHot", "hostHot", null, 1000, ServerType.HISTORICAL, "hot", 1)
+                    .toImmutableDruidServer(),
+                mockPeon
+            )
+        )
+        .addTier(
             DruidServer.DEFAULT_TIER,
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverNorm",
-                        "hostNorm",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        DruidServer.DEFAULT_TIER,
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer(
+                    "serverNorm",
+                    "hostNorm",
+                    null,
+                    1000,
+                    ServerType.HISTORICAL,
+                    DruidServer.DEFAULT_TIER,
+                    0
+                ).toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .build();
+    DruidCoordinatorRuntimeParams params = makeCoordinatorRuntimeParams(druidCluster, balancerStrategy).build();
 
     DruidCoordinatorRuleRunner runner = new DruidCoordinatorRuleRunner(new ReplicationThrottler(7, 1), coordinator);
     DruidCoordinatorRuntimeParams afterParams = runner.run(params);
@@ -1242,66 +935,40 @@ public class DruidCoordinatorRuleRunnerTest
         1,
         0
     );
-    List<DataSegment> longerAvailableSegments = Lists.newArrayList(availableSegments);
-    longerAvailableSegments.add(overFlowSegment);
-
-    DruidServer server1 = new DruidServer(
-        "serverNorm1",
-        "hostNorm1",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment availableSegment : longerAvailableSegments) {
-      server1.addDataSegment(availableSegment);
+    List<DataSegment> longerUsedSegments = Lists.newArrayList(usedSegments);
+    longerUsedSegments.add(overFlowSegment);
+
+    DruidServer server1 = new DruidServer("serverNorm1", "hostNorm1", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : longerUsedSegments) {
+      server1.addDataSegment(segment);
     }
-    DruidServer server2 = new DruidServer(
-        "serverNorm2",
-        "hostNorm2",
-        null,
-        1000,
-        ServerType.HISTORICAL,
-        "normal",
-        0
-    );
-    for (DataSegment availableSegment : longerAvailableSegments) {
-      server2.addDataSegment(availableSegment);
+    DruidServer server2 = new DruidServer("serverNorm2", "hostNorm2", null, 1000, ServerType.HISTORICAL, "normal", 0);
+    for (DataSegment segment : longerUsedSegments) {
+      server2.addDataSegment(segment);
     }
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             "normal",
-            Stream.of(
-                new ServerHolder(
-                    server1.toImmutableDruidServer(),
-                    mockPeon
-                ),
-                new ServerHolder(
-                    server2.toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(server1.toImmutableDruidServer(), mockPeon),
+            new ServerHolder(server2.toImmutableDruidServer(), mockPeon)
         )
-    );
+        .build();
 
     SegmentReplicantLookup segmentReplicantLookup = SegmentReplicantLookup.make(druidCluster);
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params = new DruidCoordinatorRuntimeParams.Builder()
+    DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
         .withDruidCluster(druidCluster)
-        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMillisToWaitBeforeDeleting(0L).build())
-        .withAvailableSegmentsInTest(longerAvailableSegments)
+        .withDynamicConfigs(COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS)
+        .withUsedSegmentsInTest(longerUsedSegments)
         .withDatabaseRuleManager(databaseRuleManager)
         .withSegmentReplicantLookup(segmentReplicantLookup)
         .withBalancerStrategy(balancerStrategy)
-        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
         .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
@@ -1316,7 +983,7 @@ public class DruidCoordinatorRuleRunnerTest
   @Test
   public void testRulesRunOnNonOvershadowedSegmentsOnly()
   {
-    Set<DataSegment> availableSegments = new HashSet<>();
+    Set<DataSegment> usedSegments = new HashSet<>();
     DataSegment v1 = new DataSegment(
         "test",
         Intervals.of("2012-01-01/2012-01-02"),
@@ -1339,11 +1006,10 @@ public class DruidCoordinatorRuleRunnerTest
         IndexIO.CURRENT_VERSION_ID,
         1
     );
-    availableSegments.add(v1);
-    availableSegments.add(v2);
-    EasyMock.expect(dataSourcesSnapshot.getOvershadowedSegments()).andReturn(ImmutableSet.of(v1.getId())).anyTimes();
-    EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes();
-    EasyMock.replay(coordinator, dataSourcesSnapshot);
+    usedSegments.add(v1);
+    usedSegments.add(v2);
+
+    mockCoordinator();
     mockPeon.loadSegment(EasyMock.eq(v2), EasyMock.anyObject());
     EasyMock.expectLastCall().once();
     mockEmptyPeon();
@@ -1352,43 +1018,38 @@ public class DruidCoordinatorRuleRunnerTest
         Collections.singletonList(new ForeverLoadRule(ImmutableMap.of(DruidServer.DEFAULT_TIER, 1)))).atLeastOnce();
     EasyMock.replay(databaseRuleManager);
 
-    DruidCluster druidCluster = new DruidCluster(
-        null,
-        ImmutableMap.of(
+    DruidCluster druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier(
             DruidServer.DEFAULT_TIER,
-            Stream.of(
-                new ServerHolder(
-                    new DruidServer(
-                        "serverHot",
-                        "hostHot",
-                        null,
-                        1000,
-                        ServerType.HISTORICAL,
-                        DruidServer.DEFAULT_TIER,
-                        0
-                    ).toImmutableDruidServer(),
-                    mockPeon
-                )
-            ).collect(Collectors.toCollection(() -> new TreeSet<>(Collections.reverseOrder())))
+            new ServerHolder(
+                new DruidServer(
+                    "serverHot",
+                    "hostHot",
+                    null,
+                    1000,
+                    ServerType.HISTORICAL,
+                    DruidServer.DEFAULT_TIER,
+                    0
+                ).toImmutableDruidServer(),
+                mockPeon
+            )
         )
-    );
+        .build();
 
-    ListeningExecutorService exec = MoreExecutors.listeningDecorator(
-        Executors.newFixedThreadPool(1));
-    BalancerStrategy balancerStrategy =
-        new CostBalancerStrategyFactory().createBalancerStrategy(exec);
+    ListeningExecutorService exec = MoreExecutors.listeningDecorator(Executors.newFixedThreadPool(1));
+    BalancerStrategy balancerStrategy = new CostBalancerStrategyFactory().createBalancerStrategy(exec);
 
-    DruidCoordinatorRuntimeParams params =
-        new DruidCoordinatorRuntimeParams.Builder()
-            .withDruidCluster(druidCluster)
-            .withAvailableSegmentsInTest(availableSegments)
-            .withDatabaseRuleManager(databaseRuleManager)
-            .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
-            .withBalancerStrategy(balancerStrategy)
-            .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
-            .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build())
-            .withDataSourcesSnapshot(dataSourcesSnapshot)
-            .build();
+    DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
+        .newBuilder()
+        .withDruidCluster(druidCluster)
+        .withUsedSegmentsInTest(usedSegments)
+        .withDatabaseRuleManager(databaseRuleManager)
+        .withSegmentReplicantLookup(SegmentReplicantLookup.make(new DruidCluster()))
+        .withBalancerStrategy(balancerStrategy)
+        .withBalancerReferenceTimestamp(DateTimes.of("2013-01-01"))
+        .withDynamicConfigs(CoordinatorDynamicConfig.builder().withMaxSegmentsToMove(5).build())
+        .build();
 
     DruidCoordinatorRuntimeParams afterParams = ruleRunner.run(params);
     CoordinatorStats stats = afterParams.getCoordinatorStats();
@@ -1398,9 +1059,9 @@ public class DruidCoordinatorRuleRunnerTest
     Assert.assertTrue(stats.getTiers("unassignedCount").isEmpty());
     Assert.assertTrue(stats.getTiers("unassignedSize").isEmpty());
 
-    Assert.assertEquals(2, availableSegments.size());
-    Assert.assertEquals(availableSegments, params.getAvailableSegments());
-    Assert.assertEquals(availableSegments, afterParams.getAvailableSegments());
+    Assert.assertEquals(2, usedSegments.size());
+    Assert.assertEquals(usedSegments, params.getUsedSegments());
+    Assert.assertEquals(usedSegments, afterParams.getUsedSegments());
 
     EasyMock.verify(mockPeon);
     exec.shutdown();
@@ -1409,9 +1070,9 @@ public class DruidCoordinatorRuleRunnerTest
   private void mockCoordinator()
   {
     EasyMock.expect(coordinator.getDynamicConfigs()).andReturn(createCoordinatorDynamicConfig()).anyTimes();
-    coordinator.removeSegment(EasyMock.anyObject());
+    coordinator.markSegmentAsUnused(EasyMock.anyObject());
     EasyMock.expectLastCall().anyTimes();
-    EasyMock.replay(coordinator);
+    EasyMock.replay(coordinator, segmentsMetadata);
   }
 
   private void mockEmptyPeon()
@@ -1426,7 +1087,7 @@ public class DruidCoordinatorRuleRunnerTest
   private CoordinatorDynamicConfig createCoordinatorDynamicConfig()
   {
     return CoordinatorDynamicConfig.builder()
-                                   .withMillisToWaitBeforeDeleting(0)
+                                   .withLeadingTimeMillisBeforeCanMarkAsUnusedOvershadowedSegments(0)
                                    .withMergeBytesLimit(0)
                                    .withMergeSegmentsLimit(0)
                                    .withMaxSegmentsToMove(0)
@@ -1434,8 +1095,8 @@ public class DruidCoordinatorRuleRunnerTest
                                    .withReplicationThrottleLimit(24)
                                    .withBalancerComputeThreads(0)
                                    .withEmitBalancingStats(false)
-                                   .withKillDataSourceWhitelist(null)
-                                   .withKillAllDataSources(false)
+                                   .withSpecificDataSourcesToKillUnusedSegmentsIn(null)
+                                   .withKillUnusedSegmentsInAllDataSources(false)
                                    .withMaxSegmentsInNodeLoadingQueue(1000)
                                    .build();
   }
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java
index 3df7fc6..b587842 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/DruidCoordinatorTest.java
@@ -23,6 +23,7 @@ import com.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
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java
index 001dc2a..73e829c 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/ReservoirSegmentSamplerTest.java
@@ -21,6 +21,7 @@ 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);
 
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java
index b8699fb..13942b0 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/cost/CachingCostBalancerStrategyTest.java
@@ -115,7 +115,7 @@ 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);
   }
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java
index 0ae0ee6..f450bf3 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java
@@ -20,15 +20,17 @@
 package org.apache.druid.server.coordinator.helper;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import org.apache.druid.client.ImmutableDruidDataSource;
 import org.apache.druid.client.ImmutableDruidServer;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
 import org.apache.druid.server.coordinator.CoordinatorStats;
 import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidClusterBuilder;
 import org.apache.druid.server.coordinator.DruidCoordinator;
+import org.apache.druid.server.coordinator.DruidCoordinatorRuleRunnerTest;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
 import org.apache.druid.server.coordinator.LoadQueuePeon;
 import org.apache.druid.server.coordinator.ServerHolder;
@@ -38,17 +40,13 @@ import org.joda.time.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())))
-        ));
+    druidCluster = DruidClusterBuilder
+        .newBuilder()
+        .addTier("normal", new ServerHolder(druidServer, mockPeon))
+        .build();
 
-    DruidCoordinatorRuntimeParams params = DruidCoordinatorRuntimeParams
+    DruidCoordinatorRuntimeParams params = CoordinatorRuntimeParamsTestHelpers
         .newBuilder()
-        .withAvailableSegmentsInTest(availableSegments)
+        .withUsedSegmentsInTest(usedSegments)
         .withCoordinatorStats(new CoordinatorStats())
         .withDruidCluster(druidCluster)
+        .withDynamicConfigs(
+            DruidCoordinatorRuleRunnerTest.COORDINATOR_CONFIG_WITH_ZERO_LEADING_TIME_BEFORE_CAN_MARK_AS_UNUSED_OVERSHADOWED_SEGMENTS
+        )
         .build();
-    druidCoordinatorCleanupOvershadowed.run(params);
+    druidCoordinatorMarkAsUnusedOvershadowedSegments.run(params);
     EasyMock.verify(coordinator, druidDataSource, druidServer);
   }
 }
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java
index bdf711f..0eb8b39 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentCompactorTest.java
@@ -21,7 +21,9 @@ 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 k = 0; k < 2; k++) {
-          DataSegment segment = createSegment(dataSource, j, true, k);
-          timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment));
-          segment = createSegment(dataSource, j, false, k);
-          timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment));
-        }
-      }
-
-      for (int j = 7; j < 9; j++) {
+      for (int j : new int[] {0, 1, 2, 3, 7, 8}) {
         for (int k = 0; k < 2; k++) {
-          DataSegment segment = createSegment(dataSource, j, true, k);
-          timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment));
-          segment = createSegment(dataSource, j, false, k);
-          timeline.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(segment));
+          segments.add(createSegment(dataSource, j, true, k));
+          segments.add(createSegment(dataSource, j, false, k));
         }
       }
-
-      dataSources.put(dataSource, timeline);
     }
+    dataSources = DataSourcesSnapshot
+        .fromUsedSegments(segments, ImmutableMap.of())
+        .getUsedSegmentsTimelinesPerDataSource();
   }
 
   private static DataSegment createSegment(String dataSource, int startDay, boolean beforeNoon, int partition)
@@ -261,9 +248,9 @@ public 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);
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java
index 0f0bc03..6882c71 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorSegmentKillerTest.java
@@ -25,6 +25,7 @@ import org.apache.druid.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)
     );
   }
 }
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java
index 9c5b9b4..b28bd7f 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java
@@ -625,10 +625,7 @@ 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(
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java
index 431f6de..cdb5e41 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/BroadcastDistributionRuleTest.java
@@ -20,13 +20,14 @@
 package org.apache.druid.server.coordinator.rules;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
 import org.apache.druid.client.DruidServer;
 import org.apache.druid.java.util.common.DateTimes;
 import org.apache.druid.java.util.common.Intervals;
 import org.apache.druid.server.coordination.ServerType;
+import org.apache.druid.server.coordinator.CoordinatorRuntimeParamsTestHelpers;
 import org.apache.druid.server.coordinator.CoordinatorStats;
 import org.apache.druid.server.coordinator.DruidCluster;
+import org.apache.druid.server.coordinator.DruidClusterBuilder;
 import org.apache.druid.server.coordinator.DruidCoordinatorRuntimeParams;
 import org.apache.druid.server.coordinator.LoadQueuePeonTester;
 import org.apache.druid.server.coordinator.SegmentReplicantLookup;
@@ -38,12 +39,8 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.util.ArrayList;
... 2375 lines suppressed ...


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org