diff --git a/.idea/inspectionProfiles/Druid.xml b/.idea/inspectionProfiles/Druid.xml
index 5791c1c456b3..1dd160601d90 100644
--- a/.idea/inspectionProfiles/Druid.xml
+++ b/.idea/inspectionProfiles/Druid.xml
@@ -28,6 +28,10 @@
+
+
+
+
@@ -105,6 +109,7 @@
+
@@ -341,11 +346,19 @@
-
+
-
+
+
+
+
+
+
+
+
+
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 af1f1fe57d6a..91c1409b4071 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
@@ -24,9 +24,9 @@
import org.apache.druid.client.DataSourcesSnapshot;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.server.coordinator.helper.CompactionSegmentIterator;
-import org.apache.druid.server.coordinator.helper.CompactionSegmentSearchPolicy;
-import org.apache.druid.server.coordinator.helper.NewestSegmentFirstPolicy;
+import org.apache.druid.server.coordinator.duty.CompactionSegmentIterator;
+import org.apache.druid.server.coordinator.duty.CompactionSegmentSearchPolicy;
+import org.apache.druid.server.coordinator.duty.NewestSegmentFirstPolicy;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NumberedShardSpec;
diff --git a/core/src/main/java/org/apache/druid/java/util/metrics/AllocationMetricCollectors.java b/core/src/main/java/org/apache/druid/java/util/metrics/AllocationMetricCollectors.java
index 36ffb09cbced..f0d1b2c78e64 100644
--- a/core/src/main/java/org/apache/druid/java/util/metrics/AllocationMetricCollectors.java
+++ b/core/src/main/java/org/apache/druid/java/util/metrics/AllocationMetricCollectors.java
@@ -22,7 +22,6 @@
import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
-
import java.lang.management.ManagementFactory;
import java.lang.management.ThreadMXBean;
import java.lang.reflect.Method;
diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java
index 7f6df4213a28..45fb6639082c 100644
--- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java
+++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnector.java
@@ -19,6 +19,7 @@
package org.apache.druid.metadata;
+import javax.annotation.Nullable;
import java.util.List;
/**
@@ -36,7 +37,11 @@ Void insertOrUpdate(
byte[] value
);
- byte[] lookup(
+ /**
+ * Returns the value of the valueColumn when there is only one row matched to the given key.
+ * This method returns null if there is no such row and throws an error if there are more than one rows.
+ */
+ @Nullable byte[] lookup(
String tableName,
String keyColumn,
String valueColumn,
diff --git a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java
index 8c84bbf89a30..a6651074588e 100644
--- a/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java
+++ b/core/src/main/java/org/apache/druid/metadata/MetadataStorageConnectorConfig.java
@@ -21,6 +21,7 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.java.util.common.StringUtils;
+
import java.util.Properties;
/**
diff --git a/core/src/main/java/org/apache/druid/segment/SegmentUtils.java b/core/src/main/java/org/apache/druid/segment/SegmentUtils.java
index a6d98a47566f..aae1643befda 100644
--- a/core/src/main/java/org/apache/druid/segment/SegmentUtils.java
+++ b/core/src/main/java/org/apache/druid/segment/SegmentUtils.java
@@ -19,6 +19,7 @@
package org.apache.druid.segment;
+import com.google.common.collect.Collections2;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hasher;
import com.google.common.hash.Hashing;
@@ -37,7 +38,6 @@
import java.util.Collection;
import java.util.Collections;
import java.util.List;
-import java.util.stream.Collectors;
/**
* Utility methods useful for implementing deep storage extensions.
@@ -78,16 +78,14 @@ public static int getVersionFromDir(File inDir) throws IOException
}
/**
- * Returns a String with identifiers of "segments" comma-separated. Useful for log messages. Not useful for anything
- * else, because this doesn't take special effort to escape commas that occur in identifiers (not common, but could
- * potentially occur in a datasource name).
+ * Returns an object whose toString() returns a String with identifiers of the given segments, comma-separated. Useful
+ * for log messages. Not useful for anything else, because this doesn't take special effort to escape commas that
+ * occur in identifiers (not common, but could potentially occur in a datasource name).
*/
- public static String commaSeparateIdentifiers(final Collection segments)
+ public static Object commaSeparatedIdentifiers(final Collection segments)
{
- return segments
- .stream()
- .map(segment -> segment.getId().toString())
- .collect(Collectors.joining(", "));
+ // Lazy, to avoid preliminary string creation if logging level is turned off
+ return Collections2.transform(segments, DataSegment::getId);
}
private SegmentUtils()
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 2f2646846ab8..ee2a9f36f68b 100644
--- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
+++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java
@@ -38,7 +38,6 @@
import java.util.Collection;
import java.util.Comparator;
import java.util.HashMap;
-import java.util.HashSet;
import java.util.IdentityHashMap;
import java.util.Iterator;
import java.util.List;
@@ -56,25 +55,38 @@
/**
* VersionedIntervalTimeline is a data structure that manages objects on a specific timeline.
*
- * It associates a jodatime Interval and a generically-typed version with the object that is being stored.
+ * It associates an {@link Interval} and a generically-typed version with the object that is being stored.
*
* In the event of overlapping timeline entries, timeline intervals may be chunked. The underlying data associated
* with a timeline entry remains unchanged when chunking occurs.
*
- * After loading objects via the add() method, the lookup(Interval) method can be used to get the list of the most
- * recent objects (according to the version) that match the given interval. The intent is that objects represent
- * a certain time period and when you do a lookup(), you are asking for all of the objects that you need to look
- * at in order to get a correct answer about that time period.
+ * After loading objects via the {@link #add} method, the {@link #lookup(Interval)} method can be used to get the list
+ * of the most recent objects (according to the version) that match the given interval. The intent is that objects
+ * represent a certain time period and when you do a {@link #lookup(Interval)}, you are asking for all of the objects
+ * that you need to look at in order to get a correct answer about that time period.
*
- * The {@link #findFullyOvershadowed} method returns a list of objects that will never be returned by a call to lookup()
- * because they are overshadowed by some other object. This can be used in conjunction with the add() and remove()
- * methods to achieve "atomic" updates. First add new items, then check if those items caused anything to be
- * overshadowed, if so, remove the overshadowed elements and you have effectively updated your data set without any user
- * impact.
+ * The {@link #findFullyOvershadowed} method returns a list of objects that will never be returned by a call to {@link
+ * #lookup} because they are overshadowed by some other object. This can be used in conjunction with the {@link #add}
+ * and {@link #remove} methods to achieve "atomic" updates. First add new items, then check if those items caused
+ * anything to be overshadowed, if so, remove the overshadowed elements and you have effectively updated your data set
+ * without any user impact.
*/
public class VersionedIntervalTimeline>
implements TimelineLookup
{
+ public static VersionedIntervalTimeline forSegments(Iterable segments)
+ {
+ return forSegments(segments.iterator());
+ }
+
+ public static VersionedIntervalTimeline forSegments(Iterator segments)
+ {
+ final VersionedIntervalTimeline timeline =
+ new VersionedIntervalTimeline<>(Comparator.naturalOrder());
+ addSegments(timeline, segments);
+ return timeline;
+ }
+
private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true);
// Below timelines stores only *visible* timelineEntries
@@ -99,19 +111,6 @@ public VersionedIntervalTimeline(Comparator super VersionType> versionComparat
this.versionComparator = versionComparator;
}
- public static VersionedIntervalTimeline forSegments(Iterable segments)
- {
- return forSegments(segments.iterator());
- }
-
- public static VersionedIntervalTimeline forSegments(Iterator segments)
- {
- final VersionedIntervalTimeline timeline =
- new VersionedIntervalTimeline<>(Comparator.naturalOrder());
- addSegments(timeline, segments);
- return timeline;
- }
-
public static void addSegments(
VersionedIntervalTimeline timeline,
Iterator segments
@@ -151,6 +150,11 @@ public Collection iterateAllObjects()
);
}
+ public int getNumObjects()
+ {
+ return numObjects.get();
+ }
+
/**
* Computes a set with all objects falling within the specified interval which are at least partially "visible" in
* this interval (that is, are not fully overshadowed within this interval).
@@ -371,62 +375,69 @@ public Set> findFullyOvershadowed(
lock.readLock().lock();
try {
// 1. Put all timelineEntries and remove all visible entries to find out only non-visible timelineEntries.
- final Map> overShadowed = new HashMap<>();
- for (Map.Entry> versionEntry : allTimelineEntries.entrySet()) {
- @SuppressWarnings("unchecked")
- Map versionCopy = (TreeMap) versionEntry.getValue().clone();
- overShadowed.put(versionEntry.getKey(), versionCopy);
- }
-
- for (Entry entry : completePartitionsTimeline.entrySet()) {
- Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval());
- if (versionEntry != null) {
- versionEntry.remove(entry.getValue().getVersion());
- if (versionEntry.isEmpty()) {
- overShadowed.remove(entry.getValue().getTrueInterval());
- }
- }
- }
-
- for (Entry entry : incompletePartitionsTimeline.entrySet()) {
- Map versionEntry = overShadowed.get(entry.getValue().getTrueInterval());
- if (versionEntry != null) {
- versionEntry.remove(entry.getValue().getVersion());
- if (versionEntry.isEmpty()) {
- overShadowed.remove(entry.getValue().getTrueInterval());
- }
- }
- }
-
- final Set> retVal = new HashSet<>();
- for (Entry> versionEntry : overShadowed.entrySet()) {
- for (Entry entry : versionEntry.getValue().entrySet()) {
- final TimelineEntry timelineEntry = entry.getValue();
- retVal.add(timelineEntryToObjectHolder(timelineEntry));
- }
- }
+ final Map> overshadowedPartitionsTimeline =
+ computeOvershadowedPartitionsTimeline();
+
+ final Set> overshadowedObjects = overshadowedPartitionsTimeline
+ .values()
+ .stream()
+ .flatMap(
+ (Map entry) -> entry.values().stream().map(this::timelineEntryToObjectHolder)
+ )
+ .collect(Collectors.toSet());
- // 2. Visible timelineEntries can also have overshadowed segments. Add them to the result too.
+ // 2. Visible timelineEntries can also have overshadowed objects. Add them to the result too.
for (TimelineEntry entry : incompletePartitionsTimeline.values()) {
- final List> entryOvershadowed = entry.partitionHolder.getOvershadowed();
- if (!entryOvershadowed.isEmpty()) {
- retVal.add(
+ final List> overshadowedEntries = entry.partitionHolder.getOvershadowed();
+ if (!overshadowedEntries.isEmpty()) {
+ overshadowedObjects.add(
new TimelineObjectHolder<>(
entry.trueInterval,
entry.version,
- new PartitionHolder<>(entryOvershadowed)
+ new PartitionHolder<>(overshadowedEntries)
)
);
}
}
- return retVal;
+ return overshadowedObjects;
}
finally {
lock.readLock().unlock();
}
}
+ private Map> computeOvershadowedPartitionsTimeline()
+ {
+ final Map> overshadowedPartitionsTimeline = new HashMap<>();
+ allTimelineEntries.forEach((Interval interval, TreeMap versionEntry) -> {
+ @SuppressWarnings("unchecked")
+ Map versionEntryCopy = (TreeMap) versionEntry.clone();
+ overshadowedPartitionsTimeline.put(interval, versionEntryCopy);
+ });
+
+ for (TimelineEntry entry : completePartitionsTimeline.values()) {
+ overshadowedPartitionsTimeline.computeIfPresent(
+ entry.getTrueInterval(),
+ (Interval interval, Map versionEntry) -> {
+ versionEntry.remove(entry.getVersion());
+ return versionEntry.isEmpty() ? null : versionEntry;
+ }
+ );
+ }
+
+ for (TimelineEntry entry : incompletePartitionsTimeline.values()) {
+ overshadowedPartitionsTimeline.computeIfPresent(
+ entry.getTrueInterval(),
+ (Interval interval, Map versionEntry) -> {
+ versionEntry.remove(entry.getVersion());
+ return versionEntry.isEmpty() ? null : versionEntry;
+ }
+ );
+ }
+ return overshadowedPartitionsTimeline;
+ }
+
public boolean isOvershadowed(Interval interval, VersionType version, ObjectType object)
{
lock.readLock().lock();
diff --git a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java
index d3166aeefcbb..fcc77ddcd852 100644
--- a/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java
+++ b/core/src/main/java/org/apache/druid/timeline/partition/OvershadowableManager.java
@@ -89,14 +89,19 @@ enum State
// (start partitionId, end partitionId) -> minorVersion -> atomicUpdateGroup
private final TreeMap>> standbyGroups;
- private final TreeMap>> visibleGroup;
+ /**
+ * The values in this map must always be {@link SingleEntryShort2ObjectSortedMap}, hence there is at most one visible
+ * group per range. The same type is used as in {@link #standbyGroups} and {@link #overshadowedGroups} to reuse helper
+ * functions across all {@link State}s.
+ */
+ private final TreeMap>> visibleGroupPerRange;
private final TreeMap>> overshadowedGroups;
OvershadowableManager()
{
this.knownPartitionChunks = new HashMap<>();
this.standbyGroups = new TreeMap<>();
- this.visibleGroup = new TreeMap<>();
+ this.visibleGroupPerRange = new TreeMap<>();
this.overshadowedGroups = new TreeMap<>();
}
@@ -104,7 +109,7 @@ enum State
{
this.knownPartitionChunks = new HashMap<>(other.knownPartitionChunks);
this.standbyGroups = new TreeMap<>(other.standbyGroups);
- this.visibleGroup = new TreeMap<>(other.visibleGroup);
+ this.visibleGroupPerRange = new TreeMap<>(other.visibleGroupPerRange);
this.overshadowedGroups = new TreeMap<>(other.overshadowedGroups);
}
@@ -124,7 +129,7 @@ private TreeMap>>
case STANDBY:
return standbyGroups;
case VISIBLE:
- return visibleGroup;
+ return visibleGroupPerRange;
case OVERSHADOWED:
return overshadowedGroups;
default:
@@ -670,7 +675,7 @@ boolean addChunk(PartitionChunk chunk)
final AtomicUpdateGroup newAtomicUpdateGroup = new AtomicUpdateGroup<>(chunk);
// Decide the initial state of the new atomicUpdateGroup
- final boolean overshadowed = visibleGroup
+ final boolean overshadowed = visibleGroupPerRange
.values()
.stream()
.flatMap(map -> map.values().stream())
@@ -786,7 +791,7 @@ private List> findLatestNonFullyAvailableAtomicUpdateGroups
}
final List> visibles = new ArrayList<>();
- for (Short2ObjectSortedMap> map : manager.visibleGroup.values()) {
+ for (Short2ObjectSortedMap> map : manager.visibleGroupPerRange.values()) {
visibles.addAll(map.values());
}
return visibles;
@@ -808,7 +813,7 @@ private List> findLatestFullyAvailableOvershadowedAtomicUpd
final OvershadowableManager manager = new OvershadowableManager<>(overshadowedGroups);
final List> visibles = new ArrayList<>();
- for (Short2ObjectSortedMap> map : manager.visibleGroup.values()) {
+ for (Short2ObjectSortedMap> map : manager.visibleGroupPerRange.values()) {
for (AtomicUpdateGroup atomicUpdateGroup : map.values()) {
if (!atomicUpdateGroup.isFull()) {
return Collections.emptyList();
@@ -886,12 +891,15 @@ PartitionChunk removeChunk(PartitionChunk partitionChunk)
public boolean isEmpty()
{
- return visibleGroup.isEmpty();
+ return visibleGroupPerRange.isEmpty();
}
public boolean isComplete()
{
- return visibleGroup.values().stream().allMatch(map -> Iterables.getOnlyElement(map.values()).isFull());
+ return visibleGroupPerRange
+ .values()
+ .stream()
+ .allMatch(map -> Iterables.getOnlyElement(map.values()).isFull());
}
@Nullable
@@ -916,7 +924,7 @@ PartitionChunk getChunk(int partitionId)
Stream> createVisibleChunksStream()
{
- return visibleGroup
+ return visibleGroupPerRange
.values()
.stream()
.flatMap((Short2ObjectSortedMap> map) -> map.values().stream())
@@ -959,14 +967,14 @@ public boolean equals(Object o)
OvershadowableManager> that = (OvershadowableManager>) o;
return Objects.equals(knownPartitionChunks, that.knownPartitionChunks) &&
Objects.equals(standbyGroups, that.standbyGroups) &&
- Objects.equals(visibleGroup, that.visibleGroup) &&
+ Objects.equals(visibleGroupPerRange, that.visibleGroupPerRange) &&
Objects.equals(overshadowedGroups, that.overshadowedGroups);
}
@Override
public int hashCode()
{
- return Objects.hash(knownPartitionChunks, standbyGroups, visibleGroup, overshadowedGroups);
+ return Objects.hash(knownPartitionChunks, standbyGroups, visibleGroupPerRange, overshadowedGroups);
}
@Override
@@ -975,7 +983,7 @@ public String toString()
return "OvershadowableManager{" +
"knownPartitionChunks=" + knownPartitionChunks +
", standbyGroups=" + standbyGroups +
- ", visibleGroup=" + visibleGroup +
+ ", visibleGroupPerRangePerVersion=" + visibleGroupPerRange +
", overshadowedGroups=" + overshadowedGroups +
'}';
}
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 c946bc88bf0a..c55e11369b90 100644
--- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
+++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java
@@ -648,7 +648,7 @@ public void testOverlapLargeUnderlyingWithSmallDayAlignedOverlays()
createExpected("2011-01-04/2011-01-05", "3", 3),
createExpected("2011-01-05/2011-01-06", "4", 4)
),
- timeline.lookup(Intervals.of("0000-01-01/3000-01-01"))
+ timeline.lookup(Intervals.ETERNITY)
);
}
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index dd0acc686968..ac4396602145 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -674,7 +674,7 @@ These Coordinator static configurations can be defined in the `coordinator/runti
|Property|Description|Default|
|--------|-----------|-------|
-|`druid.coordinator.period`|The run period for the Coordinator. The Coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
+|`druid.coordinator.period`|The run period for the Coordinator. The Coordinator operates by maintaining the current state of the world in memory and periodically looking at the set of "used" segments and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
|`druid.coordinator.period.indexingPeriod`|How often to send compact/merge/conversion tasks to the indexing service. It's recommended to be longer than `druid.manager.segments.pollDuration`|PT1800S (30 mins)|
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
|`druid.coordinator.load.timeout`|The timeout duration for when the Coordinator assigns a segment to a Historical process.|PT15M|
@@ -749,7 +749,7 @@ Issuing a GET request at the same URL will return the spec that is currently in
|Property|Description|Default|
|--------|-----------|-------|
-|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be active before it can start removing (marking unused) segments in metadata storage.|900000 (15 mins)|
+|`millisToWaitBeforeDeleting`|How long does the Coordinator need to be a leader before it can start marking overshadowed segments as unused in metadata storage.|900000 (15 mins)|
|`mergeBytesLimit`|The maximum total uncompressed size in bytes of segments to merge.|524288000L|
|`mergeSegmentsLimit`|The maximum number of segments that can be in a single [append task](../ingestion/tasks.md).|100|
|`maxSegmentsToMove`|The maximum number of segments that can be moved at any given time.|5|
@@ -757,9 +757,9 @@ Issuing a GET request at the same URL will return the spec that is currently in
|`replicationThrottleLimit`|The maximum number of segments that can be replicated at one time.|10|
|`balancerComputeThreads`|Thread pool size for computing moving cost of segments in segment balancing. Consider increasing this if you have a lot of segments and moving segments starts to get stuck.|1|
|`emitBalancingStats`|Boolean flag for whether or not we should emit balancing stats. This is an expensive operation.|false|
-|`killDataSourceWhitelist`|List of dataSources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
+|`killDataSourceWhitelist`|List of specific data sources for which kill tasks are sent if property `druid.coordinator.kill.on` is true. This can be a list of comma-separated data source names or a JSON array.|none|
|`killAllDataSources`|Send kill tasks for ALL dataSources if property `druid.coordinator.kill.on` is true. If this is set to true then `killDataSourceWhitelist` must not be specified or be empty list.|false|
-|`killPendingSegmentsSkipList`|List of dataSources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated dataSources or a JSON array.|none|
+|`killPendingSegmentsSkipList`|List of data sources for which pendingSegments are _NOT_ cleaned up if property `druid.coordinator.kill.pendingSegments.on` is true. This can be a list of comma-separated data sources or a JSON array.|none|
|`maxSegmentsInNodeLoadingQueue`|The maximum number of segments that could be queued for loading to any given server. This parameter could be used to speed up segments loading process, especially if there are "slow" nodes in the cluster (with low loading speed) or if too much segments scheduled to be replicated to some particular node (faster loading could be preferred to better segments distribution). Desired value depends on segments loading speed, acceptable replication time and number of nodes. Value 1000 could be a start point for a rather big cluster. Default value is 0 (loading queue is unbounded) |0|
|`decommissioningNodes`| 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 `decommissioningMaxPercentOfMaxSegmentsToMove`.|none|
|`decommissioningMaxPercentOfMaxSegmentsToMove`| The maximum number of segments that may be moved away from 'decommissioning' servers to non-decommissioning (that is, active) servers during one Coordinator run. This value is relative to the total maximum segment movements allowed during one run which is determined by `maxSegmentsToMove`. If `decommissioningMaxPercentOfMaxSegmentsToMove` is 0, segments will neither be moved from _or to_ 'decommissioning' servers, effectively putting them in a sort of "maintenance" mode that will not participate in balancing or assignment by load rules. Decommissioning can also become stalled if there are no available active servers to place the segments. By leveraging the maximum percent of decommissioning segment movements, an operator can prevent active servers from overload by prioritizing balancing, or decrease decommissioning time instead. The value should be between 0 and 100.|70|
@@ -993,8 +993,8 @@ Worker select strategies control how Druid assigns tasks to MiddleManagers.
###### Equal Distribution
-Tasks are assigned to the MiddleManager with the most available capacity at the time the task begins running. This is
-useful if you want work evenly distributed across your MiddleManagers.
+Tasks are assigned to the MiddleManager with the most free slots at the time the task begins running. This is useful if
+you want work evenly distributed across your MiddleManagers.
|Property|Description|Default|
|--------|-----------|-------|
@@ -1160,7 +1160,7 @@ Middle managers pass their configurations down to their child peons. The MiddleM
|`druid.indexer.runner.ports`|A JSON array of integers to specify ports that used for peon processes. If provided and non-empty, ports for peon processes will be chosen from these ports. And `druid.indexer.runner.startPort/druid.indexer.runner.endPort` will be completely ignored.|`[]`|
|`druid.worker.ip`|The IP of the worker.|localhost|
|`druid.worker.version`|Version identifier for the MiddleManager.|0|
-|`druid.worker.capacity`|Maximum number of tasks the MiddleManager can accept.|Number of available processors - 1|
+|`druid.worker.capacity`|Maximum number of tasks the MiddleManager can accept.|Number of CPUs on the machine - 1|
|`druid.worker.category`|A string to name the category that the MiddleManager node belongs to.|`__default_worker_category`|
#### Peon Processing
@@ -1599,8 +1599,9 @@ See [cache configuration](#cache-configuration) for how to configure cache setti
This section describes caching configuration that is common to Broker, Historical, and MiddleManager/Peon processes.
-Caching can optionally be enabled on the Broker, Historical, and MiddleManager/Peon processes. See [Broker](#broker-caching),
-[Historical](#historical-caching), and [Peon](#peon-caching) configuration options for how to enable it for different processes.
+Caching could optionally be enabled on the Broker, Historical, and MiddleManager/Peon processes. See
+[Broker](#broker-caching), [Historical](#historical-caching), and [Peon](#peon-caching) configuration options for how to
+enable it for different processes.
Druid uses a local in-memory cache by default, unless a different type of cache is specified.
Use the `druid.cache.type` configuration to set a different kind of cache.
diff --git a/docs/dependencies/metadata-storage.md b/docs/dependencies/metadata-storage.md
index 072d00d8e92a..51551a1df010 100644
--- a/docs/dependencies/metadata-storage.md
+++ b/docs/dependencies/metadata-storage.md
@@ -71,18 +71,15 @@ See [BasicDataSource Configuration](https://commons.apache.org/proper/commons-db
This is dictated by the `druid.metadata.storage.tables.segments` property.
-This table stores metadata about the segments that are available in the system.
-The table is polled by the [Coordinator](../design/coordinator.md) to
-determine the set of segments that should be available for querying in the
-system. The table has two main functional columns, the other columns are for
-indexing purposes.
-
-The `used` column is a boolean "tombstone". A 1 means that the segment should
-be "used" by the cluster (i.e., it should be loaded and available for requests).
-A 0 means that the segment should not be actively loaded into the cluster. We
-do this as a means of removing segments from the cluster without actually
-removing their metadata (which allows for simpler rolling back if that is ever
-an issue).
+This table stores metadata about the segments that should be available in the system. (This set of segments is called
+"used segments" elsewhere in the documentation and throughout the project.) The table is polled by the
+[Coordinator](../design/coordinator.md) to determine the set of segments that should be available for querying in the
+system. The table has two main functional columns, the other columns are for indexing purposes.
+
+Value 1 in the `used` column means that the segment should be "used" by the cluster (i.e., it should be loaded and
+available for requests). Value 0 means that the segment should not be loaded into the cluster. We do this as a means of
+unloading segments from the cluster without actually removing their metadata (which allows for simpler rolling back if
+that is ever an issue).
The `payload` column stores a JSON blob that has all of the metadata for the segment (some of the data stored in this payload is redundant with some of the columns in the table, that is intentional). This looks something like
diff --git a/docs/design/architecture.md b/docs/design/architecture.md
index a369e216db64..c801cc54154b 100644
--- a/docs/design/architecture.md
+++ b/docs/design/architecture.md
@@ -69,8 +69,8 @@ if every single data server is lost and re-provisioned.
For more details, please see the [Deep storage](../dependencies/deep-storage.md) page.
### Metadata storage
-The metadata storage holds various shared system metadata such as segment availability information and task information.
-In a clustered deployment, this is typically going to be a traditional RDBMS like PostgreSQL or MySQL. In a single-server
+The metadata storage holds various shared system metadata such as segment usage information and task information. In a
+clustered deployment, this is typically going to be a traditional RDBMS like PostgreSQL or MySQL. In a single-server
deployment, it is typically going to be a locally-stored Apache Derby database.
For more details, please see the [Metadata storage](../dependencies/metadata-storage.md) page.
diff --git a/docs/design/coordinator.md b/docs/design/coordinator.md
index fbf5455cd0cf..faba68bcb37d 100644
--- a/docs/design/coordinator.md
+++ b/docs/design/coordinator.md
@@ -33,11 +33,24 @@ For a list of API endpoints supported by the Coordinator, see [Coordinator API](
### Overview
-The Druid Coordinator process is primarily responsible for segment management and distribution. More specifically, the Druid Coordinator process communicates to Historical processes to load or drop segments based on configurations. The Druid Coordinator is responsible for loading new segments, dropping outdated segments, managing segment replication, and balancing segment load.
-
-The Druid Coordinator runs periodically and the time between each run is a configurable parameter. Each time the Druid Coordinator runs, it assesses the current state of the cluster before deciding on the appropriate actions to take. Similar to the Broker and Historical processes, the Druid Coordinator maintains a connection to a Zookeeper cluster for current cluster information. The Coordinator also maintains a connection to a database containing information about available segments and rules. Available segments are stored in a segment table and list all segments that should be loaded in the cluster. Rules are stored in a rule table and indicate how segments should be handled.
-
-Before any unassigned segments are serviced by Historical processes, the available Historical processes for each tier are first sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always assigned to the processes with least capacity to maintain a level of balance between processes. The Coordinator does not directly communicate with a historical process when assigning it a new segment; instead the Coordinator creates some temporary information about the new segment under load queue path of the historical process. Once this request is seen, the historical process will load the segment and begin servicing it.
+The Druid Coordinator process is primarily responsible for segment management and distribution. More specifically, the
+Druid Coordinator process communicates to Historical processes to load or drop segments based on configurations. The
+Druid Coordinator is responsible for loading new segments, dropping outdated segments, ensuring that segments are
+"replicated" (that is, loaded on multiple different Historical nodes) proper (configured) number of times, and moving
+("balancing") segments between Historical nodes to keep the latter evenly loaded.
+
+The Druid Coordinator runs its duties periodically and the time between each run is a configurable parameter. On each
+run, the Coordinator assesses the current state of the cluster before deciding on the appropriate actions to take.
+Similar to the Broker and Historical processes, the Druid Coordinator maintains a connection to a Zookeeper cluster for
+current cluster information. The Coordinator also maintains a connection to a database containing information about
+"used" segments (that is, the segments that *should* be loaded in the cluster) and the loading rules.
+
+Before any unassigned segments are serviced by Historical processes, the Historical processes for each tier are first
+sorted in terms of capacity, with least capacity servers having the highest priority. Unassigned segments are always
+assigned to the processes with least capacity to maintain a level of balance between processes. The Coordinator does not
+directly communicate with a historical process when assigning it a new segment; instead the Coordinator creates some
+temporary information about the new segment under load queue path of the historical process. Once this request is seen,
+the historical process will load the segment and begin servicing it.
### Running
@@ -51,7 +64,12 @@ Segments can be automatically loaded and dropped from the cluster based on a set
### Cleaning up segments
-Each run, the Druid coordinator compares the list of available database segments in the database with the current segments in the cluster. Segments that are not in the database but are still being served in the cluster are flagged and appended to a removal list. Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are also dropped.
+On each run, the Druid Coordinator compares the set of used segments in the database with the segments served by some
+Historical nodes in the cluster. Coordinator sends requests to Historical nodes to unload unused segments or segments
+that are removed from the database.
+
+Segments that are overshadowed (their versions are too old and their data has been replaced by newer segments) are
+marked as unused. During the next Coordinator's run, they will be unloaded from Historical nodes in the cluster.
### Segment availability
diff --git a/docs/operations/api-reference.md b/docs/operations/api-reference.md
index 4537b5f5f47d..db11cfe6c040 100644
--- a/docs/operations/api-reference.md
+++ b/docs/operations/api-reference.md
@@ -146,7 +146,8 @@ Returns a list of all segments for a datasource with the full segment metadata a
* `/druid/coordinator/v1/metadata/datasources/{dataSourceName}/segments/{segmentId}`
-Returns full segment metadata for a specific segment as stored in the metadata store.
+Returns full segment metadata for a specific segment as stored in the metadata store, if the segment is used. If the
+segment is unused, or is unknown, a 404 response is returned.
##### POST
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 5c0da46f0905..ac6bd095063f 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -203,17 +203,20 @@ These metrics are for the Druid Coordinator and are reset each time the Coordina
|`segment/loadQueue/failed`|Number of segments that failed to load.|server.|0|
|`segment/loadQueue/count`|Number of segments to load.|server.|Varies.|
|`segment/dropQueue/count`|Number of segments to drop.|server.|Varies.|
-|`segment/size`|Size in bytes of available segments.|dataSource.|Varies.|
-|`segment/count`|Number of available segments.|dataSource.|< max|
-|`segment/overShadowed/count`|Number of overShadowed segments.||Varies.|
-|`segment/unavailable/count`|Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries.|datasource.|0|
-|`segment/underReplicated/count`|Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries.|tier, datasource.|0|
+|`segment/size`|Total size of used segments in a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|Varies.|
+|`segment/count`|Number of used segments belonging to a data source. Emitted only for data sources to which at least one used segment belongs.|dataSource.|< max|
+|`segment/overShadowed/count`|Number of overshadowed segments.||Varies.|
+|`segment/unavailable/count`|Number of segments (not including replicas) left to load until segments that should be loaded in the cluster are available for queries.|dataSource.|0|
+|`segment/underReplicated/count`|Number of segments (including replicas) left to load until segments that should be loaded in the cluster are available for queries.|tier, dataSource.|0|
|`tier/historical/count`|Number of available historical nodes in each tier.|tier.|Varies.|
|`tier/replication/factor`|Configured maximum replication factor in each tier.|tier.|Varies.|
|`tier/required/capacity`|Total capacity in bytes required in each tier.|tier.|Varies.|
|`tier/total/capacity`|Total capacity in bytes available in each tier.|tier.|Varies.|
-If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](../configuration/index.html#dynamic-configuration), then [log entries](../configuration/logging.md) for class `org.apache.druid.server.coordinator.helper.DruidCoordinatorLogger` will have extra information on balancing decisions.
+If `emitBalancingStats` is set to `true` in the Coordinator [dynamic configuration](
+../configuration/index.html#dynamic-configuration), then [log entries](../configuration/logging.md) for class
+`org.apache.druid.server.coordinator.duty.EmitClusterStatsAndMetrics` will have extra information on balancing
+decisions.
## General Health
diff --git a/docs/operations/rule-configuration.md b/docs/operations/rule-configuration.md
index e66eef02be7f..4228994a30d3 100644
--- a/docs/operations/rule-configuration.md
+++ b/docs/operations/rule-configuration.md
@@ -26,9 +26,12 @@ title: "Retaining or automatically dropping data"
In Apache Druid, Coordinator processes use rules to determine what data should be loaded to or dropped from the cluster. Rules are used for data retention and query execution, and are set on the Coordinator console (http://coordinator_ip:port).
There are three types of rules, i.e., load rules, drop rules, and broadcast rules. Load rules indicate how segments should be assigned to different historical process tiers and how many replicas of a segment should exist in each tier.
-Drop rules indicate when segments should be dropped entirely from the cluster. Finally, broadcast rules indicate how segments of different data sources should be co-located in Historical processes.
+Drop rules indicate when segments should be dropped entirely from the cluster. Finally, broadcast rules indicate how segments of different datasources should be co-located in Historical processes.
-The Coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The Coordinator will cycle through all available segments and match each segment with the first rule that applies. Each segment may only match a single rule.
+The Coordinator loads a set of rules from the metadata storage. Rules may be specific to a certain datasource and/or a
+default set of rules can be configured. Rules are read in order and hence the ordering of rules is important. The
+Coordinator will cycle through all used segments and match each segment with the first rule that applies. Each segment
+may only match a single rule.
Note: It is recommended that the Coordinator console is used to configure rules. However, the Coordinator process does have HTTP endpoints to programmatically configure rules.
@@ -167,8 +170,8 @@ The interval of a segment will be compared against the specified period. The per
## Broadcast Rules
-Broadcast rules indicate how segments of different data sources should be co-located in Historical processes.
-Once a broadcast rule is configured for a data source, all segments of the data source are broadcasted to the servers holding _any segments_ of the co-located data sources.
+Broadcast rules indicate how segments of different datasources should be co-located in Historical processes.
+Once a broadcast rule is configured for a datasource, all segments of the datasource are broadcasted to the servers holding _any segments_ of the co-located datasources.
### Forever Broadcast Rule
@@ -182,7 +185,7 @@ Forever broadcast rules are of the form:
```
* `type` - this should always be "broadcastForever"
-* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
+* `colocatedDataSources` - A JSON List containing datasource names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
### Interval Broadcast Rule
@@ -197,7 +200,7 @@ Interval broadcast rules are of the form:
```
* `type` - this should always be "broadcastByInterval"
-* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
+* `colocatedDataSources` - A JSON List containing datasource names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
* `interval` - A JSON Object representing ISO-8601 Periods. Only the segments of the interval will be broadcasted.
### Period Broadcast Rule
@@ -214,21 +217,24 @@ Period broadcast rules are of the form:
```
* `type` - this should always be "broadcastByPeriod"
-* `colocatedDataSources` - A JSON List containing data source names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
+* `colocatedDataSources` - A JSON List containing datasource names to be co-located. `null` and empty list means broadcasting to every process in the cluster.
* `period` - A JSON Object representing ISO-8601 Periods
* `includeFuture` - A JSON Boolean indicating whether the load period should include the future. This property is optional, Default is true.
The interval of a segment will be compared against the specified period. The period is from some time in the past to the future or to the current time, which depends on `includeFuture` is true or false. The rule matches if the period *overlaps* the interval.
-> broadcast rules don't guarantee that segments of the data sources are always co-located because segments for the colocated data sources are not loaded together atomically.
-> If you want to always co-locate the segments of some data sources together, it is recommended to leave colocatedDataSources empty.
+> broadcast rules don't guarantee that segments of the datasources are always co-located because segments for the colocated datasources are not loaded together atomically.
+> If you want to always co-locate the segments of some datasources together, it is recommended to leave colocatedDataSources empty.
## Permanently deleting data
-Druid can fully drop data from the cluster, wipe the metadata store entry, and remove the data from deep storage for any segments that are
-marked as unused (segments dropped from the cluster via rules are always marked as unused). You can submit a [kill task](../ingestion/tasks.md) to the [Overlord](../design/overlord.md) to do this.
+Druid can fully drop data from the cluster, wipe the metadata store entry, and remove the data from deep storage for any
+segments that are marked as unused (segments dropped from the cluster via rules are always marked as unused). You can
+submit a [kill task](../ingestion/tasks.md) to the [Overlord](../design/overlord.md) to do this.
## Reloading dropped data
-Data that has been dropped from a Druid cluster cannot be reloaded using only rules. To reload dropped data in Druid, you must first set your retention period (i.e. changing the retention period from 1 month to 2 months), and
-then enable the datasource in the Druid Coordinator console, or through the Druid Coordinator endpoints.
+Data that has been dropped from a Druid cluster cannot be reloaded using only rules. To reload dropped data in Druid,
+you must first set your retention period (i.e. changing the retention period from 1 month to 2 months), and then mark as
+used all segments belonging to the datasource in the Druid Coordinator console, or through the Druid Coordinator
+endpoints.
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 bd43ac639a49..dd3db5008a97 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
@@ -39,7 +39,6 @@
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
-import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
@@ -48,7 +47,7 @@
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.EntryExistsException;
import org.apache.druid.metadata.MetadataSupervisorManager;
-import org.apache.druid.metadata.SQLMetadataSegmentManager;
+import org.apache.druid.metadata.SqlSegmentsMetadataManager;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Duration;
import org.joda.time.Interval;
@@ -64,18 +63,17 @@
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.TimeUnit;
-import java.util.function.IntSupplier;
public class MaterializedViewSupervisor implements Supervisor
{
private static final EmittingLogger log = new EmittingLogger(MaterializedViewSupervisor.class);
- private static final Interval ALL_INTERVAL = Intervals.of("0000-01-01/3000-01-01");
private static final int DEFAULT_MAX_TASK_COUNT = 1;
// there is a lag between derivatives and base dataSource, to prevent repeatedly building for some delay data.
- private static final long DEFAULT_MIN_DATA_LAG_MS = 24 * 3600 * 1000L;
+ private static final long DEFAULT_MIN_DATA_LAG_MS = TimeUnit.DAYS.toMillis(1);
+
private final MetadataSupervisorManager metadataSupervisorManager;
private final IndexerMetadataStorageCoordinator metadataStorageCoordinator;
- private final SQLMetadataSegmentManager segmentManager;
+ private final SqlSegmentsMetadataManager sqlSegmentsMetadataManager;
private final MaterializedViewSupervisorSpec spec;
private final TaskMaster taskMaster;
private final TaskStorage taskStorage;
@@ -102,7 +100,7 @@ public MaterializedViewSupervisor(
TaskMaster taskMaster,
TaskStorage taskStorage,
MetadataSupervisorManager metadataSupervisorManager,
- SQLMetadataSegmentManager segmentManager,
+ SqlSegmentsMetadataManager sqlSegmentsMetadataManager,
IndexerMetadataStorageCoordinator metadataStorageCoordinator,
MaterializedViewTaskConfig config,
MaterializedViewSupervisorSpec spec
@@ -111,7 +109,7 @@ public MaterializedViewSupervisor(
this.taskMaster = taskMaster;
this.taskStorage = taskStorage;
this.metadataStorageCoordinator = metadataStorageCoordinator;
- this.segmentManager = segmentManager;
+ this.sqlSegmentsMetadataManager = sqlSegmentsMetadataManager;
this.metadataSupervisorManager = metadataSupervisorManager;
this.config = config;
this.spec = spec;
@@ -132,7 +130,7 @@ public void start()
synchronized (stateLock) {
Preconditions.checkState(!started, "already started");
- DataSourceMetadata metadata = metadataStorageCoordinator.getDataSourceMetadata(dataSource);
+ DataSourceMetadata metadata = metadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
if (null == metadata) {
metadataStorageCoordinator.insertDataSourceMetadata(
dataSource,
@@ -164,7 +162,7 @@ public void run()
return;
}
- DataSourceMetadata metadata = metadataStorageCoordinator.getDataSourceMetadata(dataSource);
+ DataSourceMetadata metadata = metadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
if (metadata instanceof DerivativeDataSourceMetadata
&& spec.getBaseDataSource().equals(((DerivativeDataSourceMetadata) metadata).getBaseDataSource())
&& spec.getDimensions().equals(((DerivativeDataSourceMetadata) metadata).getDimensions())
@@ -259,7 +257,7 @@ public void reset(DataSourceMetadata dataSourceMetadata)
{
if (dataSourceMetadata == null) {
// if oldMetadata is different from spec, tasks and segments will be removed when reset.
- DataSourceMetadata oldMetadata = metadataStorageCoordinator.getDataSourceMetadata(dataSource);
+ DataSourceMetadata oldMetadata = metadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
if (oldMetadata instanceof DerivativeDataSourceMetadata) {
if (!((DerivativeDataSourceMetadata) oldMetadata).getBaseDataSource().equals(spec.getBaseDataSource()) ||
!((DerivativeDataSourceMetadata) oldMetadata).getDimensions().equals(spec.getDimensions()) ||
@@ -308,7 +306,8 @@ void checkSegmentsAndSubmitTasks()
//if the number of running tasks reach the max task count, supervisor won't submit new tasks.
return;
}
- Pair, Map>> toBuildIntervalAndBaseSegments = checkSegments();
+ Pair, Map>> toBuildIntervalAndBaseSegments =
+ checkSegments();
SortedMap sortedToBuildVersion = toBuildIntervalAndBaseSegments.lhs;
Map> baseSegments = toBuildIntervalAndBaseSegments.rhs;
missInterval = sortedToBuildVersion.keySet();
@@ -339,21 +338,14 @@ Pair, Map> getRunningTasks()
Pair, Map>> checkSegments()
{
// Pair version, interval -> list>
+ Collection derivativeSegmentsCollection =
+ metadataStorageCoordinator.retrieveAllUsedSegments(dataSource, Segments.ONLY_VISIBLE);
Pair, Map>> derivativeSegmentsSnapshot =
- getVersionAndBaseSegments(
- metadataStorageCoordinator.getUsedSegmentsForInterval(
- dataSource,
- ALL_INTERVAL,
- Segments.ONLY_VISIBLE
- )
- );
+ getVersionAndBaseSegments(derivativeSegmentsCollection);
// Pair max(created_date), interval -> list>
Pair, Map>> baseSegmentsSnapshot =
getMaxCreateDateAndBaseSegments(
- metadataStorageCoordinator.getUsedSegmentAndCreatedDateForInterval(
- spec.getBaseDataSource(),
- ALL_INTERVAL
- )
+ metadataStorageCoordinator.retrieveUsedSegmentsAndCreatedDates(spec.getBaseDataSource())
);
// baseSegments are used to create HadoopIndexTask
Map> baseSegments = baseSegmentsSnapshot.rhs;
@@ -375,31 +367,32 @@ Pair, Map>> checkSegment
final String versionOfBase = maxCreatedDate.get(entry.getKey());
final String versionOfDerivative = derivativeVersion.get(entry.getKey());
final int baseCount = baseSegments.get(entry.getKey()).size();
- final IntSupplier usedCountSupplier = () ->
- metadataStorageCoordinator
- .getUsedSegmentsForInterval(spec.getBaseDataSource(), entry.getKey(), Segments.ONLY_VISIBLE).size();
- if (versionOfBase.compareTo(versionOfDerivative) > 0 && baseCount == usedCountSupplier.getAsInt()) {
- toBuildInterval.put(entry.getKey(), versionOfBase);
+ if (versionOfBase.compareTo(versionOfDerivative) > 0) {
+ int usedCount = metadataStorageCoordinator
+ .retrieveUsedSegmentsForInterval(spec.getBaseDataSource(), entry.getKey(), Segments.ONLY_VISIBLE).size();
+ if (baseCount == usedCount) {
+ toBuildInterval.put(entry.getKey(), versionOfBase);
+ }
}
}
// if some intervals are in running tasks and the versions are the same, remove it from toBuildInterval
- // if some intervals are in running tasks, but the versions are different, stop the task.
- for (Map.Entry version : runningVersion.entrySet()) {
- final Interval interval = version.getKey();
- final String host = version.getValue();
- if (toBuildInterval.containsKey(interval) && toBuildInterval.get(interval).equals(host)) {
- toBuildInterval.remove(interval);
- } else if (toBuildInterval.containsKey(interval) && !toBuildInterval.get(interval).equals(host)) {
- if (taskMaster.getTaskQueue().isPresent()) {
- taskMaster.getTaskQueue().get().shutdown(runningTasks.get(interval).getId(), "version mismatch");
- runningTasks.remove(interval);
+ // if some intervals are in running tasks, but the versions are different, stop the task.
+ runningVersion.forEach((interval, version) -> {
+ if (toBuildInterval.containsKey(interval)) {
+ if (toBuildInterval.get(interval).equals(version)) {
+ toBuildInterval.remove(interval);
+ } else {
+ if (taskMaster.getTaskQueue().isPresent()) {
+ taskMaster.getTaskQueue().get().shutdown(runningTasks.get(interval).getId(), "version mismatch");
+ runningTasks.remove(interval);
+ }
}
}
- }
+ });
// drop derivative segments which interval equals the interval in toDeleteBaseSegments
for (Interval interval : toDropInterval.keySet()) {
for (DataSegment segment : derivativeSegments.get(interval)) {
- segmentManager.markSegmentAsUnused(segment.getId().toString());
+ sqlSegmentsMetadataManager.markSegmentAsUnused(segment.getId().toString());
}
}
// data of the latest interval will be built firstly.
@@ -441,8 +434,7 @@ private Pair, Map>> getVersion
for (DataSegment segment : snapshot) {
Interval interval = segment.getInterval();
versions.put(interval, segment.getVersion());
- segments.putIfAbsent(interval, new ArrayList<>());
- segments.get(interval).add(segment);
+ segments.computeIfAbsent(interval, i -> new ArrayList<>()).add(segment);
}
return new Pair<>(versions, segments);
}
@@ -465,15 +457,10 @@ private Pair, Map>> getMaxCrea
if (!hasEnoughLag(interval, maxAllowedToBuildInterval)) {
continue;
}
- maxCreatedDate.put(
- interval,
- DateTimes.max(
- DateTimes.of(createDate),
- DateTimes.of(maxCreatedDate.getOrDefault(interval, DateTimes.MIN.toString()))
- ).toString()
- );
- segments.putIfAbsent(interval, new ArrayList<>());
- segments.get(interval).add(segment);
+ maxCreatedDate.merge(interval, createDate, (date1, date2) -> {
+ return DateTimes.max(DateTimes.of(date1), DateTimes.of(date2)).toString();
+ });
+ segments.computeIfAbsent(interval, i -> new ArrayList<>()).add(segment);
}
return new Pair<>(maxCreatedDate, segments);
}
@@ -506,8 +493,8 @@ private void clearTasks()
private void clearSegments()
{
log.info("Clear all metadata of dataSource %s", dataSource);
- metadataStorageCoordinator.deletePendingSegments(dataSource, ALL_INTERVAL);
- segmentManager.markAsUnusedAllSegmentsInDataSource(dataSource);
+ metadataStorageCoordinator.deletePendingSegments(dataSource);
+ sqlSegmentsMetadataManager.markAsUnusedAllSegmentsInDataSource(dataSource);
metadataStorageCoordinator.deleteDataSourceMetadata(dataSource);
}
diff --git a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java
index 0e25841fe3c6..dd385130c761 100644
--- a/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java
+++ b/extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpec.java
@@ -43,7 +43,7 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.metadata.MetadataSupervisorManager;
-import org.apache.druid.metadata.SQLMetadataSegmentManager;
+import org.apache.druid.metadata.SqlSegmentsMetadataManager;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
@@ -77,7 +77,7 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
private final ObjectMapper objectMapper;
private final MetadataSupervisorManager metadataSupervisorManager;
private final IndexerMetadataStorageCoordinator metadataStorageCoordinator;
- private final SQLMetadataSegmentManager segmentManager;
+ private final SqlSegmentsMetadataManager sqlSegmentsMetadataManager;
private final TaskMaster taskMaster;
private final TaskStorage taskStorage;
private final MaterializedViewTaskConfig config;
@@ -101,7 +101,7 @@ public MaterializedViewSupervisorSpec(
@JacksonInject TaskMaster taskMaster,
@JacksonInject TaskStorage taskStorage,
@JacksonInject MetadataSupervisorManager metadataSupervisorManager,
- @JacksonInject SQLMetadataSegmentManager segmentManager,
+ @JacksonInject SqlSegmentsMetadataManager sqlSegmentsMetadataManager,
@JacksonInject IndexerMetadataStorageCoordinator metadataStorageCoordinator,
@JacksonInject MaterializedViewTaskConfig config,
@JacksonInject AuthorizerMapper authorizerMapper,
@@ -143,7 +143,7 @@ public MaterializedViewSupervisorSpec(
this.taskMaster = taskMaster;
this.taskStorage = taskStorage;
this.metadataSupervisorManager = metadataSupervisorManager;
- this.segmentManager = segmentManager;
+ this.sqlSegmentsMetadataManager = sqlSegmentsMetadataManager;
this.metadataStorageCoordinator = metadataStorageCoordinator;
this.authorizerMapper = authorizerMapper;
this.chatHandlerProvider = chatHandlerProvider;
@@ -353,7 +353,7 @@ public Supervisor createSupervisor()
taskMaster,
taskStorage,
metadataSupervisorManager,
- segmentManager,
+ sqlSegmentsMetadataManager,
metadataStorageCoordinator,
config,
this
@@ -384,7 +384,7 @@ public SupervisorSpec createSuspendedSpec()
taskMaster,
taskStorage,
metadataSupervisorManager,
- segmentManager,
+ sqlSegmentsMetadataManager,
metadataStorageCoordinator,
config,
authorizerMapper,
@@ -411,7 +411,7 @@ public SupervisorSpec createRunningSpec()
taskMaster,
taskStorage,
metadataSupervisorManager,
- segmentManager,
+ sqlSegmentsMetadataManager,
metadataStorageCoordinator,
config,
authorizerMapper,
diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java
index e19cb784ac7c..c82b8b8fae6a 100644
--- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java
+++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorSpecTest.java
@@ -32,7 +32,7 @@
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.metadata.MetadataSupervisorManager;
-import org.apache.druid.metadata.SQLMetadataSegmentManager;
+import org.apache.druid.metadata.SqlSegmentsMetadataManager;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
@@ -69,7 +69,7 @@ public void setup()
.addValue(ExprMacroTable.class.getName(), LookupEnabledTestExprMacroTable.INSTANCE)
.addValue(ObjectMapper.class, objectMapper)
.addValue(MetadataSupervisorManager.class, null)
- .addValue(SQLMetadataSegmentManager.class, null)
+ .addValue(SqlSegmentsMetadataManager.class, null)
.addValue(IndexerMetadataStorageCoordinator.class, null)
.addValue(MaterializedViewTaskConfig.class, new MaterializedViewTaskConfig())
.addValue(AuthorizerMapper.class, EasyMock.createMock(AuthorizerMapper.class))
diff --git a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
index 3ae7129f5830..ec276795ddf7 100644
--- a/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
+++ b/extensions-contrib/materialized-view-maintenance/src/test/java/org/apache/druid/indexing/materializedview/MaterializedViewSupervisorTest.java
@@ -41,7 +41,7 @@
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import org.apache.druid.metadata.MetadataSupervisorManager;
-import org.apache.druid.metadata.SQLMetadataSegmentManager;
+import org.apache.druid.metadata.SqlSegmentsMetadataManager;
import org.apache.druid.metadata.TestDerbyConnector;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
@@ -79,7 +79,7 @@ public class MaterializedViewSupervisorTest
private TaskMaster taskMaster;
private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
private MetadataSupervisorManager metadataSupervisorManager;
- private SQLMetadataSegmentManager sqlMetadataSegmentManager;
+ private SqlSegmentsMetadataManager sqlSegmentsMetadataManager;
private TaskQueue taskQueue;
private MaterializedViewSupervisor supervisor;
private String derivativeDatasourceName;
@@ -99,7 +99,7 @@ public void setUp()
derbyConnector
);
metadataSupervisorManager = EasyMock.createMock(MetadataSupervisorManager.class);
- sqlMetadataSegmentManager = EasyMock.createMock(SQLMetadataSegmentManager.class);
+ sqlSegmentsMetadataManager = EasyMock.createMock(SqlSegmentsMetadataManager.class);
taskQueue = EasyMock.createMock(TaskQueue.class);
taskQueue.start();
objectMapper.registerSubtypes(new NamedType(HashBasedNumberedShardSpec.class, "hashed"));
@@ -118,7 +118,7 @@ public void setUp()
taskMaster,
taskStorage,
metadataSupervisorManager,
- sqlMetadataSegmentManager,
+ sqlSegmentsMetadataManager,
indexerMetadataStorageCoordinator,
new MaterializedViewTaskConfig(),
EasyMock.createMock(AuthorizerMapper.class),
@@ -335,7 +335,7 @@ public void testSuspendedDoesntRun()
taskMaster,
taskStorage,
metadataSupervisorManager,
- sqlMetadataSegmentManager,
+ sqlSegmentsMetadataManager,
indexerMetadataStorageCoordinator,
new MaterializedViewTaskConfig(),
EasyMock.createMock(AuthorizerMapper.class),
@@ -344,10 +344,10 @@ public void testSuspendedDoesntRun()
);
MaterializedViewSupervisor supervisor = (MaterializedViewSupervisor) suspended.createSupervisor();
- // mock IndexerSQLMetadataStorageCoordinator to ensure that getDataSourceMetadata is not called
+ // mock IndexerSQLMetadataStorageCoordinator to ensure that retrieveDataSourceMetadata is not called
// which will be true if truly suspended, since this is the first operation of the 'run' method otherwise
IndexerSQLMetadataStorageCoordinator mock = EasyMock.createMock(IndexerSQLMetadataStorageCoordinator.class);
- EasyMock.expect(mock.getDataSourceMetadata(suspended.getDataSourceName()))
+ EasyMock.expect(mock.retrieveDataSourceMetadata(suspended.getDataSourceName()))
.andAnswer(() -> {
Assert.fail();
return null;
diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java
index 1653539f3e36..a3c03a6245c6 100644
--- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java
+++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DataSourceOptimizer.java
@@ -94,10 +94,9 @@ public List optimize(Query query)
}
lock.readLock().lock();
try {
- totalCount.putIfAbsent(datasourceName, new AtomicLong(0));
+ totalCount.computeIfAbsent(datasourceName, dsName -> new AtomicLong(0)).incrementAndGet();
hitCount.putIfAbsent(datasourceName, new AtomicLong(0));
- costTime.putIfAbsent(datasourceName, new AtomicLong(0));
- totalCount.get(datasourceName).incrementAndGet();
+ AtomicLong costTimeOfDataSource = costTime.computeIfAbsent(datasourceName, dsName -> new AtomicLong(0));
// get all fields which the query required
Set requiredFields = MaterializedViewUtils.getRequiredFields(query);
@@ -111,10 +110,11 @@ public List optimize(Query query)
}
// if no derivatives contains all required dimensions, this materialized view selection failed.
if (derivativesWithRequiredFields.isEmpty()) {
- missFields.putIfAbsent(datasourceName, new ConcurrentHashMap<>());
- missFields.get(datasourceName).putIfAbsent(requiredFields, new AtomicLong(0));
- missFields.get(datasourceName).get(requiredFields).incrementAndGet();
- costTime.get(datasourceName).addAndGet(System.currentTimeMillis() - start);
+ missFields
+ .computeIfAbsent(datasourceName, dsName -> new ConcurrentHashMap<>())
+ .computeIfAbsent(requiredFields, rf -> new AtomicLong(0))
+ .incrementAndGet();
+ costTimeOfDataSource.addAndGet(System.currentTimeMillis() - start);
return Collections.singletonList(query);
}
diff --git a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DerivativeDataSourceManager.java b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DerivativeDataSourceManager.java
index 199067bb84b1..3cff0da0075a 100644
--- a/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DerivativeDataSourceManager.java
+++ b/extensions-contrib/materialized-view-selection/src/main/java/org/apache/druid/query/materializedview/DerivativeDataSourceManager.java
@@ -193,8 +193,7 @@ private void updateDerivatives()
ConcurrentHashMap> newDerivatives = new ConcurrentHashMap<>();
for (DerivativeDataSource derivative : derivativeDataSources) {
- newDerivatives.putIfAbsent(derivative.getBaseDataSource(), new TreeSet<>());
- newDerivatives.get(derivative.getBaseDataSource()).add(derivative);
+ newDerivatives.computeIfAbsent(derivative.getBaseDataSource(), ds -> new TreeSet<>()).add(derivative);
}
ConcurrentHashMap> current;
do {
diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityAuthenticationException.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityAuthenticationException.java
index 16b8a782f2bf..41b0c99a10a4 100644
--- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityAuthenticationException.java
+++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/BasicSecurityAuthenticationException.java
@@ -26,6 +26,7 @@
*/
public class BasicSecurityAuthenticationException extends IllegalArgumentException
{
+
public BasicSecurityAuthenticationException(String formatText, Object... arguments)
{
super(StringUtils.nonStrictFormat(formatText, arguments));
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
index 43253b067e14..d4468bd39e03 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/KafkaIndexTaskTest.java
@@ -364,7 +364,7 @@ public void testRunAfterDataInserted() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -411,7 +411,7 @@ public void testRunAfterDataInsertedWithLegacyParser() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -462,7 +462,7 @@ public void testRunBeforeDataInserted() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -608,7 +608,7 @@ public void testIncrementalHandOff() throws Exception
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -742,14 +742,14 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
Assert.assertEquals(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -834,10 +834,15 @@ public void testTimeBasedIncrementalHandOff() throws Exception
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
+ DataSourceMetadata newDataSchemaMetadata()
+ {
+ return metadataStorageCoordinator.retrieveDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource());
+ }
+
@Test(timeout = 60_000L)
public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception
{
@@ -988,7 +993,7 @@ public void testRunWithMinimumMessageTime() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1040,7 +1045,7 @@ public void testRunWithMaximumMessageTime() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1094,7 +1099,7 @@ public void testRunWithTransformSpec() throws Exception
assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
// Check segments in deep storage
@@ -1182,7 +1187,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1233,7 +1238,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1277,7 +1282,7 @@ public void testReportParseExceptions() throws Exception
// Check published metadata
Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
}
@Test(timeout = 60_000L)
@@ -1327,7 +1332,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
@@ -1400,7 +1405,7 @@ public void testMultipleParseExceptionsFailure() throws Exception
// Check published metadata
Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
@@ -1488,7 +1493,7 @@ public void testRunReplicas() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1556,7 +1561,7 @@ public void testRunConflicting() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1605,7 +1610,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
SegmentDescriptorAndExpectedDim1Values desc1 = sdd("2010/P1D", 0, ImmutableList.of("c"));
SegmentDescriptorAndExpectedDim1Values desc2 = sdd("2011/P1D", 0, ImmutableList.of("d", "e"));
assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
// Run second task
final ListenableFuture future2 = runTask(task2);
@@ -1623,7 +1628,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e"));
SegmentDescriptorAndExpectedDim1Values desc4 = sdd("2013/P1D", 0, ImmutableList.of("f"));
assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
}
@Test(timeout = 60_000L)
@@ -1670,7 +1675,7 @@ public void testRunOneTaskTwoPartitions() throws Exception
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 2L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1739,7 +1744,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1837,7 +1842,7 @@ public void testRestore() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1953,7 +1958,7 @@ public void testRestoreAfterPersistingSequences() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2043,7 +2048,7 @@ public void testRunWithPauseAndResume() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2169,7 +2174,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2314,7 +2319,7 @@ public void testRunTransactionModeRollback() throws Exception
);
Assert.assertEquals(
new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
diff --git a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
index a50356f3e54b..0b29b65a628b 100644
--- a/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
+++ b/extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java
@@ -300,7 +300,7 @@ public void testNoInitialState() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -355,7 +355,7 @@ public void testSkipOffsetGaps() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -379,7 +379,7 @@ public void testMultiTask() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -434,7 +434,7 @@ public void testReplicas() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -489,7 +489,7 @@ public void testLateMessageRejectionPeriod() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -528,7 +528,7 @@ public void testEarlyMessageRejectionPeriod() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -570,7 +570,7 @@ public void testLatestOffset() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -610,7 +610,7 @@ public void testPartitionIdsUpdates() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -636,7 +636,7 @@ public void testAlwaysUsesEarliestOffsetForNewlyDiscoveredPartitions() throws Ex
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -708,7 +708,7 @@ public void testDatasourceMetadata() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 20L, 2, 30L), ImmutableSet.of())
)
@@ -747,7 +747,7 @@ public void testBadMetadataOffsets() throws Exception
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
// for simplicity in testing the offset availability check, we use negative stored offsets in metadata here,
// because the stream's earliest offset is 0, although that would not happen in real usage.
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(
topic,
@@ -797,7 +797,7 @@ public void testDontKillTasksWithMismatchedType() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -897,7 +897,7 @@ public void testKillBadPartitionAssignment() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -945,7 +945,7 @@ public void testRequeueTaskWhenFailed() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1048,7 +1048,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception
EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING));
EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true);
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1138,7 +1138,7 @@ public void testQueueNextTasksOnSuccess() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1235,7 +1235,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1349,7 +1349,7 @@ public void testDiscoverExistingPublishingTask() throws Exception
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of(task)).anyTimes();
EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1465,7 +1465,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation()
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of(task)).anyTimes();
EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1598,7 +1598,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1672,9 +1672,9 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception
Capture captured = Capture.newInstance(CaptureType.ALL);
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
- EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
+ EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.emptyList()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1733,7 +1733,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1819,7 +1819,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -1990,7 +1990,7 @@ public void testStopGracefully() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2094,7 +2094,7 @@ public void testResetDataSourceMetadata() throws Exception
new SeekableStreamStartSequenceNumbers<>(topic, ImmutableMap.of(0, 1000L), ImmutableSet.of()));
EasyMock.reset(indexerMetadataStorageCoordinator);
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
.andReturn(kafkaDataSourceMetadata);
EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(
EasyMock.capture(captureDataSource),
@@ -2141,7 +2141,7 @@ public void testResetNoDataSourceMetadata()
EasyMock.reset(indexerMetadataStorageCoordinator);
// no DataSourceMetadata in metadata store
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null);
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(null);
EasyMock.replay(indexerMetadataStorageCoordinator);
supervisor.resetInternal(resetMetadata);
@@ -2163,7 +2163,7 @@ public void testGetOffsetFromStorageForPartitionWithResetOffsetAutomatically() t
// unknown DataSourceMetadata in metadata store
// for simplicity in testing the offset availability check, we use negative stored offsets in metadata here,
// because the stream's earliest offset is 0, although that would not happen in real usage.
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
.andReturn(
new KafkaDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(1, -100L, 2, 200L))
@@ -2257,7 +2257,7 @@ public void testResetRunningTasks() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2357,7 +2357,7 @@ public void testNoDataIngestionTasks()
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2470,7 +2470,7 @@ public void testCheckpointForInactiveTaskGroup()
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
EasyMock.expect(
- indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
+ indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
).anyTimes();
EasyMock.expect(taskClient.getStatusAsync("id1")).andReturn(Futures.immediateFuture(Status.READING));
EasyMock.expect(taskClient.getStatusAsync("id2")).andReturn(Futures.immediateFuture(Status.READING));
@@ -2579,7 +2579,7 @@ public void testCheckpointForUnknownTaskGroup()
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
EasyMock.expect(
- indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
+ indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(new KafkaDataSourceMetadata(null)
).anyTimes();
replayAll();
@@ -2622,7 +2622,7 @@ public void testSuspendedNoRunningTasks() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2711,7 +2711,7 @@ public void testSuspendedRunningTasks() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2796,7 +2796,7 @@ public void testFailedInitializationAndRecovery() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2822,7 +2822,7 @@ public void testFailedInitializationAndRecovery() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -2979,7 +2979,7 @@ public void testDoNotKillCompatibleTasks()
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
@@ -3048,7 +3048,7 @@ public void testKillIncompatibleTasks()
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KafkaDataSourceMetadata(
null
)
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
index aef7accc7099..7867522a54eb 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/KinesisIndexTaskTest.java
@@ -373,7 +373,7 @@ public void testRunAfterDataInserted() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -440,10 +440,15 @@ public void testRunAfterDataInsertedWithLegacyParser() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
+ DataSourceMetadata newDataSchemaMetadata()
+ {
+ return metadataStorageCoordinator.retrieveDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource());
+ }
+
@Test(timeout = 120_000L)
public void testRunBeforeDataInserted() throws Exception
{
@@ -510,7 +515,7 @@ public void testRunBeforeDataInserted() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -622,7 +627,7 @@ public void testIncrementalHandOff() throws Exception
ImmutableMap.of(SHARD_ID1, "9", SHARD_ID0, "1")
)
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -757,7 +762,7 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception
);
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "10"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -827,7 +832,7 @@ public void testRunWithMinimumMessageTime() throws Exception
);
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -899,7 +904,7 @@ public void testRunWithMaximumMessageTime() throws Exception
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -972,7 +977,7 @@ public void testRunWithTransformSpec() throws Exception
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
// Check segments in deep storage
@@ -1103,7 +1108,7 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1173,7 +1178,7 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1236,7 +1241,7 @@ public void testReportParseExceptions() throws Exception
// Check published metadata
Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
}
@@ -1307,7 +1312,7 @@ public void testMultipleParseExceptionsSuccess() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "12"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
@@ -1399,7 +1404,7 @@ public void testMultipleParseExceptionsFailure() throws Exception
// Check published metadata
Assert.assertEquals(ImmutableList.of(), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
@@ -1513,7 +1518,7 @@ public void testRunReplicas() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1605,7 +1610,7 @@ public void testRunConflicting() throws Exception
);
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1678,7 +1683,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
SegmentDescriptorAndExpectedDim1Values desc1 = sdd("2010/P1D", 0, ImmutableList.of("c"));
SegmentDescriptorAndExpectedDim1Values desc2 = sdd("2011/P1D", 0, ImmutableList.of("d", "e"));
assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
// Run second task
final ListenableFuture future2 = runTask(task2);
@@ -1698,7 +1703,7 @@ public void testRunConflictingWithoutTransactions() throws Exception
SegmentDescriptorAndExpectedDim1Values desc3 = sdd("2011/P1D", 1, ImmutableList.of("d", "e"));
SegmentDescriptorAndExpectedDim1Values desc4 = sdd("2013/P1D", 0, ImmutableList.of("f"));
assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors());
- Assert.assertNull(metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource()));
+ Assert.assertNull(newDataSchemaMetadata());
}
@@ -1774,7 +1779,7 @@ public void testRunOneTaskTwoPartitions() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1870,7 +1875,7 @@ public void testRunTwoTasksTwoPartitions() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -1995,7 +2000,7 @@ public void testRestore() throws Exception
Assert.assertEquals(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2144,7 +2149,7 @@ public void testRestoreAfterPersistingSequences() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "6"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2248,7 +2253,7 @@ public void testRunWithPauseAndResume() throws Exception
STREAM,
ImmutableMap.of(SHARD_ID1, currentOffsets.get(SHARD_ID1))
)),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2327,7 +2332,7 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception
);
Assert.assertEquals(
new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2484,7 +2489,7 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9"))
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
@@ -2640,7 +2645,7 @@ public void testEndOfShard() throws Exception
ImmutableMap.of(SHARD_ID1, KinesisSequenceNumber.END_OF_SHARD_MARKER)
)
),
- metadataStorageCoordinator.getDataSourceMetadata(NEW_DATA_SCHEMA.getDataSource())
+ newDataSchemaMetadata()
);
}
diff --git a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
index 5ce8f3659daf..ad267e8a1eaa 100644
--- a/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
+++ b/extensions-core/kinesis-indexing-service/src/test/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorTest.java
@@ -235,7 +235,7 @@ public void testNoInitialState() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -302,9 +302,10 @@ public void testMultiTask() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
- new KinesisDataSourceMetadata(null)
- ).anyTimes();
+ EasyMock
+ .expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
+ .andReturn(new KinesisDataSourceMetadata(null))
+ .anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true).times(2);
replayAll();
@@ -360,7 +361,7 @@ public void testReplicas() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -437,7 +438,7 @@ public void testLateMessageRejectionPeriod() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -489,7 +490,7 @@ public void testEarlyMessageRejectionPeriod() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -548,7 +549,7 @@ public void testDatasourceMetadata() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(
STREAM,
@@ -601,7 +602,7 @@ public void testBadMetadataOffsets() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.absent()).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamStartSequenceNumbers<>(
STREAM,
@@ -665,7 +666,7 @@ public void testDontKillTasksWithMismatchedType() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -767,7 +768,7 @@ public void testKillBadPartitionAssignment() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -831,7 +832,7 @@ public void testRequeueTaskWhenFailed() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -968,7 +969,7 @@ public void testRequeueAdoptedTaskWhenFailed() throws Exception
.andReturn(Futures.immediateFuture(SeekableStreamIndexTaskRunner.Status.READING));
EasyMock.expect(taskClient.getStartTimeAsync("id1")).andReturn(Futures.immediateFuture(now)).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(captured))).andReturn(true);
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1079,7 +1080,7 @@ public void testQueueNextTasksOnSuccess() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1199,7 +1200,7 @@ public void testBeginPublishAndQueueNextTasks() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(null)
).anyTimes();
EasyMock.expect(taskQueue.add(EasyMock.capture(firstTasks))).andReturn(true).times(4);
@@ -1355,7 +1356,7 @@ public void testDiscoverExistingPublishingTask() throws Exception
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of(task)).anyTimes();
EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1512,7 +1513,7 @@ public void testDiscoverExistingPublishingTaskWithDifferentPartitionAllocation()
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of(task)).anyTimes();
EasyMock.expect(taskStorage.getStatus("id1")).andReturn(Optional.of(TaskStatus.running("id1"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(task)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1683,7 +1684,7 @@ public void testDiscoverExistingPublishingAndReadingTask() throws Exception
EasyMock.expect(taskStorage.getStatus("id2")).andReturn(Optional.of(TaskStatus.running("id2"))).anyTimes();
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1808,7 +1809,7 @@ public void testKillUnresponsiveTasksWhileGettingStartTime() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1890,7 +1891,7 @@ public void testKillUnresponsiveTasksWhilePausing() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -1998,7 +1999,7 @@ public void testKillUnresponsiveTasksWhileSettingEndOffsets() throws Exception
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskRunner.getRunningTasks()).andReturn(Collections.EMPTY_LIST).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -2226,7 +2227,7 @@ public void testStopGracefully() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -2382,7 +2383,7 @@ public void testResetDataSourceMetadata() throws Exception
);
EasyMock.reset(indexerMetadataStorageCoordinator);
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
.andReturn(kinesisDataSourceMetadata);
EasyMock.expect(indexerMetadataStorageCoordinator.resetDataSourceMetadata(
EasyMock.capture(captureDataSource),
@@ -2432,7 +2433,7 @@ public void testResetNoDataSourceMetadata()
EasyMock.reset(indexerMetadataStorageCoordinator);
// no DataSourceMetadata in metadata store
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(null);
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(null);
EasyMock.replay(indexerMetadataStorageCoordinator);
supervisor.resetInternal(resetMetadata);
@@ -2467,46 +2468,56 @@ public void testGetOffsetFromStorageForPartitionWithResetOffsetAutomatically() t
EasyMock.reset(indexerMetadataStorageCoordinator);
// unknown DataSourceMetadata in metadata store
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
- .andReturn(
- new KinesisDataSourceMetadata(
- new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "100", SHARD_ID2, "200"))
- )
- ).times(2);
+ EasyMock
+ .expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
+ .andReturn(
+ new KinesisDataSourceMetadata(
+ new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "100", SHARD_ID2, "200"))
+ )
+ )
+ .times(2);
// Since shard 2 was in metadata before but is not in the list of shards returned by the record supplier,
// it gets deleted from metadata (it is an expired shard)
- EasyMock.expect(
- indexerMetadataStorageCoordinator.resetDataSourceMetadata(
- DATASOURCE,
- new KinesisDataSourceMetadata(
- new SeekableStreamEndSequenceNumbers<>(
- STREAM,
- ImmutableMap.of(SHARD_ID1, "100", SHARD_ID2, KinesisSequenceNumber.EXPIRED_MARKER)
+ EasyMock
+ .expect(
+ indexerMetadataStorageCoordinator.resetDataSourceMetadata(
+ DATASOURCE,
+ new KinesisDataSourceMetadata(
+ new SeekableStreamEndSequenceNumbers<>(
+ STREAM,
+ ImmutableMap.of(SHARD_ID1, "100", SHARD_ID2, KinesisSequenceNumber.EXPIRED_MARKER)
+ )
)
)
)
- ).andReturn(true).times(1);
+ .andReturn(true)
+ .times(1);
// getOffsetFromStorageForPartition() throws an exception when the offsets are automatically reset.
// Since getOffsetFromStorageForPartition() is called per partition, all partitions can't be reset at the same time.
// Instead, subsequent partitions will be reset in the following supervisor runs.
- EasyMock.expect(
- indexerMetadataStorageCoordinator.resetDataSourceMetadata(
- DATASOURCE,
- new KinesisDataSourceMetadata(
- // Only one partition is reset in a single supervisor run.
- new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of())
+ EasyMock
+ .expect(
+ indexerMetadataStorageCoordinator.resetDataSourceMetadata(
+ DATASOURCE,
+ new KinesisDataSourceMetadata(
+ // Only one partition is reset in a single supervisor run.
+ new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of())
+ )
)
)
- ).andReturn(true).times(1);
+ .andReturn(true)
+ .times(1);
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE))
- .andReturn(
- new KinesisDataSourceMetadata(
- new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "100"))
- )
- ).times(2);
+ EasyMock
+ .expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE))
+ .andReturn(
+ new KinesisDataSourceMetadata(
+ new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "100"))
+ )
+ )
+ .times(2);
replayAll();
@@ -2619,7 +2630,7 @@ public void testResetRunningTasks() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -2773,7 +2784,7 @@ public void testNoDataIngestionTasks() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -2930,7 +2941,7 @@ public void testCheckpointForInactiveTaskGroup() throws InterruptedException
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
EasyMock.expect(
- indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata(
+ indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata(
null)
).anyTimes();
EasyMock.expect(taskClient.getStatusAsync("id1"))
@@ -3082,7 +3093,7 @@ public void testCheckpointForUnknownTaskGroup()
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
EasyMock.expect(
- indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata(
+ indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(new KinesisDataSourceMetadata(
null)
).anyTimes();
@@ -3127,7 +3138,7 @@ public void testSuspendedNoRunningTasks() throws Exception
EasyMock.expect(taskMaster.getTaskQueue()).andReturn(Optional.of(taskQueue)).anyTimes();
EasyMock.expect(taskMaster.getTaskRunner()).andReturn(Optional.of(taskRunner)).anyTimes();
EasyMock.expect(taskStorage.getActiveTasksByDatasource(DATASOURCE)).andReturn(ImmutableList.of()).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -3253,7 +3264,7 @@ public void testSuspendedRunningTasks() throws Exception
EasyMock.expect(taskStorage.getTask("id1")).andReturn(Optional.of(id1)).anyTimes();
EasyMock.expect(taskStorage.getTask("id2")).andReturn(Optional.of(id2)).anyTimes();
EasyMock.expect(taskStorage.getTask("id3")).andReturn(Optional.of(id3)).anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -3459,7 +3470,7 @@ public void testDoNotKillCompatibleTasks() throws InterruptedException, EntryExi
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -3555,7 +3566,7 @@ public void testKillIncompatibleTasks() throws InterruptedException, EntryExists
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -3769,7 +3780,7 @@ private List testShardSplitPhaseOne() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -3837,7 +3848,7 @@ private List testShardSplitPhaseTwo(List phaseOneTasks) throws Excep
EasyMock.reset(supervisorRecordSupplier);
// first task ran, its shard 0 has reached EOS
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers(
STREAM,
@@ -3994,7 +4005,7 @@ private void testShardSplitPhaseThree(List phaseTwoTasks) throws Exception
EasyMock.reset(supervisorRecordSupplier);
// second set of tasks ran, shard 0 has expired, but shard 1 and 2 have data
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers(
STREAM,
@@ -4215,7 +4226,7 @@ private List testShardMergePhaseOne() throws Exception
EasyMock.expect(taskClient.getStartTimeAsync(EasyMock.anyString()))
.andReturn(Futures.immediateFuture(DateTimes.nowUtc()))
.anyTimes();
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
null
)
@@ -4291,7 +4302,7 @@ private List testShardMergePhaseTwo(List phaseOneTasks) throws Excep
EasyMock.reset(supervisorRecordSupplier);
// first tasks ran, both shard 0 and shard 1 have reached EOS, merged into shard 2
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers(
STREAM,
@@ -4432,7 +4443,7 @@ private void testShardMergePhaseThree(List phaseTwoTasks) throws Exception
EasyMock.reset(supervisorRecordSupplier);
// second set of tasks ran, shard 0 has expired, but shard 1 and 2 have data
- EasyMock.expect(indexerMetadataStorageCoordinator.getDataSourceMetadata(DATASOURCE)).andReturn(
+ EasyMock.expect(indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(DATASOURCE)).andReturn(
new KinesisDataSourceMetadata(
new SeekableStreamEndSequenceNumbers(
STREAM,
diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java
index 66a61db70158..e3695ab1cdfc 100644
--- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java
+++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentMoverTest.java
@@ -85,7 +85,10 @@ public void testMove() throws Exception
);
Map targetLoadSpec = movedSegment.getLoadSpec();
- Assert.assertEquals("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip", MapUtils.getString(targetLoadSpec, "key"));
+ Assert.assertEquals(
+ "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
+ MapUtils.getString(targetLoadSpec, "key")
+ );
Assert.assertEquals("archive", MapUtils.getString(targetLoadSpec, "bucket"));
Assert.assertTrue(mockS3Client.didMove());
}
@@ -108,7 +111,10 @@ public void testMoveNoop() throws Exception
Map targetLoadSpec = movedSegment.getLoadSpec();
- Assert.assertEquals("targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip", MapUtils.getString(targetLoadSpec, "key"));
+ Assert.assertEquals(
+ "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
+ MapUtils.getString(targetLoadSpec, "key")
+ );
Assert.assertEquals("archive", MapUtils.getString(targetLoadSpec, "bucket"));
Assert.assertFalse(mockS3Client.didMove());
}
@@ -261,8 +267,7 @@ public PutObjectResult putObject(String bucketName, String key)
@Override
public PutObjectResult putObject(String bucketName, String key, File file)
{
- storage.putIfAbsent(bucketName, new HashSet<>());
- storage.get(bucketName).add(key);
+ storage.computeIfAbsent(bucketName, bName -> new HashSet<>()).add(key);
return new PutObjectResult();
}
}
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopIngestionSpec.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopIngestionSpec.java
index 7d3a04999aba..2aa91e24578b 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopIngestionSpec.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/HadoopIngestionSpec.java
@@ -139,7 +139,7 @@ public HadoopIngestionSpec withTuningConfig(HadoopTuningConfig config)
);
}
- public static HadoopIngestionSpec updateSegmentListIfDatasourcePathSpecIsUsed(
+ public static void updateSegmentListIfDatasourcePathSpecIsUsed(
HadoopIngestionSpec spec,
ObjectMapper jsonMapper,
UsedSegmentsRetriever segmentsRetriever
@@ -173,7 +173,7 @@ public static HadoopIngestionSpec updateSegmentListIfDatasourcePathSpecIsUsed(
DatasourceIngestionSpec.class
);
- Collection usedVisibleSegments = segmentsRetriever.getUsedSegmentsForIntervals(
+ Collection usedVisibleSegments = segmentsRetriever.retrieveUsedSegmentsForIntervals(
ingestionSpecObj.getDataSource(),
ingestionSpecObj.getIntervals(),
Segments.ONLY_VISIBLE
@@ -213,8 +213,6 @@ public static HadoopIngestionSpec updateSegmentListIfDatasourcePathSpecIsUsed(
datasourcePathSpec.put(segments, windowedSegments);
}
}
-
- return spec;
}
}
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java
index b4fe44f7bbcf..e1afb4cb6b62 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java
@@ -33,7 +33,7 @@
*/
public class MetadataStoreBasedUsedSegmentsRetriever implements UsedSegmentsRetriever
{
- private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
+ private final IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator;
@Inject
public MetadataStoreBasedUsedSegmentsRetriever(IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator)
@@ -45,12 +45,12 @@ public MetadataStoreBasedUsedSegmentsRetriever(IndexerMetadataStorageCoordinator
}
@Override
- public Collection getUsedSegmentsForIntervals(
+ public Collection retrieveUsedSegmentsForIntervals(
String dataSource,
List intervals,
Segments visibility
)
{
- return indexerMetadataStorageCoordinator.getUsedSegmentsForIntervals(dataSource, intervals, visibility);
+ return indexerMetadataStorageCoordinator.retrieveUsedSegmentsForIntervals(dataSource, intervals, visibility);
}
}
diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentsRetriever.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentsRetriever.java
index 428569f709c4..e1370e613563 100644
--- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentsRetriever.java
+++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentsRetriever.java
@@ -32,7 +32,8 @@
public interface UsedSegmentsRetriever
{
/**
- * Get all segments which may include any data in the interval and are marked as used.
+ * Retrieve (potentially, from a remote node) all segments which may include any data in the interval and are marked
+ * as used.
*
* The order of segments within the returned collection is unspecified, but each segment is guaranteed to appear in
* the collection only once.
@@ -48,9 +49,12 @@ public interface UsedSegmentsRetriever
* requested interval.
*
* @implNote This method doesn't return a {@link java.util.Set} because it's implemented via {@link
- * org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#getUsedSegmentsForIntervals} and which returns
- * a collection. Producing a {@link java.util.Set} would require an unnecessary copy of segments collection.
+ * org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#retrieveUsedSegmentsForIntervals} and which
+ * returns a collection. Producing a {@link java.util.Set} would require an unnecessary copy of segments collection.
*/
- Collection getUsedSegmentsForIntervals(String dataSource, List intervals, Segments visibility)
- throws IOException;
+ Collection retrieveUsedSegmentsForIntervals(
+ String dataSource,
+ List intervals,
+ Segments visibility
+ ) throws IOException;
}
diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java
index dfe667e83880..6402721e73c6 100644
--- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java
+++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest.java
@@ -45,6 +45,7 @@
import org.junit.Assert;
import org.junit.Test;
+import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collections;
import java.util.Map;
@@ -268,7 +269,7 @@ public void testUpdateSegmentListIfDatasourcePathSpecIsUsedWithMultiplePathSpec(
private HadoopDruidIndexerConfig testRunUpdateSegmentListIfDatasourcePathSpecIsUsed(
PathSpec datasourcePathSpec,
- Interval jobInterval
+ @Nullable Interval jobInterval
)
throws Exception
{
@@ -300,25 +301,29 @@ private HadoopDruidIndexerConfig testRunUpdateSegmentListIfDatasourcePathSpecIsU
UsedSegmentsRetriever segmentsRetriever = EasyMock.createMock(UsedSegmentsRetriever.class);
- EasyMock.expect(
- segmentsRetriever.getUsedSegmentsForIntervals(
- TEST_DATA_SOURCE,
- Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL) : null),
- Segments.ONLY_VISIBLE
+ EasyMock
+ .expect(
+ segmentsRetriever.retrieveUsedSegmentsForIntervals(
+ TEST_DATA_SOURCE,
+ Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL) : null),
+ Segments.ONLY_VISIBLE
+ )
)
- ).andReturn(ImmutableList.of(SEGMENT));
+ .andReturn(ImmutableList.of(SEGMENT));
- EasyMock.expect(
- segmentsRetriever.getUsedSegmentsForIntervals(
- TEST_DATA_SOURCE2,
- Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL2) : null),
- Segments.ONLY_VISIBLE
+ EasyMock
+ .expect(
+ segmentsRetriever.retrieveUsedSegmentsForIntervals(
+ TEST_DATA_SOURCE2,
+ Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL2) : null),
+ Segments.ONLY_VISIBLE
+ )
)
- ).andReturn(ImmutableList.of(SEGMENT2));
+ .andReturn(ImmutableList.of(SEGMENT2));
EasyMock.replay(segmentsRetriever);
- spec = HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(spec, jsonMapper, segmentsRetriever);
+ HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(spec, jsonMapper, segmentsRetriever);
return HadoopDruidIndexerConfig.fromString(jsonMapper.writeValueAsString(spec));
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java
index 04a026b50fed..984058895e3c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentChecker.java
@@ -27,6 +27,7 @@
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
import org.apache.druid.timeline.DataSegment;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import java.io.IOException;
@@ -47,34 +48,33 @@ public ActionBasedUsedSegmentChecker(TaskActionClient taskActionClient)
}
@Override
- public Set findUsedSegments(Set identifiers) throws IOException
+ public Set findUsedSegments(Set segmentIds) throws IOException
{
// Group by dataSource
- final Map> identifiersByDataSource = new TreeMap<>();
- for (SegmentIdWithShardSpec identifier : identifiers) {
- identifiersByDataSource.computeIfAbsent(identifier.getDataSource(), k -> new HashSet<>());
-
- identifiersByDataSource.get(identifier.getDataSource()).add(identifier);
+ final Map> idsByDataSource = new TreeMap<>();
+ for (SegmentIdWithShardSpec segmentId : segmentIds) {
+ idsByDataSource.computeIfAbsent(segmentId.getDataSource(), i -> new HashSet<>()).add(segmentId.asSegmentId());
}
- final Set retVal = new HashSet<>();
+ final Set usedSegments = new HashSet<>();
- for (Map.Entry> entry : identifiersByDataSource.entrySet()) {
+ for (Map.Entry> entry : idsByDataSource.entrySet()) {
+ String dataSource = entry.getKey();
+ Set segmentIdsInDataSource = entry.getValue();
final List intervals = JodaUtils.condenseIntervals(
- Iterables.transform(entry.getValue(), input -> input.getInterval())
+ Iterables.transform(segmentIdsInDataSource, SegmentId::getInterval)
);
- final Collection usedSegmentsForIntervals = taskActionClient.submit(
- new RetrieveUsedSegmentsAction(entry.getKey(), null, intervals, Segments.ONLY_VISIBLE)
- );
+ final Collection usedSegmentsForIntervals = taskActionClient
+ .submit(new RetrieveUsedSegmentsAction(dataSource, null, intervals, Segments.ONLY_VISIBLE));
for (DataSegment segment : usedSegmentsForIntervals) {
- if (identifiers.contains(SegmentIdWithShardSpec.fromDataSegment(segment))) {
- retVal.add(segment);
+ if (segmentIdsInDataSource.contains(segment.getId())) {
+ usedSegments.add(segment);
}
}
}
- return retVal;
+ return usedSegments;
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java
index 2c54106d4fd4..f114ec456465 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java
@@ -68,7 +68,7 @@ public TypeReference> getReturnTypeReference()
@Override
public List perform(Task task, TaskActionToolbox toolbox)
{
- return toolbox.getIndexerMetadataStorageCoordinator().getUnusedSegmentsForInterval(dataSource, interval);
+ return toolbox.getIndexerMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval(dataSource, interval);
}
@Override
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java
index 60411195debf..fab8c4846894 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java
@@ -44,7 +44,7 @@
* the collection only once.
*
* @implNote This action doesn't produce a {@link java.util.Set} because it's implemented via {@link
- * org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#getUsedSegmentsForIntervals} which returns
+ * org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#retrieveUsedSegmentsForIntervals} which returns
* a collection. Producing a {@link java.util.Set} would require an unnecessary copy of segments collection.
*/
public class RetrieveUsedSegmentsAction implements TaskAction>
@@ -115,7 +115,7 @@ public TypeReference> getReturnTypeReference()
public Collection perform(Task task, TaskActionToolbox toolbox)
{
return toolbox.getIndexerMetadataStorageCoordinator()
- .getUsedSegmentsForIntervals(dataSource, intervals, visibility);
+ .retrieveUsedSegmentsForIntervals(dataSource, intervals, visibility);
}
@Override
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
index 678144999e5c..b547f356d66b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocateAction.java
@@ -23,7 +23,6 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.base.Preconditions;
-import com.google.common.collect.ImmutableSet;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Task;
@@ -88,7 +87,8 @@ public SegmentAllocateAction(
@JsonProperty("sequenceName") String sequenceName,
@JsonProperty("previousSegmentId") String previousSegmentId,
@JsonProperty("skipSegmentLineageCheck") boolean skipSegmentLineageCheck,
- @JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory, // nullable for backward compatibility
+ // nullable for backward compatibility
+ @JsonProperty("shardSpecFactory") @Nullable ShardSpecFactory shardSpecFactory,
@JsonProperty("lockGranularity") @Nullable LockGranularity lockGranularity // nullable for backward compatibility
)
{
@@ -189,28 +189,26 @@ public SegmentIdWithShardSpec perform(
final Interval rowInterval = queryGranularity.bucket(timestamp);
- final Set usedSegmentsForRow = new HashSet<>(
- msc.getUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE)
- );
+ final Set usedSegmentsForRow =
+ new HashSet<>(msc.retrieveUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE));
- final SegmentIdWithShardSpec identifier = usedSegmentsForRow.isEmpty() ?
- tryAllocateFirstSegment(toolbox, task, rowInterval) :
- tryAllocateSubsequentSegment(
- toolbox,
- task,
- rowInterval,
- usedSegmentsForRow.iterator().next()
- );
+ final SegmentIdWithShardSpec identifier;
+ if (usedSegmentsForRow.isEmpty()) {
+ identifier = tryAllocateFirstSegment(toolbox, task, rowInterval);
+ } else {
+ identifier = tryAllocateSubsequentSegment(toolbox, task, rowInterval, usedSegmentsForRow.iterator().next());
+ }
if (identifier != null) {
return identifier;
}
// Could not allocate a pending segment. There's a chance that this is because someone else inserted a segment
- // overlapping with this row between when we called "mdc.getUsedSegmentsForInterval" and now. Check it again,
+ // overlapping with this row between when we called "msc.retrieveUsedSegmentsForInterval" and now. Check it again,
// and if it's different, repeat.
- if (!ImmutableSet.copyOf(msc.getUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE))
- .equals(usedSegmentsForRow)) {
+ Set newUsedSegmentsForRow =
+ new HashSet<>(msc.retrieveUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE));
+ if (!newUsedSegmentsForRow.equals(usedSegmentsForRow)) {
if (attempt < MAX_ATTEMPTS) {
final long shortRandomSleep = 50 + (long) (ThreadLocalRandom.current().nextDouble() * 450);
log.debug(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
index e7804564ef8a..b8f27389aaca 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentInsertAction.java
@@ -23,8 +23,8 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.timeline.DataSegment;
import java.util.Set;
@@ -84,7 +84,7 @@ public boolean isAudited()
public String toString()
{
return "SegmentInsertAction{" +
- "segments=" + Iterables.transform(segments, DataSegment::getId) +
+ "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
index 83e8bb96bb23..143805c82d4b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentMetadataUpdateAction.java
@@ -24,13 +24,13 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.timeline.DataSegment;
import java.util.Set;
@@ -111,7 +111,7 @@ public boolean isAudited()
public String toString()
{
return "SegmentMetadataUpdateAction{" +
- "segments=" + Iterables.transform(segments, DataSegment::getId) +
+ "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
index 4197a2167d3f..562e8681c761 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentNukeAction.java
@@ -24,13 +24,13 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.CriticalAction;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.query.DruidMetrics;
+import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.timeline.DataSegment;
import java.util.Set;
@@ -113,7 +113,7 @@ public boolean isAudited()
public String toString()
{
return "SegmentNukeAction{" +
- "segments=" + Iterables.transform(segments, DataSegment::getId) +
+ "segments=" + SegmentUtils.commaSeparatedIdentifiers(segments) +
'}';
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java
index 3b117de942fe..9194838f53e1 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractFixedIntervalTask.java
@@ -22,12 +22,17 @@
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
+import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskLockType;
+import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
import org.apache.druid.indexing.common.config.TaskConfig;
+import org.apache.druid.java.util.common.ISE;
import org.joda.time.Interval;
+import java.io.IOException;
import java.util.Map;
public abstract class AbstractFixedIntervalTask extends AbstractTask
@@ -86,4 +91,19 @@ public Interval getInterval()
public void stopGracefully(TaskConfig taskConfig)
{
}
+
+ TaskLock getAndCheckLock(TaskToolbox toolbox) throws IOException
+ {
+ // Confirm we have a lock (will throw if there isn't exactly one element)
+ final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient()));
+
+ if (!myLock.getDataSource().equals(getDataSource())) {
+ throw new ISE("Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
+ }
+
+ if (!myLock.getInterval().equals(getInterval())) {
+ throw new ISE("Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval());
+ }
+ return myLock;
+ }
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
index daaec1ba5fab..d8e975f44589 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java
@@ -79,7 +79,7 @@
import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
@@ -136,7 +136,7 @@ private static String makeTaskId(RealtimeAppenderatorIngestionSpec spec)
private final RealtimeAppenderatorIngestionSpec spec;
@JsonIgnore
- private final Queue> pendingHandoffs;
+ private final Queue> pendingHandoffs;
@JsonIgnore
private volatile Appenderator appenderator = null;
@@ -677,7 +677,7 @@ private void publishSegments(
String sequenceName
)
{
- final ListenableFuture publishFuture = driver.publish(
+ final ListenableFuture publishFuture = driver.publish(
publisher,
committerSupplier.get(),
Collections.singletonList(sequenceName)
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java
index a53e63106d73..9779989a2d1e 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/ArchiveTask.java
@@ -21,7 +21,6 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
@@ -63,16 +62,7 @@ public String getType()
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
- // Confirm we have a lock (will throw if there isn't exactly one element)
- final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient()));
-
- if (!myLock.getDataSource().equals(getDataSource())) {
- throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
- }
-
- if (!myLock.getInterval().equals(getInterval())) {
- throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval());
- }
+ final TaskLock myLock = getAndCheckLock(toolbox);
// List unused segments
final List unusedSegments = toolbox
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIOConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIOConfig.java
index e107d2e4a38b..60972de2673a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIOConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionIOConfig.java
@@ -27,7 +27,8 @@
import java.util.Objects;
/**
- * {@link IOConfig} for {@link CompactionTask}.
+ * {@link IOConfig} for {@link CompactionTask}. Should be synchronized with {@link
+ * org.apache.druid.client.indexing.ClientCompactionIOConfig}.
*
* @see CompactionInputSpec
*/
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
index 679e6c53d85c..1fa963684320 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java
@@ -30,7 +30,9 @@
import com.google.common.collect.HashBiMap;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.Lists;
+import org.apache.curator.shaded.com.google.common.base.Verify;
import org.apache.druid.client.coordinator.CoordinatorClient;
+import org.apache.druid.client.indexing.ClientCompactionTaskQuery;
import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionSchema.MultiValueHandling;
@@ -81,6 +83,7 @@
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
+import org.apache.druid.server.coordinator.duty.CompactSegments;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.TimelineObjectHolder;
@@ -104,8 +107,15 @@
import java.util.stream.Collectors;
import java.util.stream.IntStream;
+/**
+ * The client representation of this task is {@link ClientCompactionTaskQuery}. JSON
+ * serialization fields of this class must correspond to those of {@link
+ * ClientCompactionTaskQuery}.
+ */
public class CompactionTask extends AbstractBatchIndexTask
{
+ private static final Logger log = new Logger(CompactionTask.class);
+
/**
* The CompactionTask creates and runs multiple IndexTask instances. When the {@link AppenderatorsManager}
* is asked to clean up, it does so on a per-task basis keyed by task ID. However, the subtask IDs of the
@@ -118,9 +128,12 @@ public class CompactionTask extends AbstractBatchIndexTask
*/
public static final String CTX_KEY_APPENDERATOR_TRACKING_TASK_ID = "appenderatorTrackingTaskId";
- private static final Logger log = new Logger(CompactionTask.class);
private static final String TYPE = "compact";
+ static {
+ Verify.verify(TYPE.equals(CompactSegments.COMPACTION_TASK_TYPE));
+ }
+
private final CompactionIOConfig ioConfig;
@Nullable
private final DimensionsSpec dimensionsSpec;
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
index 33f7b9d3c527..353790770675 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/HadoopIndexTask.java
@@ -313,7 +313,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception
final ClassLoader loader = buildClassLoader(toolbox);
boolean determineIntervals = !spec.getDataSchema().getGranularitySpec().bucketIntervals().isPresent();
- spec = HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(
+ HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(
spec,
jsonMapper,
new OverlordActionBasedUsedSegmentsRetriever(toolbox)
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 4d39df4c1cc6..12d75dab8bf8 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
@@ -31,7 +31,6 @@
import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import com.google.common.util.concurrent.ListenableFuture;
@@ -80,6 +79,7 @@
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.IndexSpec;
+import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.segment.indexing.BatchIOConfig;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.IngestionSpec;
@@ -94,7 +94,7 @@
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
@@ -930,7 +930,8 @@ private TaskStatus generateAndPublishSegments(
? getSegmentLockHelper().getLockedExistingSegments()
: null;
// Probably we can publish atomicUpdateGroup along with segments.
- final SegmentsAndMetadata published = awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout);
+ final SegmentsAndCommitMetadata published =
+ awaitPublish(driver.publishAll(inputSegments, publisher), pushTimeout);
appenderator.close();
ingestionState = IngestionState.COMPLETED;
@@ -949,7 +950,7 @@ private TaskStatus generateAndPublishSegments(
buildSegmentsMeters.getUnparseable(),
buildSegmentsMeters.getThrownAway()
);
- log.info("Published segments: %s", Lists.transform(published.getSegments(), DataSegment::getId));
+ log.info("Published segments: %s", SegmentUtils.commaSeparatedIdentifiers(published.getSegments()));
toolbox.getTaskReportFileWriter().write(getId(), getTaskCompletionReports());
return TaskStatus.success(getId());
@@ -972,8 +973,8 @@ private TaskStatus generateAndPublishSegments(
}
}
- private static SegmentsAndMetadata awaitPublish(
- ListenableFuture publishFuture,
+ private static SegmentsAndCommitMetadata awaitPublish(
+ ListenableFuture publishFuture,
long publishTimeout
) throws ExecutionException, InterruptedException, TimeoutException
{
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
index f20cf50bed3b..51f3b0f47ff7 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/InputSourceProcessor.java
@@ -40,7 +40,7 @@
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.utils.CircularBuffer;
import org.joda.time.Interval;
@@ -87,9 +87,9 @@ public InputSourceProcessor(
* All read data is consumed by {@link BatchAppenderatorDriver} which creates new segments.
* All created segments are pushed when all input data is processed successfully.
*
- * @return {@link SegmentsAndMetadata} for the pushed segments.
+ * @return {@link SegmentsAndCommitMetadata} for the pushed segments.
*/
- public SegmentsAndMetadata process(
+ public SegmentsAndCommitMetadata process(
DataSchema dataSchema,
BatchAppenderatorDriver driver,
PartitionsSpec partitionsSpec,
@@ -155,8 +155,8 @@ public SegmentsAndMetadata process(
// in the future.
// If those segments are not pushed here, the remaining available space in appenderator will be kept
// small which could lead to smaller segments.
- final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
- LOG.debug("Pushed segments: %s", SegmentUtils.commaSeparateIdentifiers(pushed.getSegments()));
+ final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout);
+ LOG.debug("Pushed segments: %s", SegmentUtils.commaSeparatedIdentifiers(pushed.getSegments()));
}
}
} else {
@@ -174,9 +174,9 @@ public SegmentsAndMetadata process(
}
}
- final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+ final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout);
- LOG.debug("Pushed segments: %s", SegmentUtils.commaSeparateIdentifiers(pushed.getSegments()));
+ LOG.debug("Pushed segments: %s", SegmentUtils.commaSeparatedIdentifiers(pushed.getSegments()));
return pushed;
}
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/KillUnusedSegmentsTask.java
similarity index 91%
rename from indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java
rename to indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTask.java
index b7439d66de06..602c21e72a32 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/KillUnusedSegmentsTask.java
@@ -21,7 +21,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
-import org.apache.druid.client.indexing.ClientKillQuery;
+import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
@@ -44,14 +44,15 @@
import java.util.stream.Collectors;
/**
- * The client representation of this task is {@link ClientKillQuery}.
+ * The client representation of this task is {@link ClientKillUnusedSegmentsTaskQuery}.
* JSON serialization fields of this class must correspond to those of {@link
- * ClientKillQuery}, except for "id" and "context" fields.
+ * ClientKillUnusedSegmentsTaskQuery}, except for "id" and "context" fields.
*/
-public class KillTask extends AbstractFixedIntervalTask
+public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
{
+
@JsonCreator
- public KillTask(
+ public KillUnusedSegmentsTask(
@JsonProperty("id") String id,
@JsonProperty("dataSource") String dataSource,
@JsonProperty("interval") Interval interval,
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java
index 1985c1299c7a..4a0b3ac7a386 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/MoveTask.java
@@ -22,7 +22,6 @@
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.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
@@ -71,16 +70,7 @@ public String getType()
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
- // Confirm we have a lock (will throw if there isn't exactly one element)
- final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient()));
-
- if (!myLock.getDataSource().equals(getDataSource())) {
- throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
- }
-
- if (!myLock.getInterval().equals(getInterval())) {
- throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval());
- }
+ final TaskLock myLock = getAndCheckLock(toolbox);
// List unused segments
final List unusedSegments = toolbox
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java
index 4e3960a6dab6..45ae99a72bda 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RestoreTask.java
@@ -21,7 +21,6 @@
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskLock;
import org.apache.druid.indexing.common.TaskToolbox;
@@ -64,16 +63,7 @@ public String getType()
@Override
public TaskStatus run(TaskToolbox toolbox) throws Exception
{
- // Confirm we have a lock (will throw if there isn't exactly one element)
- final TaskLock myLock = Iterables.getOnlyElement(getTaskLocks(toolbox.getTaskActionClient()));
-
- if (!myLock.getDataSource().equals(getDataSource())) {
- throw new ISE("WTF?! Lock dataSource[%s] != task dataSource[%s]", myLock.getDataSource(), getDataSource());
- }
-
- if (!myLock.getInterval().equals(getInterval())) {
- throw new ISE("WTF?! Lock interval[%s] != task interval[%s]", myLock.getInterval(), getInterval());
- }
+ final TaskLock myLock = getAndCheckLock(toolbox);
// List unused segments
final List unusedSegments = toolbox
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java
index 020f29efa3ae..546fd65089d1 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/SegmentLockHelper.java
@@ -29,6 +29,7 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.io.Closer;
+import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@@ -141,16 +142,16 @@ private void verifySegmentGranularity(List segments)
if (!nonAlignedSegments.isEmpty()) {
throw new ISE(
- "Non-aligned segments[%s] for granularity[%s]",
- nonAlignedSegments.stream().map(DataSegment::getId).collect(Collectors.toList()),
+ "Non-aligned segments %s for granularity[%s]",
+ SegmentUtils.commaSeparatedIdentifiers(nonAlignedSegments),
knownSegmentGranularity
);
}
}
} else {
throw new ISE(
- "Found different granularities in segments[%s]",
- segments.stream().map(DataSegment::getId).collect(Collectors.toList())
+ "Found different granularities in segments %s",
+ SegmentUtils.commaSeparatedIdentifiers(segments)
);
}
}
@@ -167,8 +168,8 @@ private boolean tryLockSegments(TaskActionClient actionClient, List
.allMatch(segment -> segment.getVersion().equals(segmentsInInterval.get(0).getVersion()));
Preconditions.checkState(
hasSameVersion,
- "Segments[%s] should have same version",
- segmentsInInterval.stream().map(DataSegment::getId).collect(Collectors.toList())
+ "Segments %s should have same version",
+ SegmentUtils.commaSeparatedIdentifiers(segmentsInInterval)
);
final List lockResults = actionClient.submit(
new SegmentLockTryAcquireAction(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java
index a5db7586439b..c069c73cec70 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Task.java
@@ -53,14 +53,15 @@
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
- @Type(name = "kill", value = KillTask.class),
+ @Type(name = "kill", value = KillUnusedSegmentsTask.class),
@Type(name = "move", value = MoveTask.class),
@Type(name = "archive", value = ArchiveTask.class),
@Type(name = "restore", value = RestoreTask.class),
@Type(name = "index", value = IndexTask.class),
@Type(name = ParallelIndexSupervisorTask.TYPE, value = ParallelIndexSupervisorTask.class),
@Type(name = SinglePhaseSubTask.TYPE, value = SinglePhaseSubTask.class),
- @Type(name = SinglePhaseSubTask.OLD_TYPE_NAME, value = LegacySinglePhaseSubTask.class), // for backward compatibility
+ // for backward compatibility
+ @Type(name = SinglePhaseSubTask.OLD_TYPE_NAME, value = LegacySinglePhaseSubTask.class),
@Type(name = PartialHashSegmentGenerateTask.TYPE, value = PartialHashSegmentGenerateTask.class),
@Type(name = PartialHashSegmentMergeTask.TYPE, value = PartialHashSegmentMergeTask.class),
@Type(name = PartialRangeSegmentGenerateTask.TYPE, value = PartialRangeSegmentGenerateTask.class),
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java
index ada53eedd1d3..e16e52ecd05d 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/Tasks.java
@@ -19,8 +19,11 @@
package org.apache.druid.indexing.common.task;
+import org.apache.curator.shaded.com.google.common.base.Verify;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.guava.Comparators;
+import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
+import org.apache.druid.server.coordinator.duty.CompactSegments;
import org.joda.time.Interval;
import java.util.ArrayList;
@@ -34,6 +37,11 @@ public class Tasks
public static final int DEFAULT_REALTIME_TASK_PRIORITY = 75;
public static final int DEFAULT_BATCH_INDEX_TASK_PRIORITY = 50;
public static final int DEFAULT_MERGE_TASK_PRIORITY = 25;
+
+ static {
+ Verify.verify(DEFAULT_MERGE_TASK_PRIORITY == DataSourceCompactionConfig.DEFAULT_COMPACTION_TASK_PRIORITY);
+ }
+
public static final int DEFAULT_TASK_PRIORITY = 0;
public static final long DEFAULT_LOCK_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5);
public static final boolean DEFAULT_FORCE_TIME_CHUNK_LOCK = true;
@@ -42,14 +50,21 @@ public class Tasks
public static final String PRIORITY_KEY = "priority";
public static final String LOCK_TIMEOUT_KEY = "taskLockTimeout";
public static final String FORCE_TIME_CHUNK_LOCK_KEY = "forceTimeChunkLock";
- // This context is used in auto compaction. When it is set in the context, the segments created by the task
- // will fill 'lastCompactionState' in its metadata. This will be used to track what segments are compacted or not.
- // See DataSegment and NewestSegmentFirstIterator for more details.
+ /**
+ *This context is used in auto compaction. When it is set in the context, the segments created by the task
+ * will fill 'lastCompactionState' in its metadata. This will be used to track what segments are compacted or not.
+ * See {@link org.apache.druid.timeline.DataSegment} and {@link
+ * org.apache.druid.server.coordinator.duty.NewestSegmentFirstIterator} for more details.
+ */
public static final String STORE_COMPACTION_STATE_KEY = "storeCompactionState";
- public static SortedSet computeCompactIntervals(SortedSet intervals)
+ static {
+ Verify.verify(STORE_COMPACTION_STATE_KEY.equals(CompactSegments.STORE_COMPACTION_STATE_KEY));
+ }
+
+ public static SortedSet computeCondensedIntervals(SortedSet intervals)
{
- final SortedSet compactIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
+ final SortedSet condensedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
List toBeAccumulated = new ArrayList<>();
for (Interval interval : intervals) {
if (toBeAccumulated.size() == 0) {
@@ -58,15 +73,15 @@ public static SortedSet computeCompactIntervals(SortedSet in
if (toBeAccumulated.get(toBeAccumulated.size() - 1).abuts(interval)) {
toBeAccumulated.add(interval);
} else {
- compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
+ condensedIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
toBeAccumulated.clear();
toBeAccumulated.add(interval);
}
}
}
if (toBeAccumulated.size() > 0) {
- compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
+ condensedIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
}
- return compactIntervals;
+ return condensedIntervals;
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
index ed307145b073..ab43f61f4c14 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTask.java
@@ -812,8 +812,7 @@ private static void publishSegments(TaskToolbox toolbox, Map report.getNewSegments().stream())
.map(SegmentIdWithShardSpec::fromDataSegment)
.collect(Collectors.toSet());
- if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers)
- .equals(newSegments)) {
+ if (usedSegmentChecker.findUsedSegments(segmentsIdentifiers).equals(newSegments)) {
LOG.info("Our segments really do exist, awaiting handoff.");
} else {
throw new ISE("Failed to publish segments[%s]", newSegments);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java
index 642936109660..7d41dfaeff1b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/PartialSegmentGenerateTask.java
@@ -45,7 +45,7 @@
import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.timeline.DataSegment;
import java.io.File;
@@ -188,7 +188,7 @@ private List generateSegments(
pushTimeout,
inputRowIteratorBuilder
);
- final SegmentsAndMetadata pushed = inputSourceProcessor.process(
+ final SegmentsAndCommitMetadata pushed = inputSourceProcessor.process(
dataSchema,
driver,
partitionsSpec,
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java
index 8bf40136eafe..e94af771e41a 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseSubTask.java
@@ -70,7 +70,7 @@
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentAllocator;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory;
@@ -484,7 +484,7 @@ private Set generateAndPushSegments(
// There can be some segments waiting for being published even though any rows won't be added to them.
// If those segments are not published here, the available space in appenderator will be kept to be small
// which makes the size of segments smaller.
- final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+ final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout);
pushedSegments.addAll(pushed.getSegments());
LOG.info("Pushed segments[%s]", pushed.getSegments());
}
@@ -503,7 +503,7 @@ private Set generateAndPushSegments(
}
}
- final SegmentsAndMetadata pushed = driver.pushAllAndClear(pushTimeout);
+ final SegmentsAndCommitMetadata pushed = driver.pushAllAndClear(pushTimeout);
pushedSegments.addAll(pushed.getSegments());
LOG.info("Pushed segments[%s]", pushed.getSegments());
appenderator.close();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java
index f4a7889c51f3..588f1628d770 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/batch/parallel/TaskMonitor.java
@@ -90,12 +90,12 @@ public class TaskMonitor
@GuardedBy("taskCountLock")
private int numFailedTasks;
/**
- * This metric is used only for unit tests because the current task status system doesn't track the killed task
- * status. Currently, this metric only represents number of killed tasks by {@link ParallelIndexTaskRunner}.
+ * This metric is used only for unit tests because the current task status system doesn't track the canceled task
+ * status. Currently, this metric only represents the number of canceled tasks by {@link ParallelIndexTaskRunner}.
* See {@link #stop()}, {@link ParallelIndexPhaseRunner#run()}, and
* {@link ParallelIndexPhaseRunner#stopGracefully()}.
*/
- private int numKilledTasks;
+ private int numCanceledTasks;
@GuardedBy("startStopLock")
private boolean running = false;
@@ -185,7 +185,7 @@ public void start(long taskStatusCheckingPeriod)
}
/**
- * Stop task monitoring and kill all running tasks.
+ * Stop task monitoring and cancel all running tasks.
*/
public void stop()
{
@@ -194,6 +194,7 @@ public void stop()
running = false;
taskStatusChecker.shutdownNow();
+
synchronized (taskCountLock) {
if (numRunningTasks > 0) {
final Iterator iterator = runningTasks.values().iterator();
@@ -201,15 +202,15 @@ public void stop()
final MonitorEntry entry = iterator.next();
iterator.remove();
final String taskId = entry.runningTask.getId();
- log.info("Request to kill subtask[%s]", taskId);
- indexingServiceClient.killTask(taskId);
+ log.info("Request to cancel subtask[%s]", taskId);
+ indexingServiceClient.cancelTask(taskId);
numRunningTasks--;
- numKilledTasks++;
+ numCanceledTasks++;
}
if (numRunningTasks > 0) {
log.warn(
- "Inconsistent state: numRunningTasks[%d] is still not zero after trying to kill all running tasks.",
+ "Inconsistent state: numRunningTasks[%d] is still not zero after trying to cancel all running tasks.",
numRunningTasks
);
}
@@ -343,9 +344,9 @@ int getNumRunningTasks()
}
@VisibleForTesting
- int getNumKilledTasks()
+ int getNumCanceledTasks()
{
- return numKilledTasks;
+ return numCanceledTasks;
}
ParallelIndexingPhaseProgress getProgress()
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java b/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java
index 0ae705e1f30e..73bc411fb01d 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java
@@ -45,7 +45,7 @@ public OverlordActionBasedUsedSegmentsRetriever(TaskToolbox toolbox)
}
@Override
- public Collection getUsedSegmentsForIntervals(
+ public Collection retrieveUsedSegmentsForIntervals(
String dataSource,
List intervals,
Segments visibility
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java
index 4d4d0f30ad16..bc07f02d008b 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/input/DruidInputSource.java
@@ -370,8 +370,10 @@ public static List> getTimelineForInte
Collection usedSegments;
while (true) {
try {
- usedSegments =
- coordinatorClient.getDatabaseSegmentDataSourceSegments(dataSource, Collections.singletonList(interval));
+ usedSegments = coordinatorClient.fetchUsedSegmentsInDataSourceForIntervals(
+ dataSource,
+ Collections.singletonList(interval)
+ );
break;
}
catch (Throwable e) {
@@ -405,7 +407,7 @@ public static List> getTimelineForSegm
Comparators.intervalsByStartThenEnd()
);
for (WindowedSegmentId windowedSegmentId : Preconditions.checkNotNull(segmentIds, "segmentIds")) {
- final DataSegment segment = coordinatorClient.getDatabaseSegmentDataSourceSegment(
+ final DataSegment segment = coordinatorClient.fetchUsedSegment(
dataSource,
windowedSegmentId.getSegmentId()
);
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java
index cd550c2ad7bf..4671c5bb7a96 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageAdapter.java
@@ -64,6 +64,6 @@ public int deletePendingSegments(String dataSource, Interval deleteInterval)
activeTaskInterval.getStart()
);
- return indexerMetadataStorageCoordinator.deletePendingSegments(dataSource, deleteInterval);
+ return indexerMetadataStorageCoordinator.deletePendingSegmentsCreatedInInterval(dataSource, deleteInterval);
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
index 6226abd68bcb..e466e5ba60c8 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/http/OverlordResource.java
@@ -32,6 +32,7 @@
import org.apache.druid.audit.AuditEntry;
import org.apache.druid.audit.AuditInfo;
import org.apache.druid.audit.AuditManager;
+import org.apache.druid.client.indexing.ClientTaskQuery;
import org.apache.druid.common.config.ConfigManager.SetResult;
import org.apache.druid.common.config.JacksonConfigManager;
import org.apache.druid.indexer.RunnerTaskState;
@@ -146,14 +147,16 @@ public OverlordResource(
this.workerTaskRunnerQueryAdapter = workerTaskRunnerQueryAdapter;
}
+ /**
+ * Warning, magic: {@link org.apache.druid.client.indexing.HttpIndexingServiceClient#runTask} may call this method
+ * remotely with {@link ClientTaskQuery} objects, but we deserialize {@link Task} objects. See the comment for {@link
+ * ClientTaskQuery} for details.
+ */
@POST
@Path("/task")
@Consumes(MediaType.APPLICATION_JSON)
@Produces(MediaType.APPLICATION_JSON)
- public Response taskPost(
- final Task task,
- @Context final HttpServletRequest req
- )
+ public Response taskPost(final Task task, @Context final HttpServletRequest req)
{
final String dataSource = task.getDataSource();
final ResourceAction resourceAction = new ResourceAction(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
index 3fbbcb83bfaf..e37e1fb27fc6 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java
@@ -78,13 +78,14 @@
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.aggregation.AggregatorFactory;
+import org.apache.druid.segment.SegmentUtils;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.FireDepartmentMetrics;
import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorDriverAddResult;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
-import org.apache.druid.segment.realtime.appenderator.SegmentsAndMetadata;
+import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.StreamAppenderatorDriver;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
@@ -215,8 +216,8 @@ public enum Status
private final AppenderatorsManager appenderatorsManager;
private final Set publishingSequences = Sets.newConcurrentHashSet();
- private final List> publishWaitList = new ArrayList<>();
- private final List> handOffWaitList = new ArrayList<>();
+ private final List> publishWaitList = new ArrayList<>();
+ private final List> handOffWaitList = new ArrayList<>();
private final LockGranularity lockGranularityToUse;
@@ -856,7 +857,7 @@ public void onFailure(Throwable t)
// handoffFuture. handoffFuture can throw an exception if 1) the corresponding publishFuture failed or 2) it
// failed to persist sequences. It might also return null if handoff failed, but was recoverable.
// See publishAndRegisterHandoff() for details.
- List handedOffList = Collections.emptyList();
+ List handedOffList = Collections.emptyList();
if (tuningConfig.getHandoffConditionTimeout() == 0) {
handedOffList = Futures.allAsList(handOffWaitList).get();
} else {
@@ -874,7 +875,7 @@ public void onFailure(Throwable t)
}
}
- for (SegmentsAndMetadata handedOff : handedOffList) {
+ for (SegmentsAndCommitMetadata handedOff : handedOffList) {
log.info(
"Handoff complete for segments: %s",
String.join(", ", Lists.transform(handedOff.getSegments(), DataSegment::toString))
@@ -956,12 +957,12 @@ public void onFailure(Throwable t)
private void checkPublishAndHandoffFailure() throws ExecutionException, InterruptedException
{
// Check if any publishFuture failed.
- final List> publishFinished = publishWaitList
+ final List> publishFinished = publishWaitList
.stream()
.filter(Future::isDone)
.collect(Collectors.toList());
- for (ListenableFuture publishFuture : publishFinished) {
+ for (ListenableFuture publishFuture : publishFinished) {
// If publishFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3).
publishFuture.get();
}
@@ -969,12 +970,12 @@ private void checkPublishAndHandoffFailure() throws ExecutionException, Interrup
publishWaitList.removeAll(publishFinished);
// Check if any handoffFuture failed.
- final List> handoffFinished = handOffWaitList
+ final List> handoffFinished = handOffWaitList
.stream()
.filter(Future::isDone)
.collect(Collectors.toList());
- for (ListenableFuture handoffFuture : handoffFinished) {
+ for (ListenableFuture handoffFuture : handoffFinished) {
// If handoffFuture failed, the below line will throw an exception and catched by (1), and then (2) or (3).
handoffFuture.get();
}
@@ -986,13 +987,13 @@ private void publishAndRegisterHandoff(SequenceMetadata publishFuture = Futures.transform(
+ final ListenableFuture publishFuture = Futures.transform(
driver.publish(
sequenceMetadata.createPublisher(this, toolbox, ioConfig.isUseTransaction()),
sequenceMetadata.getCommitterSupplier(this, stream, lastPersistedOffsets).get(),
Collections.singletonList(sequenceMetadata.getSequenceName())
),
- (Function) publishedSegmentsAndMetadata -> {
+ (Function) publishedSegmentsAndMetadata -> {
if (publishedSegmentsAndMetadata == null) {
throw new ISE(
"Transaction failure publishing segments for sequence [%s]",
@@ -1006,21 +1007,21 @@ private void publishAndRegisterHandoff(SequenceMetadata handoffFuture = SettableFuture.create();
+ final SettableFuture handoffFuture = SettableFuture.create();
handOffWaitList.add(handoffFuture);
Futures.addCallback(
publishFuture,
- new FutureCallback()
+ new FutureCallback()
{
@Override
- public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata)
+ public void onSuccess(SegmentsAndCommitMetadata publishedSegmentsAndCommitMetadata)
{
log.info(
- "Published segments [%s] for sequence [%s] with metadata [%s].",
- String.join(", ", Lists.transform(publishedSegmentsAndMetadata.getSegments(), DataSegment::toString)),
+ "Published segments %s for sequence [%s] with metadata [%s].",
+ SegmentUtils.commaSeparatedIdentifiers(publishedSegmentsAndCommitMetadata.getSegments()),
sequenceMetadata.getSequenceName(),
- Preconditions.checkNotNull(publishedSegmentsAndMetadata.getCommitMetadata(), "commitMetadata")
+ Preconditions.checkNotNull(publishedSegmentsAndCommitMetadata.getCommitMetadata(), "commitMetadata")
);
sequences.remove(sequenceMetadata);
@@ -1036,23 +1037,20 @@ public void onSuccess(SegmentsAndMetadata publishedSegmentsAndMetadata)
}
Futures.transform(
- driver.registerHandoff(publishedSegmentsAndMetadata),
- new Function()
+ driver.registerHandoff(publishedSegmentsAndCommitMetadata),
+ new Function()
{
@Nullable
@Override
- public Void apply(@Nullable SegmentsAndMetadata handoffSegmentsAndMetadata)
+ public Void apply(@Nullable SegmentsAndCommitMetadata handoffSegmentsAndCommitMetadata)
{
- if (handoffSegmentsAndMetadata == null) {
+ if (handoffSegmentsAndCommitMetadata == null) {
log.warn(
"Failed to hand off segments: %s",
- String.join(
- ", ",
- Lists.transform(publishedSegmentsAndMetadata.getSegments(), DataSegment::toString)
- )
+ SegmentUtils.commaSeparatedIdentifiers(publishedSegmentsAndCommitMetadata.getSegments())
);
}
- handoffFuture.set(handoffSegmentsAndMetadata);
+ handoffFuture.set(handoffSegmentsAndCommitMetadata);
return null;
}
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
index d3a15cde8c4e..95e08029f9f0 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
@@ -1145,7 +1145,7 @@ public void resetInternal(DataSourceMetadata dataSourceMetadata)
if (resetMetadata.getSeekableStreamSequenceNumbers().getStream().equals(ioConfig.getStream())) {
// metadata can be null
- final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
+ final DataSourceMetadata metadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
if (metadata != null && !checkSourceMetadataMatch(metadata)) {
throw new IAE(
"Datasource metadata instance does not match required, found instance of [%s]",
@@ -1578,7 +1578,7 @@ private void verifyAndMergeCheckpoints(final TaskGroup taskGroup)
throw new RuntimeException(e);
}
- final DataSourceMetadata rawDataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
+ final DataSourceMetadata rawDataSourceMetadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
if (rawDataSourceMetadata != null && !checkSourceMetadataMatch(rawDataSourceMetadata)) {
throw new IAE(
@@ -2057,8 +2057,8 @@ private void cleanupClosedAndExpiredPartitions(
// Mark partitions as expired in metadata
@SuppressWarnings("unchecked")
SeekableStreamDataSourceMetadata currentMetadata =
- (SeekableStreamDataSourceMetadata) indexerMetadataStorageCoordinator.getDataSourceMetadata(
- dataSource);
+ (SeekableStreamDataSourceMetadata)
+ indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
SeekableStreamDataSourceMetadata cleanedMetadata =
createDataSourceMetadataWithExpiredPartitions(currentMetadata, newlyExpiredPartitions);
@@ -2805,10 +2805,10 @@ private void createNewTasks()
ImmutableMap simpleStartingOffsets = startingOffsets
.entrySet()
.stream()
- .filter(x -> x.getValue().get() != null)
+ .filter(entry -> entry.getValue().get() != null)
.collect(
Collectors.collectingAndThen(
- Collectors.toMap(Entry::getKey, x -> x.getValue().get()),
+ Collectors.toMap(Entry::getKey, entry -> entry.getValue().get()),
ImmutableMap::copyOf
)
);
@@ -2818,10 +2818,10 @@ private void createNewTasks()
simpleUnfilteredStartingOffsets = unfilteredStartingOffsets
.entrySet()
.stream()
- .filter(x -> x.getValue().get() != null)
+ .filter(entry -> entry.getValue().get() != null)
.collect(
Collectors.collectingAndThen(
- Collectors.toMap(Entry::getKey, x -> x.getValue().get()),
+ Collectors.toMap(Entry::getKey, entry -> entry.getValue().get()),
ImmutableMap::copyOf
)
);
@@ -2829,15 +2829,18 @@ private void createNewTasks()
simpleUnfilteredStartingOffsets = simpleStartingOffsets;
}
- Set exclusiveStartSequenceNumberPartitions = !useExclusiveStartingSequence
- ? Collections.emptySet()
- : startingOffsets
- .entrySet()
- .stream()
- .filter(x -> x.getValue().get() != null
- && x.getValue().isExclusive())
- .map(Entry::getKey)
- .collect(Collectors.toSet());
+ Set exclusiveStartSequenceNumberPartitions;
+ if (!useExclusiveStartingSequence) {
+ exclusiveStartSequenceNumberPartitions = Collections.emptySet();
+ } else {
+ exclusiveStartSequenceNumberPartitions = startingOffsets
+ .entrySet()
+ .stream()
+ .filter(x -> x.getValue().get() != null
+ && x.getValue().isExclusive())
+ .map(Entry::getKey)
+ .collect(Collectors.toSet());
+ }
activelyReadingTaskGroups.put(
groupId,
@@ -2988,7 +2991,7 @@ private OrderedSequenceNumber getOffsetFromStorageForPartiti
private Map getOffsetsFromMetadataStorage()
{
- final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.getDataSourceMetadata(dataSource);
+ final DataSourceMetadata dataSourceMetadata = indexerMetadataStorageCoordinator.retrieveDataSourceMetadata(dataSource);
if (dataSourceMetadata instanceof SeekableStreamDataSourceMetadata
&& checkSourceMetadataMatch(dataSourceMetadata)) {
@SuppressWarnings("unchecked")
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java
index 9f3d63823e78..86aba77cfd63 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java
@@ -74,9 +74,7 @@ public void setup() throws IOException
expectedUsedSegments.forEach(s -> actionTestKit.getTaskLockbox().unlock(task, s.getInterval()));
- expectedUnusedSegments.forEach(
- s -> actionTestKit.getMetadataSegmentManager().markSegmentAsUnused(s.getId().toString())
- );
+ expectedUnusedSegments.forEach(s -> actionTestKit.getSegmentsMetadataManager().markSegmentAsUnused(s.getId().toString()));
}
private DataSegment createSegment(Interval interval, String version)
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java
index a79b3ba327fc..c10df89ad986 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentInsertActionTest.java
@@ -32,6 +32,7 @@
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.assertj.core.api.Assertions;
import org.hamcrest.CoreMatchers;
import org.joda.time.Interval;
import org.junit.Assert;
@@ -118,13 +119,10 @@ public void testSimple() throws Exception
.build()
);
- Assert.assertEquals(
- ImmutableSet.of(SEGMENT1, SEGMENT2),
- ImmutableSet.copyOf(
- actionTestKit.getMetadataStorageCoordinator()
- .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE)
- )
- );
+ Assertions.assertThat(
+ actionTestKit.getMetadataStorageCoordinator()
+ .retrieveUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE)
+ ).containsExactlyInAnyOrder(SEGMENT1, SEGMENT2);
}
@Test
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java
index f371deb597aa..b9eb14c01c0c 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentTransactionalInsertActionTest.java
@@ -33,6 +33,7 @@
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.LinearShardSpec;
+import org.assertj.core.api.Assertions;
import org.hamcrest.CoreMatchers;
import org.joda.time.Interval;
import org.junit.Assert;
@@ -122,17 +123,14 @@ public void testTransactional() throws Exception
);
Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(SEGMENT2)), result2);
- Assert.assertEquals(
- ImmutableSet.of(SEGMENT1, SEGMENT2),
- ImmutableSet.copyOf(
- actionTestKit.getMetadataStorageCoordinator()
- .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE)
- )
- );
+ Assertions.assertThat(
+ actionTestKit.getMetadataStorageCoordinator()
+ .retrieveUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE)
+ ).containsExactlyInAnyOrder(SEGMENT1, SEGMENT2);
Assert.assertEquals(
new ObjectMetadata(ImmutableList.of(2)),
- actionTestKit.getMetadataStorageCoordinator().getDataSourceMetadata(DATA_SOURCE)
+ actionTestKit.getMetadataStorageCoordinator().retrieveDataSourceMetadata(DATA_SOURCE)
);
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java
index e1477d603743..df7311206b0f 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskActionTestKit.java
@@ -29,11 +29,11 @@
import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
-import org.apache.druid.metadata.MetadataSegmentManager;
-import org.apache.druid.metadata.MetadataSegmentManagerConfig;
import org.apache.druid.metadata.MetadataStorageConnectorConfig;
import org.apache.druid.metadata.MetadataStorageTablesConfig;
-import org.apache.druid.metadata.SQLMetadataSegmentManager;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.metadata.SegmentsMetadataManagerConfig;
+import org.apache.druid.metadata.SqlSegmentsMetadataManager;
import org.apache.druid.metadata.TestDerbyConnector;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.easymock.EasyMock;
@@ -48,7 +48,7 @@ public class TaskActionTestKit extends ExternalResource
private TaskLockbox taskLockbox;
private TestDerbyConnector testDerbyConnector;
private IndexerMetadataStorageCoordinator metadataStorageCoordinator;
- private MetadataSegmentManager metadataSegmentManager;
+ private SegmentsMetadataManager segmentsMetadataManager;
private TaskActionToolbox taskActionToolbox;
public TaskLockbox getTaskLockbox()
@@ -61,9 +61,9 @@ public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator()
return metadataStorageCoordinator;
}
- public MetadataSegmentManager getMetadataSegmentManager()
+ public SegmentsMetadataManager getSegmentsMetadataManager()
{
- return metadataSegmentManager;
+ return segmentsMetadataManager;
}
public TaskActionToolbox getTaskActionToolbox()
@@ -86,9 +86,9 @@ public void before()
testDerbyConnector
);
taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator);
- metadataSegmentManager = new SQLMetadataSegmentManager(
+ segmentsMetadataManager = new SqlSegmentsMetadataManager(
objectMapper,
- Suppliers.ofInstance(new MetadataSegmentManagerConfig()),
+ Suppliers.ofInstance(new SegmentsMetadataManagerConfig()),
Suppliers.ofInstance(metadataStorageTablesConfig),
testDerbyConnector
);
@@ -116,7 +116,7 @@ public void after()
taskLockbox = null;
testDerbyConnector = null;
metadataStorageCoordinator = null;
- metadataSegmentManager = null;
+ segmentsMetadataManager = null;
taskActionToolbox = null;
}
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactQuerySerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java
similarity index 95%
rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactQuerySerdeTest.java
rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java
index a67128ef6d01..8be6b85864df 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactQuerySerdeTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/ClientCompactionTaskQuerySerdeTest.java
@@ -24,10 +24,10 @@
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
import org.apache.druid.client.coordinator.CoordinatorClient;
-import org.apache.druid.client.indexing.ClientCompactQuery;
-import org.apache.druid.client.indexing.ClientCompactQueryTuningConfig;
import org.apache.druid.client.indexing.ClientCompactionIOConfig;
import org.apache.druid.client.indexing.ClientCompactionIntervalSpec;
+import org.apache.druid.client.indexing.ClientCompactionTaskQuery;
+import org.apache.druid.client.indexing.ClientCompactionTaskQueryTuningConfig;
import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.client.indexing.NoopIndexingServiceClient;
import org.apache.druid.data.input.SegmentsSplitHintSpec;
@@ -55,7 +55,7 @@
import java.io.IOException;
import java.util.HashMap;
-public class ClientCompactQuerySerdeTest
+public class ClientCompactionTaskQuerySerdeTest
{
private static final RowIngestionMetersFactory ROW_INGESTION_METERS_FACTORY = new TestUtils()
.getRowIngestionMetersFactory();
@@ -66,7 +66,7 @@ public class ClientCompactQuerySerdeTest
public void testSerde() throws IOException
{
final ObjectMapper mapper = setupInjectablesInObjectMapper(new DefaultObjectMapper());
- final ClientCompactQuery query = new ClientCompactQuery(
+ final ClientCompactionTaskQuery query = new ClientCompactionTaskQuery(
"datasource",
new ClientCompactionIOConfig(
new ClientCompactionIntervalSpec(
@@ -74,7 +74,7 @@ public void testSerde() throws IOException
"testSha256OfSortedSegmentIds"
)
),
- new ClientCompactQueryTuningConfig(
+ new ClientCompactionTaskQueryTuningConfig(
100,
40000,
2000L,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java
index 98bb64bdb511..89098fb4600e 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskParallelRunTest.java
@@ -103,17 +103,19 @@ public CompactionTaskParallelRunTest(LockGranularity lockGranularity)
coordinatorClient = new CoordinatorClient(null, null)
{
@Override
- public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals)
+ public Collection fetchUsedSegmentsInDataSourceForIntervals(
+ String dataSource,
+ List intervals
+ )
{
- return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals, Segments.ONLY_VISIBLE);
+ return getStorageCoordinator().retrieveUsedSegmentsForIntervals(dataSource, intervals, Segments.ONLY_VISIBLE);
}
@Override
- public DataSegment getDatabaseSegmentDataSourceSegment(String dataSource, String segmentId)
+ public DataSegment fetchUsedSegment(String dataSource, String segmentId)
{
- ImmutableDruidDataSource druidDataSource = getMetadataSegmentManager().getImmutableDataSourceWithUsedSegments(
- dataSource
- );
+ ImmutableDruidDataSource druidDataSource =
+ getSegmentsMetadataManager().getImmutableDataSourceWithUsedSegments(dataSource);
if (druidDataSource == null) {
throw new ISE("Unknown datasource[%s]", dataSource);
}
@@ -189,7 +191,7 @@ public void testDruidInputSourceCreateSplitsWithIndividualSplits() throws Except
);
List segments = new ArrayList<>(
- coordinatorClient.getDatabaseSegmentDataSourceSegments(
+ coordinatorClient.fetchUsedSegmentsInDataSourceForIntervals(
DATA_SOURCE,
ImmutableList.of(interval)
)
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
index 779113fc3d76..2b0874f5150a 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskRunTest.java
@@ -182,9 +182,12 @@ public CompactionTaskRunTest(LockGranularity lockGranularity)
coordinatorClient = new CoordinatorClient(null, null)
{
@Override
- public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals)
+ public Collection fetchUsedSegmentsInDataSourceForIntervals(
+ String dataSource,
+ List intervals
+ )
{
- return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals, Segments.ONLY_VISIBLE);
+ return getStorageCoordinator().retrieveUsedSegmentsForIntervals(dataSource, intervals, Segments.ONLY_VISIBLE);
}
};
segmentLoaderFactory = new SegmentLoaderFactory(getIndexIO(), getObjectMapper());
@@ -521,7 +524,7 @@ public void testCompactThenAppend() throws Exception
expectedSegments.addAll(appendResult.rhs);
final Set usedSegments = new HashSet<>(
- getStorageCoordinator().getUsedSegmentsForIntervals(
+ getStorageCoordinator().retrieveUsedSegmentsForIntervals(
DATA_SOURCE,
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02")),
Segments.ONLY_VISIBLE
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java
index caad2db871be..ce9580475dc7 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java
@@ -1245,7 +1245,10 @@ private static class TestCoordinatorClient extends CoordinatorClient
}
@Override
- public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals)
+ public Collection fetchUsedSegmentsInDataSourceForIntervals(
+ String dataSource,
+ List intervals
+ )
{
return ImmutableSet.copyOf(segmentMap.keySet());
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
index b3ecce4fab0e..3eeeb8bc4aeb 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/IngestionTestBase.java
@@ -49,10 +49,10 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.metadata.EntryExistsException;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
-import org.apache.druid.metadata.MetadataSegmentManager;
-import org.apache.druid.metadata.MetadataSegmentManagerConfig;
import org.apache.druid.metadata.SQLMetadataConnector;
-import org.apache.druid.metadata.SQLMetadataSegmentManager;
+import org.apache.druid.metadata.SegmentsMetadataManager;
+import org.apache.druid.metadata.SegmentsMetadataManagerConfig;
+import org.apache.druid.metadata.SqlSegmentsMetadataManager;
import org.apache.druid.metadata.TestDerbyConnector;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9;
@@ -93,7 +93,7 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
private SegmentLoaderFactory segmentLoaderFactory;
private TaskStorage taskStorage;
private IndexerSQLMetadataStorageCoordinator storageCoordinator;
- private MetadataSegmentManager segmentManager;
+ private SegmentsMetadataManager segmentsMetadataManager;
private TaskLockbox lockbox;
@Before
@@ -110,9 +110,9 @@ public void setUp() throws IOException
derbyConnectorRule.metadataTablesConfigSupplier().get(),
derbyConnectorRule.getConnector()
);
- segmentManager = new SQLMetadataSegmentManager(
+ segmentsMetadataManager = new SqlSegmentsMetadataManager(
objectMapper,
- MetadataSegmentManagerConfig::new,
+ SegmentsMetadataManagerConfig::new,
derbyConnectorRule.metadataTablesConfigSupplier(),
derbyConnectorRule.getConnector()
);
@@ -167,9 +167,9 @@ public IndexerMetadataStorageCoordinator getMetadataStorageCoordinator()
return storageCoordinator;
}
- public MetadataSegmentManager getMetadataSegmentManager()
+ public SegmentsMetadataManager getSegmentsMetadataManager()
{
- return segmentManager;
+ return segmentsMetadataManager;
}
public TaskLockbox getLockbox()
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java
similarity index 78%
rename from indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java
rename to indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java
index 4db77719b198..688158dc27c3 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/KillUnusedSegmentsTaskTest.java
@@ -27,6 +27,7 @@
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.DataSegment;
+import org.assertj.core.api.Assertions;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
@@ -35,7 +36,7 @@
import java.util.List;
import java.util.Set;
-public class KillTaskTest extends IngestionTestBase
+public class KillUnusedSegmentsTaskTest extends IngestionTestBase
{
private static final String DATA_SOURCE = "dataSource";
@@ -62,33 +63,31 @@ public void testKill() throws Exception
Assert.assertEquals(segments, announced);
Assert.assertTrue(
- getMetadataSegmentManager().markSegmentAsUnused(
+ getSegmentsMetadataManager().markSegmentAsUnused(
newSegment(Intervals.of("2019-02-01/2019-03-01"), version).getId().toString()
)
);
Assert.assertTrue(
- getMetadataSegmentManager().markSegmentAsUnused(
+ getSegmentsMetadataManager().markSegmentAsUnused(
newSegment(Intervals.of("2019-03-01/2019-04-01"), version).getId().toString()
)
);
- final KillTask task = new KillTask(null, DATA_SOURCE, Intervals.of("2019-03-01/2019-04-01"), null);
+ final KillUnusedSegmentsTask task =
+ new KillUnusedSegmentsTask(null, DATA_SOURCE, Intervals.of("2019-03-01/2019-04-01"), null);
Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode());
final List unusedSegments =
- getMetadataStorageCoordinator().getUnusedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"));
+ getMetadataStorageCoordinator().retrieveUnusedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"));
Assert.assertEquals(ImmutableList.of(newSegment(Intervals.of("2019-02-01/2019-03-01"), version)), unusedSegments);
- Assert.assertEquals(
- ImmutableSet.of(
- newSegment(Intervals.of("2019-01-01/2019-02-01"), version),
- newSegment(Intervals.of("2019-04-01/2019-05-01"), version)
- ),
- ImmutableSet.copyOf(
- getMetadataStorageCoordinator()
- .getUsedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"), Segments.ONLY_VISIBLE)
- )
+ Assertions.assertThat(
+ getMetadataStorageCoordinator()
+ .retrieveUsedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"), Segments.ONLY_VISIBLE)
+ ).containsExactlyInAnyOrder(
+ newSegment(Intervals.of("2019-01-01/2019-02-01"), version),
+ newSegment(Intervals.of("2019-04-01/2019-05-01"), version)
);
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java
index b8ae69e2894f..d415488435f6 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TaskSerdeTest.java
@@ -24,7 +24,7 @@
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
-import org.apache.druid.client.indexing.ClientKillQuery;
+import org.apache.druid.client.indexing.ClientKillUnusedSegmentsTaskQuery;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.NoopInputFormat;
@@ -344,7 +344,7 @@ public void testIndexTaskwithResourceSerde() throws Exception
@Test
public void testKillTaskSerde() throws Exception
{
- final KillTask task = new KillTask(
+ final KillUnusedSegmentsTask task = new KillUnusedSegmentsTask(
null,
"foo",
Intervals.of("2010-01-01/P1D"),
@@ -354,7 +354,7 @@ public void testKillTaskSerde() throws Exception
final String json = jsonMapper.writeValueAsString(task);
Thread.sleep(100); // Just want to run the clock a bit to make sure the task id doesn't change
- final KillTask task2 = (KillTask) jsonMapper.readValue(json, Task.class);
+ final KillUnusedSegmentsTask task2 = (KillUnusedSegmentsTask) jsonMapper.readValue(json, Task.class);
Assert.assertEquals("foo", task.getDataSource());
Assert.assertEquals(Intervals.of("2010-01-01/P1D"), task.getInterval());
@@ -364,9 +364,9 @@ public void testKillTaskSerde() throws Exception
Assert.assertEquals(task.getDataSource(), task2.getDataSource());
Assert.assertEquals(task.getInterval(), task2.getInterval());
- final KillTask task3 = (KillTask) jsonMapper.readValue(
+ final KillUnusedSegmentsTask task3 = (KillUnusedSegmentsTask) jsonMapper.readValue(
jsonMapper.writeValueAsString(
- new ClientKillQuery(
+ new ClientKillUnusedSegmentsTaskQuery(
"foo",
Intervals.of("2010-01-01/P1D")
)
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TasksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TasksTest.java
index 2bbad51392db..248df1db44d3 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TasksTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/TasksTest.java
@@ -34,7 +34,7 @@ public class TasksTest
{
@Test
- public void testComputeCompactIntervals()
+ public void testComputeCondensedIntervals()
{
final SortedSet inputIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
for (int m = 1; m < 13; m++) {
@@ -59,23 +59,23 @@ public void testComputeCompactIntervals()
inputIntervals.add(Intervals.of("2017-12-31/2018-01-01"));
- final SortedSet compactIntervals = Tasks.computeCompactIntervals(inputIntervals);
- final Iterator compactIntervalIterator = compactIntervals.iterator();
- Assert.assertTrue(compactIntervalIterator.hasNext());
+ final SortedSet condensedIntervals = Tasks.computeCondensedIntervals(inputIntervals);
+ final Iterator condensedIntervalIterator = condensedIntervals.iterator();
+ Assert.assertTrue(condensedIntervalIterator.hasNext());
- Interval compactInterval = compactIntervalIterator.next();
+ Interval condensedInterval = condensedIntervalIterator.next();
final SortedSet checkedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
for (Interval inputInterval : inputIntervals) {
- if (!compactInterval.contains(inputInterval)) {
- if (compactIntervalIterator.hasNext()) {
- compactInterval = compactIntervalIterator.next();
- Assert.assertTrue(compactInterval.contains(inputInterval));
+ if (!condensedInterval.contains(inputInterval)) {
+ if (condensedIntervalIterator.hasNext()) {
+ condensedInterval = condensedIntervalIterator.next();
+ Assert.assertTrue(condensedInterval.contains(inputInterval));
}
}
checkedIntervals.add(inputInterval);
}
- Assert.assertFalse(compactIntervalIterator.hasNext());
+ Assert.assertFalse(condensedIntervalIterator.hasNext());
Assert.assertEquals(inputIntervals, checkedIntervals);
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
index d0aeede08100..029d2bf20c6a 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/AbstractParallelIndexSupervisorTaskTest.java
@@ -260,7 +260,7 @@ public TaskStatusResponse getTaskStatus(String taskId)
}
@Override
- public String killTask(String taskId)
+ public String cancelTask(String taskId)
{
final Future taskStatusFuture = tasks.remove(taskId);
if (taskStatusFuture != null) {
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java
index 7564f8cecc4e..c0b515d31db1 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/ParallelIndexSupervisorTaskKillTest.java
@@ -119,7 +119,7 @@ public void testStopGracefully() throws Exception
// completeSubTaskSpecs should be empty because no task has reported its status to TaskMonitor
Assert.assertTrue(runner.getCompleteSubTaskSpecs().isEmpty());
- Assert.assertEquals(4, runner.getTaskMonitor().getNumKilledTasks());
+ Assert.assertEquals(4, runner.getTaskMonitor().getNumCanceledTasks());
}
@Test(timeout = 5000L)
@@ -159,7 +159,7 @@ public void testSubTaskFail() throws Exception
Assert.assertEquals(TaskState.FAILED, status.getStatusCode());
}
- Assert.assertEquals(3, runner.getTaskMonitor().getNumKilledTasks());
+ Assert.assertEquals(3, runner.getTaskMonitor().getNumCanceledTasks());
}
private ParallelIndexSupervisorTask newTask(
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java
index 0bc1b93de477..a1935630b63a 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/batch/parallel/SinglePhaseParallelIndexingTest.java
@@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
import org.apache.druid.client.indexing.IndexingServiceClient;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.LocalInputSource;
@@ -182,16 +181,16 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s
runTestTask(inputInterval, Granularities.DAY);
final Interval interval = inputInterval == null ? Intervals.ETERNITY : inputInterval;
- final Collection oldSegments =
- getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
+ final Collection allSegments =
+ getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
// Reingest the same data. Each segment should get replaced by a segment with a newer version.
runTestTask(inputInterval, secondSegmentGranularity);
// Verify that the segment has been replaced.
final Collection newSegments =
- getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
- Set allSegments = ImmutableSet.builder().addAll(oldSegments).addAll(newSegments).build();
+ getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
+ allSegments.addAll(newSegments);
final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments);
final Set visibles = timeline.findNonOvershadowedObjectsInInterval(interval, Partitions.ONLY_COMPLETE);
Assert.assertEquals(new HashSet<>(newSegments), visibles);
@@ -302,11 +301,11 @@ public void testAppendToExisting() throws Exception
final Interval interval = Intervals.of("2017/2018");
runTestTask(interval, Granularities.DAY, true);
final Collection oldSegments =
- getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
+ getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
runTestTask(interval, Granularities.DAY, true);
final Collection newSegments =
- getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
+ getStorageCoordinator().retrieveUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE);
Assert.assertTrue(newSegments.containsAll(oldSegments));
final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(newSegments);
final Set visibles = timeline.findNonOvershadowedObjectsInInterval(interval, Partitions.ONLY_COMPLETE);
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
index d4e40781016e..1a786ec2ef78 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java
@@ -144,7 +144,7 @@ public class IngestSegmentFirehoseFactoryTest
private final Set published = new HashSet<>();
@Override
- public List getUsedSegmentsForIntervals(
+ public List retrieveUsedSegmentsForIntervals(
String dataSource,
List interval,
Segments visibility
@@ -154,7 +154,7 @@ public List getUsedSegmentsForIntervals(
}
@Override
- public List getUnusedSegmentsForInterval(String dataSource, Interval interval)
+ public List retrieveUnusedSegmentsForInterval(String dataSource, Interval interval)
{
return ImmutableList.of();
}
@@ -213,7 +213,10 @@ public static Collection constructorFeeder() throws IOException
final CoordinatorClient cc = new CoordinatorClient(null, null)
{
@Override
- public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals)
+ public Collection fetchUsedSegmentsInDataSourceForIntervals(
+ String dataSource,
+ List intervals
+ )
{
return ImmutableSet.copyOf(SEGMENT_SET);
}
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
index f78124f386ce..7c2760cafbed 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java
@@ -321,7 +321,10 @@ public static Collection