diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml index 2c6e9ebda0f3..f7dbf9be42cf 100644 --- a/benchmarks/pom.xml +++ b/benchmarks/pom.xml @@ -104,6 +104,10 @@ json-flattener 0.1.0 + + com.fasterxml.jackson.core + jackson-core + com.fasterxml.jackson.dataformat jackson-dataformat-smile diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java index bde4da9f1311..14c36867e9d9 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java @@ -100,6 +100,7 @@ public void setup() .interval(schemaInfo.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java index 2f92f0b85ba4..46be7ff1d571 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java @@ -114,6 +114,7 @@ public void setup() .interval(schemaInfo.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index e8212cefa2cd..8766233f3c25 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -199,6 +199,7 @@ public void setup() .interval(schemaInfo.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(i)) + .size(0) .build(); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); LOG.info("Starting benchmark setup using cacheDir[%s], rows[%,d].", segmentGenerator.getCacheDir(), rowsPerSegment); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java index 1f6153d7af24..ec5d378aa2cc 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SerializingQueryRunner.java @@ -19,10 +19,11 @@ package org.apache.druid.benchmark.query; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.context.ResponseContext; @@ -52,17 +53,12 @@ public Sequence run( { return Sequences.map( baseRunner.run(queryPlus, responseContext), - new Function() - { - @Override - public T apply(T input) - { - try { - return smileMapper.readValue(smileMapper.writeValueAsBytes(input), clazz); - } - catch (Exception e) { - throw new RuntimeException(e); - } + input -> { + try { + return JacksonUtils.readValue(smileMapper, smileMapper.writeValueAsBytes(input), clazz); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); } } ); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java index 667210c28874..cbb339e5a087 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlBenchmark.java @@ -173,6 +173,7 @@ public void setup() .interval(schemaInfo.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(); final PlannerConfig plannerConfig = new PlannerConfig(); diff --git a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java index 8021a9909e23..d2787ddd31b2 100644 --- a/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java +++ b/benchmarks/src/main/java/org/apache/druid/benchmark/query/SqlVsNativeBenchmark.java @@ -98,6 +98,7 @@ public void setup() .interval(schemaInfo.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(); final SegmentGenerator segmentGenerator = closer.register(new SegmentGenerator()); diff --git a/core/src/main/java/org/apache/druid/common/config/JacksonConfigManager.java b/core/src/main/java/org/apache/druid/common/config/JacksonConfigManager.java index 32d7649d2c05..1075e4c1aba6 100644 --- a/core/src/main/java/org/apache/druid/common/config/JacksonConfigManager.java +++ b/core/src/main/java/org/apache/druid/common/config/JacksonConfigManager.java @@ -27,6 +27,7 @@ import org.apache.druid.audit.AuditInfo; import org.apache.druid.audit.AuditManager; import org.apache.druid.common.config.ConfigManager.SetResult; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import java.io.IOException; import java.util.concurrent.atomic.AtomicReference; @@ -115,12 +116,7 @@ public T deserialize(byte[] bytes) return defaultVal; } - try { - return jsonMapper.readValue(bytes, clazz); - } - catch (IOException e) { - throw new RuntimeException(e); - } + return JacksonUtils.readValue(jsonMapper, bytes, clazz); } }; } diff --git a/core/src/main/java/org/apache/druid/indexer/TaskStatus.java b/core/src/main/java/org/apache/druid/indexer/TaskStatus.java index a0ffe85c67cb..6bef27ee2143 100644 --- a/core/src/main/java/org/apache/druid/indexer/TaskStatus.java +++ b/core/src/main/java/org/apache/druid/indexer/TaskStatus.java @@ -67,9 +67,11 @@ public static TaskStatus fromCode(String taskId, TaskState code) return new TaskStatus(taskId, code, -1, null, null); } - // The error message can be large, so truncate it to avoid storing large objects in zookeeper/metadata storage. - // The full error message will be available via a TaskReport. - private static String truncateErrorMsg(String errorMsg) + /** + * The error message can be large, so truncate it to avoid storing large objects in zookeeper/metadata storage. + * The full error message will be available via a TaskReport. + */ + private static @Nullable String truncateErrorMsg(@Nullable String errorMsg) { if (errorMsg != null && errorMsg.length() > MAX_ERROR_MSG_LENGTH) { return errorMsg.substring(0, MAX_ERROR_MSG_LENGTH) + "..."; @@ -81,7 +83,7 @@ private static String truncateErrorMsg(String errorMsg) private final String id; private final TaskState status; private final long duration; - private final String errorMsg; + private final @Nullable String errorMsg; private final TaskLocation location; @JsonCreator @@ -89,7 +91,7 @@ protected TaskStatus( @JsonProperty("id") String id, @JsonProperty("status") TaskState status, @JsonProperty("duration") long duration, - @JsonProperty("errorMsg") String errorMsg, + @JsonProperty("errorMsg") @Nullable String errorMsg, @Nullable @JsonProperty("location") TaskLocation location ) { @@ -122,6 +124,7 @@ public long getDuration() return duration; } + @Nullable @JsonProperty("errorMsg") public String getErrorMsg() { diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index ee8bf660bd27..eb60d29a6d13 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -20,10 +20,12 @@ package org.apache.druid.java.util.common.jackson; import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; import java.util.Map; -public class JacksonUtils +public final class JacksonUtils { public static final TypeReference> TYPE_REFERENCE_MAP_STRING_OBJECT = new TypeReference>() { @@ -31,4 +33,19 @@ public class JacksonUtils public static final TypeReference> TYPE_REFERENCE_MAP_STRING_STRING = new TypeReference>() { }; + + /** Silences Jackson's {@link IOException}. */ + public static T readValue(ObjectMapper mapper, byte[] bytes, Class valueClass) + { + try { + return mapper.readValue(bytes, valueClass); + } + catch (IOException e) { + throw new RuntimeException(e); + } + } + + private JacksonUtils() + { + } } diff --git a/core/src/main/java/org/apache/druid/timeline/DataSegment.java b/core/src/main/java/org/apache/druid/timeline/DataSegment.java index 6475dc49c3f7..1fb5efae3093 100644 --- a/core/src/main/java/org/apache/druid/timeline/DataSegment.java +++ b/core/src/main/java/org/apache/druid/timeline/DataSegment.java @@ -214,6 +214,7 @@ public DataSegment( ? null : prepareCompactionState(lastCompactionState); this.binaryVersion = binaryVersion; + Preconditions.checkArgument(size >= 0); this.size = size; } diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentLister.java b/core/src/main/java/org/apache/druid/timeline/Partitions.java similarity index 52% rename from indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentLister.java rename to core/src/main/java/org/apache/druid/timeline/Partitions.java index bc8733289266..69bf4f4dc868 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentLister.java +++ b/core/src/main/java/org/apache/druid/timeline/Partitions.java @@ -17,28 +17,17 @@ * under the License. */ -package org.apache.druid.indexer.path; - -import org.apache.druid.timeline.DataSegment; -import org.joda.time.Interval; - -import java.io.IOException; -import java.util.List; +package org.apache.druid.timeline; /** + * This enum is used a parameter for several methods in {@link VersionedIntervalTimeline}, specifying whether only + * {@linkplain org.apache.druid.timeline.partition.PartitionHolder#isComplete() complete} partitions should be + * considered, or incomplete partitions as well. */ -public interface UsedSegmentLister +public enum Partitions { - /** - * Get all segments which may include any data in the interval and are flagged as used. - * - * @param dataSource The datasource to query - * @param intervals The intervals for which used segments are to be returned - * - * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the requested interval. - * - * @throws IOException - */ - List getUsedSegmentsForIntervals(String dataSource, List intervals) - throws IOException; + /** Specifies that only complete partitions should be considered. */ + ONLY_COMPLETE, + /** Specifies that complete as well as incomplete partitions should be considered. */ + INCOMPLETE_OK } 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 4998c41c0f65..728bd1ae5d53 100644 --- a/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java +++ b/core/src/main/java/org/apache/druid/timeline/VersionedIntervalTimeline.java @@ -23,6 +23,7 @@ import com.google.common.base.Function; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; +import com.google.errorprone.annotations.concurrent.GuardedBy; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.Comparators; @@ -49,6 +50,7 @@ import java.util.TreeMap; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantReadWriteLock; +import java.util.stream.Collectors; import java.util.stream.StreamSupport; /** @@ -64,12 +66,14 @@ * 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. * - * The 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 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. */ -public class VersionedIntervalTimeline> implements TimelineLookup +public class VersionedIntervalTimeline> + implements TimelineLookup { private final ReentrantReadWriteLock lock = new ReentrantReadWriteLock(true); @@ -126,9 +130,9 @@ public Map> getAllTimelineEntries( } /** - * Returns a lazy collection with all objects (including overshadowed, see {@link #findFullyOvershadowed}) in this - * VersionedIntervalTimeline to be used for iteration or {@link Collection#stream()} transformation. The order of - * objects in this collection is unspecified. + * Returns a lazy collection with all objects (including partially AND fully overshadowed, see {@link + * #findFullyOvershadowed}) in this VersionedIntervalTimeline to be used for iteration or {@link Collection#stream()} + * transformation. The order of objects in this collection is unspecified. * * Note: iteration over the returned collection may not be as trivially cheap as, for example, iteration over an * ArrayList. Try (to some reasonable extent) to organize the code so that it iterates the returned collection only @@ -147,6 +151,19 @@ public Collection iterateAllObjects() ); } + /** + * 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). + */ + public Set findNonOvershadowedObjectsInInterval(Interval interval, Partitions completeness) + { + return lookup(interval, completeness) + .stream() + .flatMap(timelineObjectHolder -> timelineObjectHolder.getObject().stream()) + .map(PartitionChunk::getObject) + .collect(Collectors.toSet()); + } + public void add(final Interval interval, VersionType version, PartitionChunk object) { addAll(Iterators.singletonIterator(object), o -> interval, o -> version); @@ -271,7 +288,7 @@ public PartitionChunk remove(Interval interval, VersionType version, /** * Does a lookup for the objects representing the given time interval. Will *only* return - * PartitionHolders that are complete. + * PartitionHolders that are {@linkplain PartitionHolder#isComplete() complete}. * * @param interval interval to find objects for * @@ -283,7 +300,7 @@ public List> lookup(Interval inter { lock.readLock().lock(); try { - return lookup(interval, false); + return lookup(interval, Partitions.ONLY_COMPLETE); } finally { lock.readLock().unlock(); @@ -295,7 +312,7 @@ public List> lookupWithIncompleteP { lock.readLock().lock(); try { - return lookup(interval, true); + return lookup(interval, Partitions.INCOMPLETE_OK); } finally { lock.readLock().unlock(); @@ -471,6 +488,7 @@ public boolean isOvershadowed(Interval interval, VersionType version, ObjectType } } + @GuardedBy("lock") private void add( NavigableMap timeline, Interval interval, @@ -507,12 +525,9 @@ private void add( } /** - * @param timeline - * @param key - * @param entry - * * @return boolean flag indicating whether or not we inserted or discarded something */ + @GuardedBy("lock") private boolean addAtKey( NavigableMap timeline, Interval key, @@ -611,6 +626,7 @@ private boolean addAtKey( return retVal; } + @GuardedBy("lock") private void addIntervalToTimeline( Interval interval, TimelineEntry entry, @@ -622,6 +638,7 @@ private void addIntervalToTimeline( } } + @GuardedBy("lock") private void remove( NavigableMap timeline, Interval interval, @@ -649,6 +666,7 @@ private void remove( } } + @GuardedBy("lock") private void remove( NavigableMap timeline, Interval interval, @@ -674,12 +692,16 @@ private void remove( } } - private List> lookup(Interval interval, boolean incompleteOk) + @GuardedBy("lock") + private List> lookup(Interval interval, Partitions completeness) { List> retVal = new ArrayList<>(); - NavigableMap timeline = (incompleteOk) - ? incompletePartitionsTimeline - : completePartitionsTimeline; + NavigableMap timeline; + if (completeness == Partitions.INCOMPLETE_OK) { + timeline = incompletePartitionsTimeline; + } else { + timeline = completePartitionsTimeline; + } for (Entry entry : timeline.entrySet()) { Interval timelineInterval = entry.getKey(); @@ -702,8 +724,8 @@ private List> lookup(Interval inte } TimelineObjectHolder firstEntry = retVal.get(0); - if (interval.overlaps(firstEntry.getInterval()) && interval.getStart() - .isAfter(firstEntry.getInterval().getStart())) { + if (interval.overlaps(firstEntry.getInterval()) && + interval.getStart().isAfter(firstEntry.getInterval().getStart())) { retVal.set( 0, new TimelineObjectHolder<>( 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 d3f090f1f1a1..d3166aeefcbb 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 @@ -54,6 +54,7 @@ import java.util.TreeMap; import java.util.function.BiPredicate; import java.util.stream.Collectors; +import java.util.stream.Stream; /** * OvershadowableManager manages the state of {@link AtomicUpdateGroup}. See the below {@link State} for details. @@ -913,9 +914,13 @@ PartitionChunk getChunk(int partitionId) } } - List> getVisibleChunks() + Stream> createVisibleChunksStream() { - return getAllChunks(visibleGroup); + return visibleGroup + .values() + .stream() + .flatMap((Short2ObjectSortedMap> map) -> map.values().stream()) + .flatMap((AtomicUpdateGroup aug) -> aug.getChunks().stream()); } List> getOvershadowedChunks() @@ -929,7 +934,7 @@ List> getStandbyChunks() return getAllChunks(standbyGroups); } - private List> getAllChunks( + private static > List> getAllChunks( TreeMap>> stateMap ) { diff --git a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java index afa84602022f..c39c4144693d 100644 --- a/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java +++ b/core/src/main/java/org/apache/druid/timeline/partition/PartitionHolder.java @@ -28,7 +28,6 @@ import java.util.Objects; import java.util.Spliterator; import java.util.stream.Stream; -import java.util.stream.StreamSupport; /** * An object that clumps together multiple other objects which each represent a shard of some space. @@ -113,18 +112,18 @@ public PartitionChunk getChunk(final int partitionNum) @Override public Iterator> iterator() { - return overshadowableManager.getVisibleChunks().iterator(); + return stream().iterator(); } @Override public Spliterator> spliterator() { - return overshadowableManager.getVisibleChunks().spliterator(); + return stream().spliterator(); } public Stream> stream() { - return StreamSupport.stream(spliterator(), false); + return overshadowableManager.createVisibleChunksStream(); } public List> getOvershadowed() diff --git a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java index b96e7e7a28b6..6d0af3f2892e 100644 --- a/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java +++ b/core/src/test/java/org/apache/druid/timeline/DataSegmentTest.java @@ -177,6 +177,7 @@ public void testIdentifier() .interval(Intervals.of("2012-01-01/2012-01-02")) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) .shardSpec(NoneShardSpec.instance()) + .size(0) .build(); Assert.assertEquals( @@ -193,6 +194,7 @@ public void testIdentifierWithZeroPartition() .interval(Intervals.of("2012-01-01/2012-01-02")) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) .shardSpec(getShardSpec(0)) + .size(0) .build(); Assert.assertEquals( @@ -209,6 +211,7 @@ public void testIdentifierWithNonzeroPartition() .interval(Intervals.of("2012-01-01/2012-01-02")) .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) .shardSpec(getShardSpec(7)) + .size(0) .build(); Assert.assertEquals( @@ -220,11 +223,8 @@ public void testIdentifierWithNonzeroPartition() @Test public void testV1SerializationNullMetrics() throws Exception { - final DataSegment segment = DataSegment.builder() - .dataSource("foo") - .interval(Intervals.of("2012-01-01/2012-01-02")) - .version(DateTimes.of("2012-01-01T11:22:33.444Z").toString()) - .build(); + final DataSegment segment = + makeDataSegment("foo", "2012-01-01/2012-01-02", DateTimes.of("2012-01-01T11:22:33.444Z").toString()); final DataSegment segment2 = MAPPER.readValue(MAPPER.writeValueAsString(segment), DataSegment.class); Assert.assertEquals("empty dimensions", ImmutableList.of(), segment2.getDimensions()); diff --git a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineSpecificDataTest.java b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineSpecificDataTest.java new file mode 100644 index 000000000000..3433e128fc6e --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineSpecificDataTest.java @@ -0,0 +1,415 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.timeline; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.timeline.partition.ImmutablePartitionHolder; +import org.apache.druid.timeline.partition.IntegerPartitionChunk; +import org.apache.druid.timeline.partition.OvershadowableInteger; +import org.apache.druid.timeline.partition.PartitionHolder; +import org.joda.time.DateTime; +import org.joda.time.Days; +import org.joda.time.Hours; +import org.joda.time.Interval; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Collections; + +/** + * This test class is separated from {@link VersionedIntervalTimelineTest} because it populates specific data for tests + * in {@link #setUp()}. + */ +public class VersionedIntervalTimelineSpecificDataTest extends VersionedIntervalTimelineTestBase +{ + @Before + public void setUp() + { + timeline = makeStringIntegerTimeline(); + + add("2011-04-01/2011-04-03", "1", 2); + add("2011-04-03/2011-04-06", "1", 3); + add("2011-04-01/2011-04-09", "2", 1); + add("2011-04-06/2011-04-09", "3", 4); + add("2011-04-01/2011-04-02", "3", 5); + + add("2011-05-01/2011-05-02", "1", 6); + add("2011-05-01/2011-05-05", "2", 7); + add("2011-05-03/2011-05-04", "3", 8); + add("2011-05-01/2011-05-10", "4", 9); + + add("2011-10-01/2011-10-02", "1", 1); + add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("3", 0, 20))); + add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger("3", 1, 21))); + add("2011-10-03/2011-10-04", "3", 3); + add("2011-10-04/2011-10-05", "4", 4); + add("2011-10-05/2011-10-06", "5", 5); + } + + @Test + public void testApril() + { + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "3", 5), + createExpected("2011-04-02/2011-04-06", "2", 1), + createExpected("2011-04-06/2011-04-09", "3", 4) + ), + timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) + ); + } + + @Test + public void testApril2() + { + Assert.assertEquals( + makeSingle("2", 1), + timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1)) + ); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "3", 5), + createExpected("2011-04-02/2011-04-03", "1", 2), + createExpected("2011-04-03/2011-04-06", "1", 3), + createExpected("2011-04-06/2011-04-09", "3", 4) + ), + timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) + ); + } + + @Test + public void testApril3() + { + Assert.assertEquals( + makeSingle("2", 1), + timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1)) + ); + Assert.assertEquals( + makeSingle("1", 2), + timeline.remove(Intervals.of("2011-04-01/2011-04-03"), "1", makeSingle("1", 2)) + ); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "3", 5), + createExpected("2011-04-03/2011-04-06", "1", 3), + createExpected("2011-04-06/2011-04-09", "3", 4) + ), + timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) + ); + } + + @Test + public void testApril4() + { + Assert.assertEquals( + makeSingle("2", 1), + timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1)) + ); + assertValues( + Arrays.asList( + createExpected("2011-04-01/2011-04-02", "3", 5), + createExpected("2011-04-02/2011-04-03", "1", 2), + createExpected("2011-04-03/2011-04-05", "1", 3) + ), + timeline.lookup(Intervals.of("2011-04-01/2011-04-05")) + ); + + assertValues( + Arrays.asList( + createExpected("2011-04-02T18/2011-04-03", "1", 2), + createExpected("2011-04-03/2011-04-04T01", "1", 3) + ), + timeline.lookup(Intervals.of("2011-04-02T18/2011-04-04T01")) + ); + } + + @Test + public void testMay() + { + assertValues( + Collections.singletonList( + createExpected("2011-05-01/2011-05-09", "4", 9) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + } + + @Test + public void testMay2() + { + Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9))); + assertValues( + Arrays.asList( + createExpected("2011-05-01/2011-05-03", "2", 7), + createExpected("2011-05-03/2011-05-04", "3", 8), + createExpected("2011-05-04/2011-05-05", "2", 7) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + } + + @Test + public void testMay3() + { + Assert.assertEquals( + makeSingle("4", 9), + timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9)) + ); + Assert.assertEquals( + makeSingle("2", 7), + timeline.remove(Intervals.of("2011-05-01/2011-05-05"), "2", makeSingle("2", 7)) + ); + assertValues( + Arrays.asList( + createExpected("2011-05-01/2011-05-02", "1", 6), + createExpected("2011-05-03/2011-05-04", "3", 8) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + } + + @Test + public void testInsertInWrongOrder() + { + DateTime overallStart = DateTimes.nowUtc().minus(Hours.TWO); + + Assert.assertTrue( + "These timestamps have to be at the end AND include now for this test to work.", + overallStart.isAfter(timeline.incompletePartitionsTimeline.lastEntry().getKey().getEnd()) + ); + + final Interval oneHourInterval1 = new Interval(overallStart.plus(Hours.THREE), overallStart.plus(Hours.FOUR)); + final Interval oneHourInterval2 = new Interval(overallStart.plus(Hours.FOUR), overallStart.plus(Hours.FIVE)); + + add(oneHourInterval1, "1", 1); + add(oneHourInterval2, "1", 1); + add(new Interval(overallStart, overallStart.plus(Days.ONE)), "2", 2); + + assertValues( + Collections.singletonList( + createExpected(oneHourInterval1.toString(), "2", 2) + ), + timeline.lookup(oneHourInterval1) + ); + } + + @Test + public void testRemove() + { + checkRemove(); + } + + @Test + public void testFindEntry() + { + Assert.assertEquals( + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), + timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1") + ); + + Assert.assertEquals( + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), + timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1") + ); + + Assert.assertEquals( + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), + timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1") + ); + + Assert.assertEquals( + new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), + timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1") + ); + + Assert.assertEquals( + null, + timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "2") + ); + + Assert.assertEquals( + null, + timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-02T17"), "1") + ); + } + + @Test + public void testPartitioning() + { + assertValues( + ImmutableList.of( + createExpected("2011-10-01/2011-10-02", "1", 1), + createExpected( + "2011-10-02/2011-10-03", + "3", + Arrays.asList( + IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("3", 0, 20)), + IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger("3", 1, 21)) + ) + ), + createExpected("2011-10-03/2011-10-04", "3", 3), + createExpected("2011-10-04/2011-10-05", "4", 4), + createExpected("2011-10-05/2011-10-06", "5", 5) + ), + timeline.lookup(Intervals.of("2011-10-01/2011-10-06")) + ); + } + + @Test + public void testPartialPartitionNotReturned() + { + testRemove(); + + add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60))); + assertValues( + ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), + timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) + ); + Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); + + add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61))); + assertValues( + ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), + timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) + ); + Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); + + add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62))); + assertValues( + ImmutableList.of( + createExpected("2011-10-05/2011-10-06", "5", 5), + createExpected( + "2011-10-06/2011-10-07", + "6", + Arrays.asList( + IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60)), + IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61)), + IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62)) + ) + ) + ), + timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) + ); + Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); + } + + @Test + public void testIncompletePartitionDoesNotOvershadow() + { + testRemove(); + + add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60))); + Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); + + add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61))); + Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); + + add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62))); + assertValues( + ImmutableSet.of( + createExpected("2011-10-05/2011-10-06", "5", 5) + ), + timeline.findFullyOvershadowed() + ); + } + + @Test + public void testRemovePartitionMakesIncomplete() + { + testIncompletePartitionDoesNotOvershadow(); + + final IntegerPartitionChunk chunk = IntegerPartitionChunk.make( + null, + 10, + 0, + new OvershadowableInteger("6", 0, 60) + ); + Assert.assertEquals(chunk, timeline.remove(Intervals.of("2011-10-05/2011-10-07"), "6", chunk)); + assertValues( + ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), + timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) + ); + Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); + } + + @Test + public void testInsertAndRemoveSameThingsion() + { + add("2011-05-01/2011-05-10", "5", 10); + assertValues( + Collections.singletonList( + createExpected("2011-05-01/2011-05-09", "5", 10) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + + Assert.assertEquals( + makeSingle("5", 10), + timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "5", makeSingle("5", 10)) + ); + assertValues( + Collections.singletonList( + createExpected("2011-05-01/2011-05-09", "4", 9) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + + add("2011-05-01/2011-05-10", "5", 10); + assertValues( + Collections.singletonList( + createExpected("2011-05-01/2011-05-09", "5", 10) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + + Assert.assertEquals( + makeSingle("4", 9), + timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9)) + ); + assertValues( + Collections.singletonList( + createExpected("2011-05-01/2011-05-09", "5", 10) + ), + timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) + ); + } + + @Test + public void testRemoveSomethingDontHave() + { + Assert.assertNull( + "Don't have it, should be null", + timeline.remove(Intervals.of("1970-01-01/2025-04-20"), "1", makeSingle("1", 1)) + ); + Assert.assertNull( + "Don't have it, should be null", + timeline.remove( + Intervals.of("2011-04-01/2011-04-09"), + "version does not exist", + makeSingle("version does not exist", 1) + ) + ); + } +} 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 28b2dccad2ae..e1ab76c506b6 100644 --- a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTest.java @@ -21,264 +21,39 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; -import com.google.common.collect.Ordering; import com.google.common.collect.Sets; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.Pair; import org.apache.druid.timeline.partition.ImmutablePartitionHolder; import org.apache.druid.timeline.partition.IntegerPartitionChunk; -import org.apache.druid.timeline.partition.NumberedOverwritingPartitionChunk; -import org.apache.druid.timeline.partition.NumberedPartitionChunk; import org.apache.druid.timeline.partition.OvershadowableInteger; -import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionHolder; -import org.apache.druid.timeline.partition.PartitionIds; -import org.apache.druid.timeline.partition.SingleElementPartitionChunk; -import org.joda.time.DateTime; -import org.joda.time.Days; -import org.joda.time.Hours; import org.joda.time.Interval; import org.junit.Assert; import org.junit.Before; import org.junit.Test; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Set; /** */ -public class VersionedIntervalTimelineTest +public class VersionedIntervalTimelineTest extends VersionedIntervalTimelineTestBase { - VersionedIntervalTimeline timeline; @Before public void setUp() { timeline = makeStringIntegerTimeline(); - - add("2011-04-01/2011-04-03", "1", 2); - add("2011-04-03/2011-04-06", "1", 3); - add("2011-04-01/2011-04-09", "2", 1); - add("2011-04-06/2011-04-09", "3", 4); - add("2011-04-01/2011-04-02", "3", 5); - - add("2011-05-01/2011-05-02", "1", 6); - add("2011-05-01/2011-05-05", "2", 7); - add("2011-05-03/2011-05-04", "3", 8); - add("2011-05-01/2011-05-10", "4", 9); - - add("2011-10-01/2011-10-02", "1", 1); - add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("3", 0, 20))); - add("2011-10-02/2011-10-03", "3", IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger("3", 1, 21))); - add("2011-10-03/2011-10-04", "3", 3); - add("2011-10-04/2011-10-05", "4", 4); - add("2011-10-05/2011-10-06", "5", 5); - } - - @Test - public void testApril() - { - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "3", 5), - createExpected("2011-04-02/2011-04-06", "2", 1), - createExpected("2011-04-06/2011-04-09", "3", 4) - ), - timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) - ); - } - - @Test - public void testApril2() - { - Assert.assertEquals( - makeSingle("2", 1), - timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1)) - ); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "3", 5), - createExpected("2011-04-02/2011-04-03", "1", 2), - createExpected("2011-04-03/2011-04-06", "1", 3), - createExpected("2011-04-06/2011-04-09", "3", 4) - ), - timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) - ); - } - - @Test - public void testApril3() - { - Assert.assertEquals( - makeSingle("2", 1), - timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1)) - ); - Assert.assertEquals( - makeSingle("1", 2), - timeline.remove(Intervals.of("2011-04-01/2011-04-03"), "1", makeSingle("1", 2)) - ); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "3", 5), - createExpected("2011-04-03/2011-04-06", "1", 3), - createExpected("2011-04-06/2011-04-09", "3", 4) - ), - timeline.lookup(Intervals.of("2011-04-01/2011-04-09")) - ); - } - - @Test - public void testApril4() - { - Assert.assertEquals( - makeSingle("2", 1), - timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "2", makeSingle("2", 1)) - ); - assertValues( - Arrays.asList( - createExpected("2011-04-01/2011-04-02", "3", 5), - createExpected("2011-04-02/2011-04-03", "1", 2), - createExpected("2011-04-03/2011-04-05", "1", 3) - ), - timeline.lookup(Intervals.of("2011-04-01/2011-04-05")) - ); - - assertValues( - Arrays.asList( - createExpected("2011-04-02T18/2011-04-03", "1", 2), - createExpected("2011-04-03/2011-04-04T01", "1", 3) - ), - timeline.lookup(Intervals.of("2011-04-02T18/2011-04-04T01")) - ); - } - - @Test - public void testMay() - { - assertValues( - Collections.singletonList( - createExpected("2011-05-01/2011-05-09", "4", 9) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - } - - @Test - public void testMay2() - { - Assert.assertNotNull(timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9))); - assertValues( - Arrays.asList( - createExpected("2011-05-01/2011-05-03", "2", 7), - createExpected("2011-05-03/2011-05-04", "3", 8), - createExpected("2011-05-04/2011-05-05", "2", 7) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - } - - @Test - public void testMay3() - { - Assert.assertEquals( - makeSingle("4", 9), - timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9)) - ); - Assert.assertEquals( - makeSingle("2", 7), - timeline.remove(Intervals.of("2011-05-01/2011-05-05"), "2", makeSingle("2", 7)) - ); - assertValues( - Arrays.asList( - createExpected("2011-05-01/2011-05-02", "1", 6), - createExpected("2011-05-03/2011-05-04", "3", 8) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - } - - @Test - public void testInsertInWrongOrder() - { - DateTime overallStart = DateTimes.nowUtc().minus(Hours.TWO); - - Assert.assertTrue( - "These timestamps have to be at the end AND include now for this test to work.", - overallStart.isAfter(timeline.incompletePartitionsTimeline.lastEntry().getKey().getEnd()) - ); - - final Interval oneHourInterval1 = new Interval(overallStart.plus(Hours.THREE), overallStart.plus(Hours.FOUR)); - final Interval oneHourInterval2 = new Interval(overallStart.plus(Hours.FOUR), overallStart.plus(Hours.FIVE)); - - add(oneHourInterval1, "1", 1); - add(oneHourInterval2, "1", 1); - add(new Interval(overallStart, overallStart.plus(Days.ONE)), "2", 2); - - assertValues( - Collections.singletonList( - createExpected(oneHourInterval1.toString(), "2", 2) - ), - timeline.lookup(oneHourInterval1) - ); - } - - @Test - public void testRemove() - { - for (TimelineObjectHolder holder : timeline.findFullyOvershadowed()) { - for (PartitionChunk chunk : holder.getObject()) { - timeline.remove(holder.getInterval(), holder.getVersion(), chunk); - } - } - - Assert.assertTrue(timeline.findFullyOvershadowed().isEmpty()); - } - - @Test - public void testFindEntry() - { - Assert.assertEquals( - new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), - timeline.findEntry(Intervals.of("2011-10-01/2011-10-02"), "1") - ); - - Assert.assertEquals( - new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), - timeline.findEntry(Intervals.of("2011-10-01/2011-10-01T10"), "1") - ); - - Assert.assertEquals( - new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), - timeline.findEntry(Intervals.of("2011-10-01T02/2011-10-02"), "1") - ); - - Assert.assertEquals( - new ImmutablePartitionHolder<>(new PartitionHolder<>(makeSingle("1", 1))), - timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "1") - ); - - Assert.assertEquals( - null, - timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-01T17"), "2") - ); - - Assert.assertEquals( - null, - timeline.findEntry(Intervals.of("2011-10-01T04/2011-10-02T17"), "1") - ); } @Test public void testFindEntryWithOverlap() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-02/2011-01-05", "2", 1); @@ -288,152 +63,11 @@ public void testFindEntryWithOverlap() ); } - @Test - public void testPartitioning() - { - assertValues( - ImmutableList.of( - createExpected("2011-10-01/2011-10-02", "1", 1), - createExpected( - "2011-10-02/2011-10-03", "3", - Arrays.asList( - IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("3", 0, 20)), - IntegerPartitionChunk.make(10, null, 1, new OvershadowableInteger("3", 1, 21)) - ) - ), - createExpected("2011-10-03/2011-10-04", "3", 3), - createExpected("2011-10-04/2011-10-05", "4", 4), - createExpected("2011-10-05/2011-10-06", "5", 5) - ), - timeline.lookup(Intervals.of("2011-10-01/2011-10-06")) - ); - } - - @Test - public void testPartialPartitionNotReturned() - { - testRemove(); - - add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60))); - assertValues( - ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), - timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) - ); - Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); - - add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61))); - assertValues( - ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), - timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) - ); - Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); - - add("2011-10-06/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62))); - assertValues( - ImmutableList.of( - createExpected("2011-10-05/2011-10-06", "5", 5), - createExpected( - "2011-10-06/2011-10-07", "6", - Arrays.asList( - IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60)), - IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61)), - IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62)) - ) - ) - ), - timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) - ); - Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); - } - - @Test - public void testIncompletePartitionDoesNotOvershadow() - { - testRemove(); - - add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(null, 10, 0, new OvershadowableInteger("6", 0, 60))); - Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); - - add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(10, 20, 1, new OvershadowableInteger("6", 1, 61))); - Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); - - add("2011-10-05/2011-10-07", "6", IntegerPartitionChunk.make(20, null, 2, new OvershadowableInteger("6", 2, 62))); - assertValues( - ImmutableSet.of( - createExpected("2011-10-05/2011-10-06", "5", 5) - ), - timeline.findFullyOvershadowed() - ); - } - - @Test - public void testRemovePartitionMakesIncomplete() - { - testIncompletePartitionDoesNotOvershadow(); - - final IntegerPartitionChunk chunk = IntegerPartitionChunk.make( - null, - 10, - 0, - new OvershadowableInteger("6", 0, 60) - ); - Assert.assertEquals(chunk, timeline.remove(Intervals.of("2011-10-05/2011-10-07"), "6", chunk)); - assertValues( - ImmutableList.of(createExpected("2011-10-05/2011-10-06", "5", 5)), - timeline.lookup(Intervals.of("2011-10-05/2011-10-07")) - ); - Assert.assertTrue("Expected no overshadowed entries", timeline.findFullyOvershadowed().isEmpty()); - } - - @Test - public void testInsertAndRemoveSameThingsion() - { - add("2011-05-01/2011-05-10", "5", 10); - assertValues( - Collections.singletonList( - createExpected("2011-05-01/2011-05-09", "5", 10) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - - Assert.assertEquals( - makeSingle("5", 10), - timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "5", makeSingle("5", 10)) - ); - assertValues( - Collections.singletonList( - createExpected("2011-05-01/2011-05-09", "4", 9) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - - add("2011-05-01/2011-05-10", "5", 10); - assertValues( - Collections.singletonList( - createExpected("2011-05-01/2011-05-09", "5", 10) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - - Assert.assertEquals( - makeSingle("4", 9), - timeline.remove(Intervals.of("2011-05-01/2011-05-10"), "4", makeSingle("4", 9)) - ); - assertValues( - Collections.singletonList( - createExpected("2011-05-01/2011-05-09", "5", 10) - ), - timeline.lookup(Intervals.of("2011-05-01/2011-05-09")) - ); - } - // 1|----| // 1|----| @Test(expected = UnsupportedOperationException.class) public void testOverlapSameVersionThrowException() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-05/2011-01-15", "1", 3); } @@ -445,8 +79,6 @@ public void testOverlapSameVersionThrowException() @Test public void testOverlapSameVersionIsOkay() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-01/2011-01-10", "2", 2); add("2011-01-01/2011-01-10", "2", 3); @@ -465,8 +97,6 @@ public void testOverlapSameVersionIsOkay() @Test public void testOverlapSecondBetween() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-10/2011-01-20", "1", 2); add("2011-01-05/2011-01-15", "2", 3); @@ -486,8 +116,6 @@ public void testOverlapSecondBetween() @Test public void testOverlapFirstBetween() { - timeline = makeStringIntegerTimeline(); - add("2011-01-05/2011-01-15", "2", 3); add("2011-01-01/2011-01-10", "1", 1); add("2011-01-10/2011-01-20", "1", 2); @@ -507,8 +135,6 @@ public void testOverlapFirstBetween() @Test public void testOverlapFirstBefore() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-05/2011-01-15", "2", 3); @@ -526,8 +152,6 @@ public void testOverlapFirstBefore() @Test public void testOverlapFirstAfter() { - timeline = makeStringIntegerTimeline(); - add("2011-01-05/2011-01-15", "2", 3); add("2011-01-01/2011-01-10", "1", 1); @@ -545,8 +169,6 @@ public void testOverlapFirstAfter() @Test public void testOverlapSecondBefore() { - timeline = makeStringIntegerTimeline(); - add("2011-01-05/2011-01-15", "1", 3); add("2011-01-01/2011-01-10", "2", 1); @@ -564,8 +186,6 @@ public void testOverlapSecondBefore() @Test public void testOverlapSecondAfter() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "2", 3); add("2011-01-05/2011-01-15", "1", 1); @@ -583,8 +203,6 @@ public void testOverlapSecondAfter() @Test public void testOverlapFirstLarger() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 2); add("2011-01-05/2011-01-15", "2", 3); @@ -603,8 +221,6 @@ public void testOverlapFirstLarger() @Test public void testOverlapSecondLarger() { - timeline = makeStringIntegerTimeline(); - add("2011-01-05/2011-01-15", "2", 3); add("2011-01-01/2011-01-20", "1", 2); @@ -623,8 +239,6 @@ public void testOverlapSecondLarger() @Test public void testOverlapSecondPartialAlign() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-10/2011-01-20", "1", 2); add("2011-01-01/2011-01-15", "2", 3); @@ -643,8 +257,6 @@ public void testOverlapSecondPartialAlign() @Test public void testOverlapFirstPartialAlign() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-15", "2", 3); add("2011-01-01/2011-01-10", "1", 1); add("2011-01-10/2011-01-20", "1", 2); @@ -664,8 +276,6 @@ public void testOverlapFirstPartialAlign() @Test public void testOverlapAscending() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-05/2011-01-20", "2", 2); add("2011-01-03/2011-01-06", "3", 3); @@ -686,8 +296,6 @@ public void testOverlapAscending() @Test public void testOverlapDescending() { - timeline = makeStringIntegerTimeline(); - add("2011-01-03/2011-01-06", "3", 3); add("2011-01-05/2011-01-20", "2", 2); add("2011-01-01/2011-01-10", "1", 1); @@ -708,8 +316,6 @@ public void testOverlapDescending() @Test public void testOverlapMixed() { - timeline = makeStringIntegerTimeline(); - add("2011-01-05/2011-01-20", "2", 2); add("2011-01-03/2011-01-06", "3", 3); add("2011-01-01/2011-01-10", "1", 1); @@ -730,8 +336,6 @@ public void testOverlapMixed() @Test public void testOverlapContainedAscending() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-02/2011-01-10", "2", 2); add("2011-01-02/2011-01-06", "3", 3); @@ -753,8 +357,6 @@ public void testOverlapContainedAscending() @Test public void testOverlapContainedDescending() { - timeline = makeStringIntegerTimeline(); - add("2011-01-02/2011-01-06", "3", 3); add("2011-01-02/2011-01-10", "2", 2); add("2011-01-01/2011-01-20", "1", 1); @@ -776,8 +378,6 @@ public void testOverlapContainedDescending() @Test public void testOverlapContainedmixed() { - timeline = makeStringIntegerTimeline(); - add("2011-01-02/2011-01-10", "2", 2); add("2011-01-02/2011-01-06", "3", 3); add("2011-01-01/2011-01-20", "1", 1); @@ -798,8 +398,6 @@ public void testOverlapContainedmixed() @Test public void testOverlapSecondContained() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-07", "1", 1); add("2011-01-07/2011-01-15", "1", 2); add("2011-01-15/2011-01-20", "1", 3); @@ -822,8 +420,6 @@ public void testOverlapSecondContained() @Test public void testOverlapFirstContained() { - timeline = makeStringIntegerTimeline(); - add("2011-01-10/2011-01-13", "2", 4); add("2011-01-01/2011-01-07", "1", 1); add("2011-01-07/2011-01-15", "1", 2); @@ -846,8 +442,6 @@ public void testOverlapFirstContained() @Test public void testOverlapSecondContainsFirst() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-01/2011-01-10", "2", 2); add("2011-01-10/2011-01-20", "2", 3); @@ -866,8 +460,6 @@ public void testOverlapSecondContainsFirst() @Test public void testOverlapFirstContainsSecond() { - timeline = makeStringIntegerTimeline(); - add("2011-01-10/2011-01-20", "2", 3); add("2011-01-01/2011-01-20", "1", 1); add("2011-01-01/2011-01-10", "2", 2); @@ -887,8 +479,6 @@ public void testOverlapFirstContainsSecond() @Test public void testOverlapLayeredAscending() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-10", "1", 1); add("2011-01-05/2011-01-15", "2", 2); add("2011-01-15/2011-01-25", "3", 3); @@ -909,8 +499,6 @@ public void testOverlapLayeredAscending() @Test public void testOverlapLayeredDescending() { - timeline = makeStringIntegerTimeline(); - add("2011-01-15/2011-01-25", "3", 3); add("2011-01-05/2011-01-15", "2", 2); add("2011-01-01/2011-01-10", "1", 1); @@ -930,8 +518,6 @@ public void testOverlapLayeredDescending() @Test public void testOverlapV1Large() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-15", "1", 1); add("2011-01-03/2011-01-05", "2", 2); add("2011-01-13/2011-01-20", "2", 3); @@ -952,8 +538,6 @@ public void testOverlapV1Large() @Test public void testOverlapV2Large() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-15", "2", 1); add("2011-01-03/2011-01-05", "1", 2); add("2011-01-13/2011-01-20", "1", 3); @@ -972,8 +556,6 @@ public void testOverlapV2Large() @Test public void testOverlapV1LargeIsAfter() { - timeline = makeStringIntegerTimeline(); - add("2011-01-03/2011-01-20", "1", 1); add("2011-01-01/2011-01-05", "2", 2); add("2011-01-13/2011-01-17", "2", 3); @@ -994,8 +576,6 @@ public void testOverlapV1LargeIsAfter() @Test public void testOverlapV1SecondLargeIsAfter() { - timeline = makeStringIntegerTimeline(); - add("2011-01-13/2011-01-17", "2", 3); add("2011-01-01/2011-01-05", "2", 2); add("2011-01-03/2011-01-20", "1", 1); @@ -1016,8 +596,6 @@ public void testOverlapV1SecondLargeIsAfter() @Test public void testOverlapV1FirstBetween() { - timeline = makeStringIntegerTimeline(); - add("2011-01-03/2011-01-17", "1", 1); add("2011-01-01/2011-01-05", "2", 2); add("2011-01-15/2011-01-20", "2", 3); @@ -1037,8 +615,6 @@ public void testOverlapV1FirstBetween() @Test public void testOverlapV1SecondBetween() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-05", "2", 2); add("2011-01-15/2011-01-20", "2", 3); add("2011-01-03/2011-01-17", "1", 1); @@ -1060,8 +636,6 @@ public void testOverlapV1SecondBetween() @Test public void testOverlapLargeUnderlyingWithSmallDayAlignedOverlays() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-05", "1", 1); add("2011-01-03/2011-01-04", "2", 2); add("2011-01-04/2011-01-05", "3", 3); @@ -1083,8 +657,6 @@ public void testOverlapLargeUnderlyingWithSmallDayAlignedOverlays() @Test public void testOverlapCausesNullEntries() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01T12/2011-01-02", "3", 3); add("2011-01-02/3011-01-03", "1", 1); add("2011-01-01/2011-01-02", "2", 2); @@ -1105,8 +677,6 @@ public void testOverlapCausesNullEntries() @Test public void testOverlapOvershadowedThirdContains() { - timeline = makeStringIntegerTimeline(); - add("2011-01-03/2011-01-06", "1", 1); add("2011-01-09/2011-01-12", "1", 2); add("2011-01-02/2011-01-08", "2", 3); @@ -1130,8 +700,6 @@ public void testOverlapOvershadowedThirdContains() @Test public void testOverlapOvershadowedAligned() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-05", "2", 1); add("2011-01-05/2011-01-10", "2", 2); add("2011-01-01/2011-01-10", "1", 3); @@ -1153,8 +721,6 @@ public void testOverlapOvershadowedAligned() @Test public void testOverlapOvershadowedSomeComplexOverlapsCantThinkOfBetterName() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-05", "2", 1); add("2011-01-10/2011-01-15", "2", 2); add("2011-01-03/2011-01-12", "1", 3); @@ -1172,8 +738,6 @@ public void testOverlapOvershadowedSomeComplexOverlapsCantThinkOfBetterName() @Test public void testOverlapAndRemove() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-10/2011-01-15", "2", 2); @@ -1190,8 +754,6 @@ public void testOverlapAndRemove() @Test public void testOverlapAndRemove2() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-10/2011-01-20", "2", 2); add("2011-01-20/2011-01-30", "3", 4); @@ -1211,8 +773,6 @@ public void testOverlapAndRemove2() @Test public void testOverlapAndRemove3() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-02/2011-01-03", "2", 2); add("2011-01-10/2011-01-14", "2", 3); @@ -1232,8 +792,6 @@ public void testOverlapAndRemove3() @Test public void testOverlapAndRemove4() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-10/2011-01-15", "2", 2); add("2011-01-15/2011-01-20", "2", 3); @@ -1253,8 +811,6 @@ public void testOverlapAndRemove4() @Test public void testOverlapAndRemove5() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-20", "1", 1); add("2011-01-10/2011-01-15", "2", 2); timeline.remove(Intervals.of("2011-01-10/2011-01-15"), "2", makeSingle("2", 2)); @@ -1268,24 +824,9 @@ public void testOverlapAndRemove5() ); } - @Test - public void testRemoveSomethingDontHave() - { - Assert.assertNull( - "Don't have it, should be null", - timeline.remove(Intervals.of("1970-01-01/2025-04-20"), "1", makeSingle("1", 1)) - ); - Assert.assertNull( - "Don't have it, should be null", - timeline.remove(Intervals.of("2011-04-01/2011-04-09"), "version does not exist", makeSingle("version does not exist", 1)) - ); - } - @Test public void testRemoveNothingBacking() { - timeline = makeStringIntegerTimeline(); - add("2011-01-01/2011-01-05", "1", 1); add("2011-01-05/2011-01-10", "2", 2); add("2011-01-10/2011-01-15", "3", 3); @@ -1306,8 +847,6 @@ public void testRemoveNothingBacking() @Test public void testOvershadowingHigherVersionWins1() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-03", "1", 2); add("2011-04-03/2011-04-06", "1", 3); add("2011-04-06/2011-04-09", "1", 4); @@ -1326,8 +865,6 @@ public void testOvershadowingHigherVersionWins1() @Test public void testOvershadowingHigherVersionWins2() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); add("2011-04-01/2011-04-03", "2", 2); add("2011-04-03/2011-04-06", "2", 3); @@ -1344,8 +881,6 @@ public void testOvershadowingHigherVersionWins2() @Test public void testOvershadowingHigherVersionWins3() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-03", "1", 2); add("2011-04-03/2011-04-06", "1", 3); add("2011-04-09/2011-04-12", "1", 4); @@ -1364,8 +899,6 @@ public void testOvershadowingHigherVersionWins3() @Test public void testOvershadowingHigherVersionWins4() { - timeline = makeStringIntegerTimeline(); - add("2011-04-03/2011-04-06", "1", 3); add("2011-04-06/2011-04-09", "1", 4); add("2011-04-01/2011-04-09", "2", 1); @@ -1382,8 +915,6 @@ public void testOvershadowingHigherVersionWins4() @Test public void testOvershadowingHigherVersionNeverOvershadowedByLower1() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); add("2011-04-03/2011-04-06", "2", 3); add("2011-04-06/2011-04-09", "2", 4); @@ -1397,8 +928,6 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower1() @Test public void testOvershadowingHigherVersionNeverOvershadowedByLower2() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); add("2011-04-01/2011-04-03", "2", 2); add("2011-04-06/2011-04-09", "2", 4); @@ -1412,8 +941,6 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower2() @Test public void testOvershadowingHigherVersionNeverOvershadowedByLower3() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); add("2011-04-01/2011-04-03", "2", 2); add("2011-04-03/2011-04-06", "2", 3); @@ -1427,8 +954,6 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower3() @Test public void testOvershadowingHigherVersionNeverOvershadowedByLower4() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "2", 1); add("2011-04-01/2011-04-03", "3", 2); add("2011-04-03/2011-04-06", "4", 3); @@ -1445,8 +970,6 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower4() @Test public void testOvershadowingHigherVersionNeverOvershadowedByLower5() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-12", "2", 1); add("2011-04-01/2011-04-03", "3", 2); add("2011-04-06/2011-04-09", "4", 3); @@ -1465,8 +988,6 @@ public void testOvershadowingHigherVersionNeverOvershadowedByLower5() @Test public void testOvershadowingSameIntervalHighVersionWins() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); add("2011-04-01/2011-04-09", "9", 2); add("2011-04-01/2011-04-09", "2", 3); @@ -1483,8 +1004,6 @@ public void testOvershadowingSameIntervalHighVersionWins() @Test public void testOvershadowingSameIntervalSameVersionAllKept() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); add("2011-04-01/2011-04-09", "9", 2); add("2011-04-01/2011-04-09", "2", 3); @@ -1502,19 +1021,15 @@ public void testOvershadowingSameIntervalSameVersionAllKept() @Test public void testNotFoundReturnsEmpty() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-09", "1", 1); Assert.assertTrue(timeline.lookup(Intervals.of("1970/1980")).isEmpty()); } - // https://github.com/apache/incubator-druid/issues/3010 + /** https://github.com/apache/incubator-druid/issues/3010 */ @Test public void testRemoveIncompleteKeepsComplete() { - timeline = makeStringIntegerTimeline(); - add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("1", 0, 77))); add("2011-04-01/2011-04-02", "1", IntegerPartitionChunk.make(1, null, 1, new OvershadowableInteger("1", 1, 88))); add("2011-04-01/2011-04-02", "2", IntegerPartitionChunk.make(null, 1, 0, new OvershadowableInteger("2", 0, 99))); @@ -1555,7 +1070,7 @@ public void testRemoveIncompleteKeepsComplete() timeline.findFullyOvershadowed() ); - testRemove(); + checkRemove(); assertValues( ImmutableList.of( @@ -1573,8 +1088,6 @@ public void testRemoveIncompleteKeepsComplete() @Test public void testIsOvershadowedWithNonOverlappingSegmentsInTimeline() { - timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-07", "1", makeSingle("1", 1)); add("2011-04-07/2011-04-09", "1", makeSingle("1", 1)); @@ -1638,8 +1151,6 @@ public void testIsOvershadowedWithNonOverlappingSegmentsInTimeline() @Test public void testIsOvershadowedWithOverlappingSegmentsInTimeline() { - timeline = makeStringIntegerTimeline(); - add("2011-04-05/2011-04-09", "11", makeSingle("11", 1)); add("2011-04-07/2011-04-11", "12", makeSingle("12", 1)); @@ -1722,8 +1233,6 @@ public void testIsOvershadowedWithOverlappingSegmentsInTimeline() @Test public void testOvershadowedByReference() { - timeline = makeStringIntegerTimeline(); - add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0)); add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0)); add("2019-01-01/2019-01-02", "0", makeNumbered("0", 2, 0)); @@ -1746,8 +1255,6 @@ public void testOvershadowedByReference() @Test public void testOvershadowedByReferenceChain() { - timeline = makeStringIntegerTimeline(); - // 2019-01-01/2019-01-02 add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0)); add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0)); @@ -1786,8 +1293,6 @@ public void testOvershadowedByReferenceChain() @Test public void testOvershadowedByReferenceAndThenVersion() { - timeline = makeStringIntegerTimeline(); - // 2019-01-01/2019-01-02 add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0)); add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0)); @@ -1826,8 +1331,6 @@ public void testOvershadowedByReferenceAndThenVersion() @Test public void testOvershadowedByVersionAndThenReference() { - timeline = makeStringIntegerTimeline(); - // 2019-01-01/2019-01-02 add("2019-01-01/2019-01-02", "0", makeNumbered("0", 0, 0)); add("2019-01-01/2019-01-02", "0", makeNumbered("0", 1, 0)); @@ -1873,8 +1376,6 @@ public void testOvershadowedByVersionAndThenReference() @Test public void testFallbackOnMissingSegment() { - timeline = makeStringIntegerTimeline(); - final Interval interval = Intervals.of("2019-01-01/2019-01-02"); add(interval, "0", makeNumbered("0", 0, 0)); @@ -1920,7 +1421,6 @@ public void testFallbackOnMissingSegment() @Test public void testAddSameChunkToFullAtomicUpdateGroup() { - timeline = makeStringIntegerTimeline(); final Interval interval = Intervals.of("2019-01-01/2019-01-02"); add(interval, "0", makeNumbered("0", 0, 0)); add(interval, "0", makeNumberedOverwriting("0", 0, 0, 0, 1, 1, 1)); @@ -1942,7 +1442,6 @@ public void testAddSameChunkToFullAtomicUpdateGroup() @Test public void testOvershadowMultipleStandbyAtomicUpdateGroup() { - timeline = makeStringIntegerTimeline(); final Interval interval = Intervals.of("2019-01-01/2019-01-02"); add(interval, "0", makeNumberedOverwriting("0", 0, 0, 0, 1, 1, 2)); add(interval, "0", makeNumberedOverwriting("0", 1, 0, 0, 1, 2, 2)); @@ -1970,7 +1469,6 @@ public void testOvershadowMultipleStandbyAtomicUpdateGroup() @Test public void testIsOvershadowedForOverwritingSegments() { - timeline = makeStringIntegerTimeline(); final Interval interval = Intervals.of("2019-01-01/2019-01-02"); add(interval, "0", makeNumberedOverwriting("0", 0, 0, 5, 10, 10, 1)); @@ -1989,160 +1487,17 @@ public void testIsOvershadowedForOverwritingSegments() Assert.assertFalse(timeline.isOvershadowed(interval, "0", makeNumberedOverwriting("0", 1, 0, 4, 15, 12, 2).getObject())); } - private TimelineObjectHolder makeTimelineObjectHolder( - String interval, - String version, - List> chunks - ) - { - return new TimelineObjectHolder<>( - Intervals.of(interval), - Intervals.of(interval), - version, - new PartitionHolder<>(chunks) - ); - } - - private Pair>> createExpected( - String intervalString, - String version, - Integer value - ) - { - return createExpected( - intervalString, - version, - Collections.singletonList(makeSingle(version, value)) - ); - } - - private Pair>> createExpected( - String intervalString, - String version, - List> values - ) - { - return Pair.of( - Intervals.of(intervalString), - Pair.of(version, new PartitionHolder<>(values)) - ); - } - - private PartitionChunk makeSingle(String majorVersion, int value) - { - return makeSingle(majorVersion, 0, value); - } - - private PartitionChunk makeSingle(String majorVersion, int partitionNum, int val) - { - return new SingleElementPartitionChunk<>(new OvershadowableInteger(majorVersion, partitionNum, val)); - } - - private PartitionChunk makeNumbered(String majorVersion, int partitionNum, int val) - { - return new NumberedPartitionChunk<>( - partitionNum, - 0, - new OvershadowableInteger(majorVersion, partitionNum, val) - ); - } - - private PartitionChunk makeNumberedOverwriting( - String majorVersion, - int partitionNumOrdinal, - int val, - int startRootPartitionId, - int endRootPartitionId, - int minorVersion, - int atomicUpdateGroupSize - ) - { - final int partitionNum = PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + partitionNumOrdinal; - return new NumberedOverwritingPartitionChunk<>( - partitionNum, - new OvershadowableInteger( - majorVersion, - partitionNum, - val, - startRootPartitionId, - endRootPartitionId, - minorVersion, - atomicUpdateGroupSize - ) - ); - } - - private void add(String interval, String version, Integer value) - { - add(Intervals.of(interval), version, value); - } - - private void add(Interval interval, String version, Integer value) - { - add(interval, version, makeSingle(version, value)); - } - - private void add(String interval, String version, PartitionChunk value) - { - add(Intervals.of(interval), version, value); - } - - private void add(Interval interval, String version, PartitionChunk value) - { - timeline.add(interval, version, value); - } - - private void assertValues( - List>>> expected, - List> actual - ) - { - Assert.assertEquals("Sizes did not match.", expected.size(), actual.size()); - - Iterator>>> expectedIter = expected.iterator(); - Iterator> actualIter = actual.iterator(); - - while (expectedIter.hasNext()) { - Pair>> pair = expectedIter.next(); - TimelineObjectHolder holder = actualIter.next(); - - Assert.assertEquals(pair.lhs, holder.getInterval()); - Assert.assertEquals(pair.rhs.lhs, holder.getVersion()); - - final List> expectedChunks = Lists.newArrayList(pair.rhs.rhs); - final List> actualChunks = Lists.newArrayList(holder.getObject()); - - Assert.assertEquals(expectedChunks.size(), actualChunks.size()); - for (int i = 0; i < expectedChunks.size(); i++) { - // Check partitionNumber first - Assert.assertEquals(expectedChunks.get(i), actualChunks.get(i)); - final OvershadowableInteger expectedInteger = expectedChunks.get(i).getObject(); - final OvershadowableInteger actualInteger = actualChunks.get(i).getObject(); - Assert.assertEquals(expectedInteger, actualInteger); - } - } - } - - private void assertValues( - Set>>> expected, - Set> actual - ) + @Test + public void testIterateAllObjects() { - Assert.assertEquals("Sizes did not match.", expected.size(), actual.size()); - - Set>>> actualSet = - Sets.newHashSet( - Iterables.transform( - actual, - input -> new Pair<>(input.getInterval(), new Pair<>(input.getVersion(), input.getObject())) - ) - ); + add("2011-01-01/2011-01-10", "1", 1); + add("2011-01-01/2011-01-10", "2", 3); - Assert.assertEquals(expected, actualSet); + Collection overshadowableIntegers = timeline.iterateAllObjects(); + Assert.assertEquals(2, overshadowableIntegers.size()); + // Tests that the "iteration" size of the returned collection is 2 (the "reported" size from size() method may be + // deceptive). + Assert.assertEquals(2, Lists.newArrayList(overshadowableIntegers.iterator()).size()); } - private VersionedIntervalTimeline makeStringIntegerTimeline() - { - return new VersionedIntervalTimeline<>(Ordering.natural()); - } } diff --git a/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTestBase.java b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTestBase.java new file mode 100644 index 000000000000..d3a624039094 --- /dev/null +++ b/core/src/test/java/org/apache/druid/timeline/VersionedIntervalTimelineTestBase.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.timeline; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.timeline.partition.NumberedOverwritingPartitionChunk; +import org.apache.druid.timeline.partition.NumberedPartitionChunk; +import org.apache.druid.timeline.partition.OvershadowableInteger; +import org.apache.druid.timeline.partition.PartitionChunk; +import org.apache.druid.timeline.partition.PartitionHolder; +import org.apache.druid.timeline.partition.PartitionIds; +import org.apache.druid.timeline.partition.SingleElementPartitionChunk; +import org.joda.time.Interval; +import org.junit.Assert; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +/** + * The purpose of this class is to host the common static methods used in {@link VersionedIntervalTimelineTest} and + * {@link VersionedIntervalTimelineSpecificDataTest}. When Druid style allows static imports in tests, this + * functionality should be moved to {@link VersionedIntervalTimelineTest} and then used statically from {@link + * VersionedIntervalTimelineSpecificDataTest}. + */ +public class VersionedIntervalTimelineTestBase +{ + static void assertValues( + List>>> expected, + List> actual + ) + { + Assert.assertEquals("Sizes did not match.", expected.size(), actual.size()); + + Iterator>>> expectedIter = expected.iterator(); + Iterator> actualIter = actual.iterator(); + + while (expectedIter.hasNext()) { + Pair>> pair = expectedIter.next(); + TimelineObjectHolder holder = actualIter.next(); + + Assert.assertEquals(pair.lhs, holder.getInterval()); + Assert.assertEquals(pair.rhs.lhs, holder.getVersion()); + + final List> expectedChunks = Lists.newArrayList(pair.rhs.rhs); + final List> actualChunks = Lists.newArrayList(holder.getObject()); + + Assert.assertEquals(expectedChunks.size(), actualChunks.size()); + for (int i = 0; i < expectedChunks.size(); i++) { + // Check partitionNumber first + Assert.assertEquals(expectedChunks.get(i), actualChunks.get(i)); + final OvershadowableInteger expectedInteger = expectedChunks.get(i).getObject(); + final OvershadowableInteger actualInteger = actualChunks.get(i).getObject(); + Assert.assertEquals(expectedInteger, actualInteger); + } + } + } + + static void assertValues( + Set>>> expected, + Set> actual + ) + { + Assert.assertEquals("Sizes did not match.", expected.size(), actual.size()); + + Set>>> actualSet = + Sets.newHashSet( + Iterables.transform( + actual, + input -> new Pair<>(input.getInterval(), new Pair<>(input.getVersion(), input.getObject())) + ) + ); + + Assert.assertEquals(expected, actualSet); + } + + static VersionedIntervalTimeline makeStringIntegerTimeline() + { + return new VersionedIntervalTimeline<>(Ordering.natural()); + } + + VersionedIntervalTimeline timeline; + + void checkRemove() + { + for (TimelineObjectHolder holder : timeline.findFullyOvershadowed()) { + for (PartitionChunk chunk : holder.getObject()) { + timeline.remove(holder.getInterval(), holder.getVersion(), chunk); + } + } + + Assert.assertTrue(timeline.findFullyOvershadowed().isEmpty()); + } + + TimelineObjectHolder makeTimelineObjectHolder( + String interval, + String version, + List> chunks + ) + { + return new TimelineObjectHolder<>( + Intervals.of(interval), + Intervals.of(interval), + version, + new PartitionHolder<>(chunks) + ); + } + + Pair>> createExpected( + String intervalString, + String version, + Integer value + ) + { + return createExpected( + intervalString, + version, + Collections.singletonList(makeSingle(version, value)) + ); + } + + Pair>> createExpected( + String intervalString, + String version, + List> values + ) + { + return Pair.of( + Intervals.of(intervalString), + Pair.of(version, new PartitionHolder<>(values)) + ); + } + + PartitionChunk makeSingle(String majorVersion, int value) + { + return makeSingle(majorVersion, 0, value); + } + + private PartitionChunk makeSingle(String majorVersion, int partitionNum, int val) + { + return new SingleElementPartitionChunk<>(new OvershadowableInteger(majorVersion, partitionNum, val)); + } + + PartitionChunk makeNumbered(String majorVersion, int partitionNum, int val) + { + return new NumberedPartitionChunk<>( + partitionNum, + 0, + new OvershadowableInteger(majorVersion, partitionNum, val) + ); + } + + PartitionChunk makeNumberedOverwriting( + String majorVersion, + int partitionNumOrdinal, + int val, + int startRootPartitionId, + int endRootPartitionId, + int minorVersion, + int atomicUpdateGroupSize + ) + { + final int partitionNum = PartitionIds.NON_ROOT_GEN_START_PARTITION_ID + partitionNumOrdinal; + return new NumberedOverwritingPartitionChunk<>( + partitionNum, + new OvershadowableInteger( + majorVersion, + partitionNum, + val, + startRootPartitionId, + endRootPartitionId, + minorVersion, + atomicUpdateGroupSize + ) + ); + } + + void add(String interval, String version, Integer value) + { + add(Intervals.of(interval), version, value); + } + + void add(Interval interval, String version, Integer value) + { + add(interval, version, makeSingle(version, value)); + } + + void add(String interval, String version, PartitionChunk value) + { + add(Intervals.of(interval), version, value); + } + + protected void add(Interval interval, String version, PartitionChunk value) + { + timeline.add(interval, version, value); + } +} diff --git a/core/src/test/java/org/apache/druid/timeline/partition/OvershadowableManagerTest.java b/core/src/test/java/org/apache/druid/timeline/partition/OvershadowableManagerTest.java index 71566d01f5a9..a48cef7d0dab 100644 --- a/core/src/test/java/org/apache/druid/timeline/partition/OvershadowableManagerTest.java +++ b/core/src/test/java/org/apache/druid/timeline/partition/OvershadowableManagerTest.java @@ -20,6 +20,7 @@ package org.apache.druid.timeline.partition; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Sets; import org.apache.druid.timeline.partition.OvershadowableManager.RootPartitionRange; import org.apache.druid.timeline.partition.OvershadowableManager.State; import org.junit.Assert; @@ -980,7 +981,7 @@ private void assertManagerState() Assert.assertEquals( "Mismatched visible chunks", new HashSet<>(expectedVisibleChunks), - new HashSet<>(manager.getVisibleChunks()) + Sets.newHashSet(manager.createVisibleChunksStream().iterator()) ); Assert.assertEquals( "Mismatched overshadowed chunks", 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 c6ff2784ccb2..bd43ac639a49 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 @@ -31,6 +31,7 @@ import org.apache.druid.indexing.common.task.HadoopIndexTask; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskMaster; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.Supervisor; @@ -54,6 +55,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -341,7 +343,8 @@ Pair, Map>> checkSegment getVersionAndBaseSegments( metadataStorageCoordinator.getUsedSegmentsForInterval( dataSource, - ALL_INTERVAL + ALL_INTERVAL, + Segments.ONLY_VISIBLE ) ); // Pair max(created_date), interval -> list> @@ -373,7 +376,8 @@ Pair, Map>> checkSegment final String versionOfDerivative = derivativeVersion.get(entry.getKey()); final int baseCount = baseSegments.get(entry.getKey()).size(); final IntSupplier usedCountSupplier = () -> - metadataStorageCoordinator.getUsedSegmentsForInterval(spec.getBaseDataSource(), entry.getKey()).size(); + metadataStorageCoordinator + .getUsedSegmentsForInterval(spec.getBaseDataSource(), entry.getKey(), Segments.ONLY_VISIBLE).size(); if (versionOfBase.compareTo(versionOfDerivative) > 0 && baseCount == usedCountSupplier.getAsInt()) { toBuildInterval.put(entry.getKey(), versionOfBase); } @@ -429,7 +433,7 @@ private void submitTasks( } private Pair, Map>> getVersionAndBaseSegments( - List snapshot + Collection snapshot ) { Map versions = new HashMap<>(); @@ -444,7 +448,7 @@ private Pair, Map>> getVersion } private Pair, Map>> getMaxCreateDateAndBaseSegments( - List> snapshot + Collection> snapshot ) { Interval maxAllowedToBuildInterval = snapshot.parallelStream() 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 5ba45af10b98..eeacc68c8378 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 @@ -35,6 +35,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -46,11 +47,8 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; -import java.io.IOException; import java.sql.ResultSet; -import java.sql.SQLException; import java.util.HashSet; import java.util.List; import java.util.Set; @@ -157,29 +155,24 @@ private void updateDerivatives() { List> derivativesInDatabase = connector.retryWithHandle( handle -> - handle.createQuery( - StringUtils.format("SELECT DISTINCT dataSource,commit_metadata_payload FROM %1$s", dbTables.get().getDataSourceTable()) - ) - .map(new ResultSetMapper>() - { - @Override - public Pair map(int index, ResultSet r, StatementContext ctx) throws SQLException - { - String datasourceName = r.getString("dataSource"); - try { - DataSourceMetadata payload = objectMapper.readValue( - r.getBytes("commit_metadata_payload"), - DataSourceMetadata.class); - if (!(payload instanceof DerivativeDataSourceMetadata)) { - return null; - } - DerivativeDataSourceMetadata metadata = (DerivativeDataSourceMetadata) payload; - return new Pair<>(datasourceName, metadata); - } - catch (IOException e) { - throw new RuntimeException(e); - } + handle + .createQuery( + StringUtils.format( + "SELECT DISTINCT dataSource,commit_metadata_payload FROM %1$s", + dbTables.get().getDataSourceTable() + ) + ) + .map((int index, ResultSet r, StatementContext ctx) -> { + String datasourceName = r.getString("dataSource"); + DataSourceMetadata payload = JacksonUtils.readValue( + objectMapper, + r.getBytes("commit_metadata_payload"), + DataSourceMetadata.class); + if (!(payload instanceof DerivativeDataSourceMetadata)) { + return null; } + DerivativeDataSourceMetadata metadata = (DerivativeDataSourceMetadata) payload; + return new Pair<>(datasourceName, metadata); }) .list() ); @@ -240,21 +233,17 @@ public Long withHandle(Handle handle) ) .bind("dataSource", datasource) .map( - new ResultSetMapper() - { - @Override - public Object map(int index, ResultSet r, StatementContext ctx) throws SQLException - { - try { - intervals.add(Intervals.utc(DateTimes.of(r.getString("start")).getMillis(), DateTimes.of(r.getString("end")).getMillis())); - DataSegment segment = objectMapper.readValue(r.getBytes("payload"), DataSegment.class); - totalSize += segment.getSize(); - } - catch (IOException e) { - throw new RuntimeException(e); - } - return null; - } + (int index, ResultSet r, StatementContext ctx) -> { + intervals.add( + Intervals.utc( + DateTimes.of(r.getString("start")).getMillis(), + DateTimes.of(r.getString("end")).getMillis() + ) + ); + DataSegment segment = + JacksonUtils.readValue(objectMapper, r.getBytes("payload"), DataSegment.class); + totalSize += segment.getSize(); + return null; } ) .first(); diff --git a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java index ab575c3b183e..8056a1094fa9 100644 --- a/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java +++ b/extensions-contrib/tdigestsketch/src/test/java/org/apache/druid/query/aggregation/tdigestsketch/sql/TDigestSketchSqlAggregatorTest.java @@ -145,6 +145,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java index 63c50b3dcaa3..77bf27c7f531 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/hll/sql/HllSketchSqlAggregatorTest.java @@ -164,6 +164,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java index 7a6f699e0a92..340b8230675a 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/quantiles/sql/DoublesSketchSqlAggregatorTest.java @@ -158,6 +158,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java index a05e108e4424..c8ae6e368b39 100644 --- a/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java +++ b/extensions-core/datasketches/src/test/java/org/apache/druid/query/aggregation/datasketches/theta/sql/ThetaSketchSqlAggregatorTest.java @@ -161,6 +161,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java index 9f5c624bde15..ff068f4289ad 100644 --- a/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java +++ b/extensions-core/druid-bloom-filter/src/test/java/org/apache/druid/query/aggregation/bloom/sql/BloomFilterSqlAggregatorTest.java @@ -185,6 +185,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusherTest.java b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusherTest.java index 7cf7df535b56..a633a14cf868 100644 --- a/extensions-core/hdfs-storage/src/test/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusherTest.java +++ b/extensions-core/hdfs-storage/src/test/java/org/apache/druid/storage/hdfs/HdfsDataSegmentPusherTest.java @@ -449,7 +449,7 @@ public void shouldMakeHDFSCompliantSegmentOutputPath() null, new NumberedShardSpec(bucket.partitionNum, 5000), -1, - -1 + 0 ), JobHelper.INDEX_ZIP, hdfsDataSegmentPusher @@ -472,7 +472,7 @@ public void shouldMakeHDFSCompliantSegmentOutputPath() null, new NumberedShardSpec(bucket.partitionNum, 5000), -1, - -1 + 0 ), new TaskAttemptID("abc", 123, TaskType.REDUCE, 1, 0), hdfsDataSegmentPusher @@ -536,7 +536,7 @@ public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() null, new NumberedShardSpec(bucket.partitionNum, 5000), -1, - -1 + 0 ), JobHelper.INDEX_ZIP, new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig()) @@ -559,7 +559,7 @@ public void shouldMakeDefaultSegmentOutputPathIfNotHDFS() null, new NumberedShardSpec(bucket.partitionNum, 5000), -1, - -1 + 0 ), new TaskAttemptID("abc", 123, TaskType.REDUCE, 1, 0), new LocalDataSegmentPusher(new LocalDataSegmentPusherConfig()) diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java index 86e87bf0d475..bb9d3d048256 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/FixedBucketsHistogramQuantileSqlAggregatorTest.java @@ -156,6 +156,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java index 30f6474d5ae7..be90e8febb19 100644 --- a/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java +++ b/extensions-core/histogram/src/test/java/org/apache/druid/query/aggregation/histogram/sql/QuantileSqlAggregatorTest.java @@ -155,6 +155,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); diff --git a/extensions-core/kafka-indexing-service/pom.xml b/extensions-core/kafka-indexing-service/pom.xml index c469ce96f63c..bcde539f2603 100644 --- a/extensions-core/kafka-indexing-service/pom.xml +++ b/extensions-core/kafka-indexing-service/pom.xml @@ -188,6 +188,11 @@ 2.12.7 test + + org.assertj + assertj-core + test + 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 a121d6141447..c835bb950385 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 @@ -23,26 +23,16 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicates; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; -import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.curator.test.TestingCluster; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.LongDimensionSchema; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; @@ -52,9 +42,6 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; @@ -68,41 +55,33 @@ import org.apache.druid.indexing.common.stats.RowIngestionMetersTotals; import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisor; import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig; import org.apache.druid.indexing.kafka.test.TestBroker; import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner.Status; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTestBase; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisor; import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ListenableFutures; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultGenericQueryMetricsFactory; @@ -115,12 +94,7 @@ import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryToolChest; -import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.scan.ScanQuery; import org.apache.druid.query.scan.ScanQueryConfig; @@ -133,17 +107,10 @@ import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; -import org.apache.druid.query.timeseries.TimeseriesResultValue; -import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.QueryableIndex; -import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; @@ -152,12 +119,9 @@ import org.apache.druid.server.DruidNode; import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CompressionUtils; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.easymock.EasyMock; -import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; import org.junit.AfterClass; @@ -173,12 +137,7 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -191,19 +150,17 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; +@SuppressWarnings("unchecked") @RunWith(Parameterized.class) -public class KafkaIndexTaskTest +public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase { private static final Logger log = new Logger(KafkaIndexTaskTest.class); - private static final ObjectMapper OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); private static final long POLL_RETRY_MS = 100; private static TestingCluster zkServer; private static TestBroker kafkaServer; private static ServiceEmitter emitter; - private static ListeningExecutorService taskExec; private static int topicPostfix; static { @@ -219,9 +176,6 @@ public static Iterable constructorFeeder() ); } - private final List runningTasks = new ArrayList<>(); - private final LockGranularity lockGranularity; - private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; private boolean logParseExceptions = true; @@ -234,50 +188,11 @@ public static Iterable constructorFeeder() private Period intermediateHandoffPeriod = null; private AppenderatorsManager appenderatorsManager; - private TaskToolboxFactory toolboxFactory; - private IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private TaskStorage taskStorage; - private TaskLockbox taskLockbox; - private File directory; private String topic; private List> records; private final Set checkpointRequestsHash = new HashSet<>(); - private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); - private static List> generateRecords(String topic) { return ImmutableList.of( @@ -332,7 +247,7 @@ private static String getTopicName() public KafkaIndexTaskTest(LockGranularity lockGranularity) { - this.lockGranularity = lockGranularity; + super(lockGranularity); } @BeforeClass @@ -440,19 +355,18 @@ public void testRunAfterDataInserted() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -491,19 +405,18 @@ public void testRunBeforeDataInserted() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -629,16 +542,17 @@ public void testIncrementalHandOff() throws Exception Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - SegmentDescriptor desc4 = sd("2011/P1D", 0); - SegmentDescriptor desc5 = sd("2011/P1D", 1); - SegmentDescriptor desc6 = sd("2012/P1D", 0); - SegmentDescriptor desc7 = sd("2013/P1D", 0); + // Check published metadata and segments in deep storage assertEqualsExceptVersion( - ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e"), ImmutableList.of("d", "h")), + sdd("2011/P1D", 1, ImmutableList.of("h"), ImmutableList.of("e")), + sdd("2012/P1D", 0, ImmutableList.of("g")), + sdd("2013/P1D", 0, ImmutableList.of("f")) + ), publishedDescriptors() ); Assert.assertEquals( @@ -647,18 +561,6 @@ public void testIncrementalHandOff() throws Exception ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", publishedDescriptors.get(3))) - && ImmutableList.of("h").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))) || - (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", publishedDescriptors.get(3))) - && ImmutableList.of("e").equals(readSegmentColumn("dim1", publishedDescriptors.get(4))))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(5))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(6))); } @Test(timeout = 60_000L) @@ -773,16 +675,17 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - SegmentDescriptor desc4 = sd("2011/P1D", 0); - SegmentDescriptor desc5 = sd("2011/P1D", 1); - SegmentDescriptor desc6 = sd("2012/P1D", 0); - SegmentDescriptor desc7 = sd("2013/P1D", 0); + // Check published metadata and segments in deep storage assertEqualsExceptVersion( - ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e"), ImmutableList.of("d", "h")), + sdd("2011/P1D", 1, ImmutableList.of("h"), ImmutableList.of("e")), + sdd("2012/P1D", 0, ImmutableList.of("g")), + sdd("2013/P1D", 0, ImmutableList.of("f")) + ), publishedDescriptors() ); Assert.assertEquals( @@ -792,28 +695,12 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - assertEqualsExceptVersion( - ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), - publishedDescriptors() - ); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 10L, 1, 2L)) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", publishedDescriptors.get(3))) - && ImmutableList.of("h").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))) || - (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", publishedDescriptors.get(3))) - && ImmutableList.of("e").equals(readSegmentColumn("dim1", publishedDescriptors.get(4))))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(5))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(6))); } @Test(timeout = 60_000L) @@ -884,21 +771,20 @@ public void testTimeBasedIncrementalHandOff() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 2L, 1, 0L)) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -1038,19 +924,18 @@ public void testRunWithMinimumMessageTime() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - final List publishedDescriptors = publishedDescriptors(); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -1089,21 +974,19 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2010/P1D", 0, ImmutableList.of("c")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2))); } @Test(timeout = 60_000L) @@ -1151,15 +1034,14 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - SegmentDescriptor desc1 = sd("2009/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1), publishedDescriptors()); + final List publishedDescriptors = publishedDescriptors(); + assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(0))); Assert.assertEquals(ImmutableList.of("bb"), readSegmentColumn("dim1t", publishedDescriptors.get(0))); } @@ -1232,19 +1114,18 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -1281,21 +1162,20 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - final List publishedDescriptors = publishedDescriptors(); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L)) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -1380,11 +1260,10 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - SegmentDescriptor desc3 = sd("2013/P1D", 0); - SegmentDescriptor desc4 = sd("2049/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of(sdd("2010/P1D", 0), sdd("2011/P1D", 0), sdd("2013/P1D", 0), sdd("2049/P1D", 0)), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) @@ -1536,18 +1415,17 @@ public void testRunReplicas() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -1602,18 +1480,18 @@ public void testRunConflicting() throws Exception Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata, should all be from the first task - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); final List publishedDescriptors = publishedDescriptors(); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -1656,8 +1534,8 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); + 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(DATA_SCHEMA.getDataSource())); @@ -1674,17 +1552,10 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc3 = sd("2011/P1D", 1); - SegmentDescriptor desc4 = sd("2013/P1D", 0); + 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(DATA_SCHEMA.getDataSource())); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(3))); } @Test(timeout = 60_000L) @@ -1719,12 +1590,12 @@ public void testRunOneTaskTwoPartitions() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); + SegmentDescriptorAndExpectedDim1Values desc1 = sdd("2010/P1D", 0, ImmutableList.of("c")); + SegmentDescriptorAndExpectedDim1Values desc2 = sdd("2011/P1D", 0, ImmutableList.of("d", "e", "h")); // desc3 will not be created in KafkaIndexTask (0.12.x) as it does not create per Kafka partition Druid segments + @SuppressWarnings("unused") SegmentDescriptor desc3 = sd("2011/P1D", 1); - SegmentDescriptor desc4 = sd("2012/P1D", 0); - final List publishedDescriptors = publishedDescriptors(); + SegmentDescriptorAndExpectedDim1Values desc4 = sdd("2012/P1D", 0, ImmutableList.of("g")); assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc4), publishedDescriptors()); Assert.assertEquals( new KafkaDataSourceMetadata( @@ -1732,22 +1603,6 @@ public void testRunOneTaskTwoPartitions() throws Exception ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals( - ImmutableList.of("g"), - readSegmentColumn( - "dim1", - publishedDescriptors.get(2) - ) - ); - - // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically - Assert.assertEquals( - ImmutableSet.of(ImmutableList.of("d", "e", "h")), - ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1))) - ); } @Test(timeout = 60_000L) @@ -1801,22 +1656,20 @@ public void testRunTwoTasksTwoPartitions() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - SegmentDescriptor desc3 = sd("2012/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")), + sdd("2012/P1D", 0, ImmutableList.of("g")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L, 1, 1L)) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(2))); } @Test(timeout = 60_000L) @@ -1902,18 +1755,17 @@ public void testRestore() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -2012,15 +1864,16 @@ public void testRestoreAfterPersistingSequences() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2008/P1D", 1); - SegmentDescriptor desc3 = sd("2009/P1D", 0); - SegmentDescriptor desc4 = sd("2009/P1D", 1); - SegmentDescriptor desc5 = sd("2010/P1D", 0); - SegmentDescriptor desc6 = sd("2011/P1D", 0); - SegmentDescriptor desc7 = sd("2012/P1D", 0); assertEqualsExceptVersion( - ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), + ImmutableList.of( + sdd("2008/P1D", 0), + sdd("2008/P1D", 1), + sdd("2009/P1D", 0), + sdd("2009/P1D", 1), + sdd("2010/P1D", 0), + sdd("2011/P1D", 0), + sdd("2012/P1D", 0) + ), publishedDescriptors() ); Assert.assertEquals( @@ -2104,19 +1957,18 @@ public void testRunWithPauseAndResume() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - final List publishedDescriptors = publishedDescriptors(); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 6L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -2228,19 +2080,18 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - final List publishedDescriptors = publishedDescriptors(); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 5L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -2370,23 +2221,20 @@ public void testRunTransactionModeRollback() throws Exception Assert.assertEquals(3, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2013/P1D", 0); - SegmentDescriptor desc4 = sd("2049/P1D", 0); - final List publishedDescriptors = publishedDescriptors(); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2013/P1D", 0, ImmutableList.of("f")), + sdd("2049/P1D", 0, ImmutableList.of("f")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KafkaDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(topic, ImmutableMap.of(0, 13L))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(3))); } @Test(timeout = 60_000L) @@ -2479,46 +2327,6 @@ private void insertData() throws ExecutionException, InterruptedException } } - private ListenableFuture runTask(final Task task) - { - //noinspection CatchMayIgnoreException - try { - taskStorage.insert(task, TaskStatus.running(task.getId())); - } - catch (EntryExistsException e) { - // suppress - } - taskLockbox.syncFromStorage(); - final TaskToolbox toolbox = toolboxFactory.build(task); - synchronized (runningTasks) { - runningTasks.add(task); - } - return taskExec.submit( - () -> { - try { - task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); - if (task.isReady(toolbox.getTaskActionClient())) { - return task.run(toolbox); - } else { - throw new ISE("Task is not ready"); - } - } - catch (Exception e) { - log.warn(e, "Task failed"); - return TaskStatus.failure(task.getId()); - } - } - ); - } - - private TaskLock getLock(final Task task, final Interval interval) - { - return Iterables.find( - taskLockbox.findLocksForTask(task), - lock -> lock.getInterval().contains(interval) - ); - } - private KafkaIndexTask createTask( final String taskId, final KafkaIndexTaskIOConfig ioConfig @@ -2788,158 +2596,4 @@ public void close() null ); } - - private void destroyToolboxFactory() - { - toolboxFactory = null; - taskStorage = null; - taskLockbox = null; - metadataStorageCoordinator = null; - } - - private List publishedDescriptors() - { - return metadataStorageCoordinator.getUsedSegmentsForInterval( - DATA_SCHEMA.getDataSource(), - Intervals.of("0000/3000") - ).stream().map(DataSegment::toDescriptor).collect(Collectors.toList()); - } - - private void unlockAppenderatorBasePersistDirForTask(KafkaIndexTask task) - throws NoSuchMethodException, InvocationTargetException, IllegalAccessException - { - Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() - .getDeclaredMethod( - "unlockBasePersistDirectory"); - unlockBasePersistDir.setAccessible(true); - unlockBasePersistDir.invoke(task.getAppenderator()); - } - - private File getSegmentDirectory() - { - return new File(directory, "segments"); - } - - private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException - { - File indexBasePath = new File( - StringUtils.format( - "%s/%s/%s_%s/%s/%d", - getSegmentDirectory(), - DATA_SCHEMA.getDataSource(), - descriptor.getInterval().getStart(), - descriptor.getInterval().getEnd(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ) - ); - - File outputLocation = new File( - directory, - StringUtils.format( - "%s_%s_%s_%s", - descriptor.getInterval().getStart(), - descriptor.getInterval().getEnd(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ) - ); - outputLocation.mkdir(); - CompressionUtils.unzip( - Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), - outputLocation, - Predicates.alwaysFalse(), - false - ); - IndexIO indexIO = new TestUtils().getTestIndexIO(); - QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) - .getColumn(); - List values = new ArrayList<>(); - for (int i = 0; i < theColumn.length(); i++) { - int id = theColumn.getSingleValueRow(i); - String value = theColumn.lookupName(id); - values.add(value); - } - return values; - } - - private long countEvents(final Task task) - { - // Do a query. - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SCHEMA.getDataSource()) - .aggregators( - ImmutableList.of( - new LongSumAggregatorFactory("rows", "rows") - ) - ).granularity(Granularities.ALL) - .intervals("0000/3000") - .build(); - - List> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList(); - - return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); - } - - private static byte[] jb(String timestamp, String dim1, String dim2, String dimLong, String dimFloat, String met1) - { - try { - return new ObjectMapper().writeValueAsBytes( - ImmutableMap.builder() - .put("timestamp", timestamp) - .put("dim1", dim1) - .put("dim2", dim2) - .put("dimLong", dimLong) - .put("dimFloat", dimFloat) - .put("met1", met1) - .build() - ); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private SegmentDescriptor sd(final String intervalString, final int partitionNum) - { - final Interval interval = Intervals.of(intervalString); - return new SegmentDescriptor(interval, "fakeVersion", partitionNum); - } - - private void assertEqualsExceptVersion(List descriptors1, List descriptors2) - { - Assert.assertEquals(descriptors1.size(), descriptors2.size()); - final Comparator comparator = (s1, s2) -> { - final int intervalCompare = Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval()); - if (intervalCompare == 0) { - return Integer.compare(s1.getPartitionNumber(), s2.getPartitionNumber()); - } else { - return intervalCompare; - } - }; - - final List copy1 = new ArrayList<>(descriptors1); - final List copy2 = new ArrayList<>(descriptors2); - copy1.sort(comparator); - copy2.sort(comparator); - - for (int i = 0; i < copy1.size(); i++) { - Assert.assertEquals(copy1.get(i).getInterval(), copy2.get(i).getInterval()); - Assert.assertEquals(copy1.get(i).getPartitionNumber(), copy2.get(i).getPartitionNumber()); - } - } - - private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException - { - Map taskReports = OBJECT_MAPPER.readValue( - reportsFile, - new TypeReference>() - { - } - ); - return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( - taskReports - ); - } } diff --git a/extensions-core/kinesis-indexing-service/pom.xml b/extensions-core/kinesis-indexing-service/pom.xml index ee4fa353443d..133c7c35bc82 100644 --- a/extensions-core/kinesis-indexing-service/pom.xml +++ b/extensions-core/kinesis-indexing-service/pom.xml @@ -176,6 +176,11 @@ hamcrest-core test + + org.assertj + assertj-core + test + 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 81de1e2e9e58..274549891b92 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 @@ -24,31 +24,18 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonTypeName; import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Predicates; -import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.io.Files; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.client.cache.CachePopulatorStats; import org.apache.druid.client.cache.MapCache; import org.apache.druid.common.aws.AWSCredentialsConfig; -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.FloatDimensionSchema; -import org.apache.druid.data.input.impl.JSONParseSpec; -import org.apache.druid.data.input.impl.LongDimensionSchema; -import org.apache.druid.data.input.impl.StringDimensionSchema; -import org.apache.druid.data.input.impl.StringInputRowParser; -import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.LookupNodeService; @@ -58,9 +45,6 @@ import org.apache.druid.indexing.common.LockGranularity; import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter; -import org.apache.druid.indexing.common.TaskLock; -import org.apache.druid.indexing.common.TaskReport; -import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TaskToolboxFactory; import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; @@ -74,17 +58,15 @@ import org.apache.druid.indexing.common.task.IndexTaskTest; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.TaskResource; -import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisor; import org.apache.druid.indexing.overlord.DataSourceMetadata; -import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.MetadataTaskStorage; import org.apache.druid.indexing.overlord.TaskLockbox; -import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.indexing.overlord.supervisor.SupervisorManager; import org.apache.druid.indexing.seekablestream.SeekableStreamEndSequenceNumbers; import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskRunner; +import org.apache.druid.indexing.seekablestream.SeekableStreamIndexTaskTestBase; import org.apache.druid.indexing.seekablestream.SeekableStreamStartSequenceNumbers; import org.apache.druid.indexing.seekablestream.SequenceMetadata; import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord; @@ -93,55 +75,34 @@ import org.apache.druid.indexing.test.TestDataSegmentAnnouncer; import org.apache.druid.indexing.test.TestDataSegmentKiller; import org.apache.druid.java.util.common.DateTimes; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.concurrent.ListenableFutures; -import org.apache.druid.java.util.common.granularity.Granularities; -import org.apache.druid.java.util.common.guava.Comparators; -import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.metrics.MonitorScheduler; import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.metadata.DerbyMetadataStorageActionHandlerFactory; -import org.apache.druid.metadata.EntryExistsException; import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator; import org.apache.druid.metadata.TestDerbyConnector; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; -import org.apache.druid.query.Druids; import org.apache.druid.query.IntervalChunkingQueryRunnerDecorator; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryToolChest; -import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; -import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesQueryEngine; import org.apache.druid.query.timeseries.TimeseriesQueryQueryToolChest; import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; -import org.apache.druid.query.timeseries.TimeseriesResultValue; -import org.apache.druid.segment.DimensionHandlerUtils; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.column.DictionaryEncodedColumn; import org.apache.druid.segment.indexing.DataSchema; -import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.loading.DataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusher; import org.apache.druid.segment.loading.LocalDataSegmentPusherConfig; -import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.segment.realtime.plumber.SegmentHandoffNotifier; @@ -152,11 +113,7 @@ import org.apache.druid.server.coordination.DataSegmentServerAnnouncer; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.server.security.AuthorizerMapper; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.utils.CompressionUtils; import org.easymock.EasyMock; -import org.easymock.EasyMockSupport; -import org.joda.time.Interval; import org.joda.time.Period; import org.junit.After; import org.junit.AfterClass; @@ -172,13 +129,8 @@ import javax.annotation.Nullable; import java.io.File; import java.io.IOException; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -192,13 +144,12 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; -import java.util.stream.Collectors; +@SuppressWarnings("unchecked") @RunWith(Parameterized.class) -public class KinesisIndexTaskTest extends EasyMockSupport +public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase { - private static final Logger LOG = new Logger(KinesisIndexTaskTest.class); private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); private static final String STREAM = "stream"; private static final String SHARD_ID1 = "1"; @@ -207,9 +158,6 @@ public class KinesisIndexTaskTest extends EasyMockSupport private static List> records; private static ServiceEmitter emitter; - private static ListeningExecutorService taskExec; - - private final List runningTasks = new ArrayList<>(); @Parameterized.Parameters(name = "{0}") public static Iterable constructorFeeder() @@ -220,8 +168,6 @@ public static Iterable constructorFeeder() ); } - private final LockGranularity lockGranularity; - private long handoffConditionTimeout = 0; private boolean reportParseExceptions = false; private boolean logParseExceptions = true; @@ -234,48 +180,9 @@ public static Iterable constructorFeeder() private int maxRecordsPerPoll; private AppenderatorsManager appenderatorsManager; - private TaskToolboxFactory toolboxFactory; - private IndexerMetadataStorageCoordinator metadataStorageCoordinator; - private TaskStorage taskStorage; - private TaskLockbox taskLockbox; - private File directory; private final Set checkpointRequestsHash = new HashSet<>(); - private File reportsFile; private RowIngestionMetersFactory rowIngestionMetersFactory; - private static final DataSchema DATA_SCHEMA = new DataSchema( - "test_ds", - OBJECT_MAPPER.convertValue( - new StringInputRowParser( - new JSONParseSpec( - new TimestampSpec("timestamp", "iso", null), - new DimensionsSpec( - Arrays.asList( - new StringDimensionSchema("dim1"), - new StringDimensionSchema("dim1t"), - new StringDimensionSchema("dim2"), - new LongDimensionSchema("dimLong"), - new FloatDimensionSchema("dimFloat") - ), - null, - null - ), - new JSONPathSpec(true, ImmutableList.of()), - ImmutableMap.of() - ), - StandardCharsets.UTF_8.name() - ), - Map.class - ), - new AggregatorFactory[]{ - new DoubleSumAggregatorFactory("met1sum", "met1"), - new CountAggregatorFactory("rows") - }, - new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), - null, - OBJECT_MAPPER - ); - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); @@ -301,7 +208,7 @@ public static void setupClass() public KinesisIndexTaskTest(LockGranularity lockGranularity) { - this.lockGranularity = lockGranularity; + super(lockGranularity); } @Before @@ -351,16 +258,16 @@ public static void tearDownClass() throws Exception private static List> generateRecords(String stream) { return ImmutableList.of( - new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "0", jbl("2008", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "1", jbl("2009", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "2", jbl("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "3", jbl("2011", "d", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "4", jbl("2011", "e", "y", "10", "20.0", "1.0")), new OrderedPartitionableRecord<>( stream, "1", "5", - jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") + jbl("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0") ), new OrderedPartitionableRecord<>( stream, @@ -375,33 +282,33 @@ private static List> generateRecords( Collections.singletonList(StringUtils.toUtf8("unparseable2")) ), new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(StringUtils.toUtf8("{}"))), - new OrderedPartitionableRecord<>(stream, "1", "9", jb("2013", "f", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "10", jb("2049", "f", "y", "notanumber", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "11", jb("2049", "f", "y", "10", "notanumber", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "12", jb("2049", "f", "y", "10", "20.0", "notanumber")), - new OrderedPartitionableRecord<>(stream, "0", "0", jb("2012", "g", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "0", "1", jb("2011", "h", "y", "10", "20.0", "1.0")) + new OrderedPartitionableRecord<>(stream, "1", "9", jbl("2013", "f", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "10", jbl("2049", "f", "y", "notanumber", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "11", jbl("2049", "f", "y", "10", "notanumber", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "12", jbl("2049", "f", "y", "10", "20.0", "notanumber")), + new OrderedPartitionableRecord<>(stream, "0", "0", jbl("2012", "g", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "0", "1", jbl("2011", "h", "y", "10", "20.0", "1.0")) ); } private static List> generateSinglePartitionRecords(String stream) { return ImmutableList.of( - new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "3", jb("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "4", jb("2011", "e", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "5", jb("2012", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "6", jb("2013", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "7", jb("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "8", jb("2011", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "9", jb("2011", "e", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "10", jb("2008", "a", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "11", jb("2009", "b", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "12", jb("2010", "c", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "13", jb("2012", "d", "y", "10", "20.0", "1.0")), - new OrderedPartitionableRecord<>(stream, "1", "14", jb("2013", "e", "y", "10", "20.0", "1.0")) + new OrderedPartitionableRecord<>(stream, "1", "0", jbl("2008", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "1", jbl("2009", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "2", jbl("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "3", jbl("2011", "d", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "4", jbl("2011", "e", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "5", jbl("2012", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "6", jbl("2013", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "7", jbl("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "8", jbl("2011", "d", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "9", jbl("2011", "e", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "10", jbl("2008", "a", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "11", jbl("2009", "b", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "12", jbl("2010", "c", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "13", jbl("2012", "d", "y", "10", "20.0", "1.0")), + new OrderedPartitionableRecord<>(stream, "1", "14", jbl("2013", "e", "y", "10", "20.0", "1.0")) ); } @@ -455,21 +362,20 @@ public void testRunAfterDataInserted() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 120_000L) @@ -525,21 +431,20 @@ public void testRunBeforeDataInserted() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2011/P1D", 0); - SegmentDescriptor desc2 = sd("2012/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2011/P1D", 0, ImmutableList.of("h")), + sdd("2012/P1D", 0, ImmutableList.of("g")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID0, "1")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("h"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 120_000L) @@ -629,15 +534,19 @@ public void testIncrementalHandOff() throws Exception Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - SegmentDescriptor desc4 = sd("2011/P1D", 0); - SegmentDescriptor desc5 = sd("2011/P1D", 1); - SegmentDescriptor desc6 = sd("2012/P1D", 0); - SegmentDescriptor desc7 = sd("2013/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6, desc7), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e"), ImmutableList.of("d", "h")), + sdd("2011/P1D", 1, ImmutableList.of("h"), ImmutableList.of("e")), + sdd("2012/P1D", 0, ImmutableList.of("g")), + sdd("2013/P1D", 0, ImmutableList.of("f")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>( @@ -647,18 +556,6 @@ public void testIncrementalHandOff() throws Exception ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertTrue((ImmutableList.of("d", "e").equals(readSegmentColumn("dim1", publishedDescriptors.get(3))) - && ImmutableList.of("h").equals(readSegmentColumn("dim1", publishedDescriptors.get(4)))) || - (ImmutableList.of("d", "h").equals(readSegmentColumn("dim1", publishedDescriptors.get(3))) - && ImmutableList.of("e").equals(readSegmentColumn("dim1", publishedDescriptors.get(4))))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(5))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(6))); } @Test(timeout = 120_000L) @@ -777,27 +674,22 @@ public void testIncrementalHandOffMaxTotalRows() throws Exception Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - SegmentDescriptor desc4 = sd("2011/P1D", 0); - SegmentDescriptor desc5 = sd("2049/P1D", 0); - SegmentDescriptor desc7 = sd("2013/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc7), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")), + sdd("2049/P1D", 0, ImmutableList.of("f")), + sdd("2013/P1D", 0, ImmutableList.of("f")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "10"))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(3))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(4))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(5))); } @@ -856,18 +748,17 @@ public void testRunWithMinimumMessageTime() throws Exception Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -925,22 +816,20 @@ public void testRunWithMaximumMessageTime() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(2, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0, ImmutableList.of("a")), + sdd("2009/P1D", 0, ImmutableList.of("b")), + sdd("2010/P1D", 0, ImmutableList.of("c")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("a"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("b"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(2))); } @@ -1007,8 +896,7 @@ public void testRunWithTransformSpec() throws Exception Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - SegmentDescriptor desc1 = sd("2009/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1), publishedDescriptors()); + assertEqualsExceptVersion(ImmutableList.of(sdd("2009/P1D", 0)), publishedDescriptors()); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), @@ -1074,7 +962,7 @@ public void testRunOnSingletonRange() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - assertEqualsExceptVersion(ImmutableList.of(sd("2010/P1D", 0)), publishedDescriptors()); + assertEqualsExceptVersion(ImmutableList.of(sdd("2010/P1D", 0)), publishedDescriptors()); } @@ -1129,21 +1017,20 @@ public void testHandoffConditionTimeoutWhenHandoffOccurs() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1199,21 +1086,20 @@ public void testHandoffConditionTimeoutWhenHandoffDoesNotOccur() throws Exceptio Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1336,11 +1222,10 @@ public void testMultipleParseExceptionsSuccess() throws Exception Assert.assertEquals(4, task.getRunner().getRowIngestionMeters().getUnparseable()); // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - SegmentDescriptor desc3 = sd("2013/P1D", 0); - SegmentDescriptor desc4 = sd("2049/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of(sdd("2010/P1D", 0), sdd("2011/P1D", 0), sdd("2013/P1D", 0), sdd("2049/P1D", 0)), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "12")) @@ -1537,20 +1422,19 @@ public void testRunReplicas() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1630,18 +1514,17 @@ public void testRunConflicting() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata, should all be from the first task - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @@ -1708,8 +1591,8 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(TaskState.SUCCESS, future1.get().getStatusCode()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); + 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(DATA_SCHEMA.getDataSource())); @@ -1728,17 +1611,10 @@ public void testRunConflictingWithoutTransactions() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc3 = sd("2011/P1D", 1); - SegmentDescriptor desc4 = sd("2013/P1D", 0); + 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(DATA_SCHEMA.getDataSource())); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - Assert.assertEquals(ImmutableList.of("f"), readSegmentColumn("dim1", publishedDescriptors.get(3))); } @@ -1801,27 +1677,20 @@ public void testRunOneTaskTwoPartitions() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - SegmentDescriptor desc4 = sd("2012/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc4), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e", "h")), + sdd("2012/P1D", 0, ImmutableList.of("g")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(2))); - - // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically - Assert.assertEquals( - ImmutableSet.of(ImmutableList.of("d", "e", "h")), - ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1))) - ); } @@ -1900,28 +1769,22 @@ public void testRunTwoTasksTwoPartitions() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - SegmentDescriptor desc3 = sd("2011/P1D", 1); - SegmentDescriptor desc4 = sd("2012/P1D", 0); - - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + // These two partitions are interleaved nondeterministically so they both may contain ("d", "e") or "h" + sdd("2011/P1D", 0, ImmutableList.of("d", "e"), ImmutableList.of("h")), + sdd("2011/P1D", 1, ImmutableList.of("d", "e"), ImmutableList.of("h")), + sdd("2012/P1D", 0, ImmutableList.of("g")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4", SHARD_ID0, "1")) ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - // Check desc2/desc3 without strong ordering because two partitions are interleaved nondeterministically - Assert.assertEquals( - ImmutableSet.of(ImmutableList.of("d", "e"), ImmutableList.of("h")), - ImmutableSet.of(readSegmentColumn("dim1", publishedDescriptors.get(1)), readSegmentColumn("dim1", publishedDescriptors.get(2))) - ); - Assert.assertEquals(ImmutableList.of("g"), readSegmentColumn("dim1", publishedDescriptors.get(3))); } @@ -2032,20 +1895,19 @@ public void testRestore() throws Exception Assert.assertEquals(1, task2.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published segments & metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "5"))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 120_000L) @@ -2176,13 +2038,17 @@ public void testRestoreAfterPersistingSequences() throws Exception Assert.assertEquals(0, task2.getRunner().getRowIngestionMeters().getThrownAway()); // Check published segments & metadata - SegmentDescriptor desc1 = sd("2008/P1D", 0); - SegmentDescriptor desc2 = sd("2009/P1D", 0); - SegmentDescriptor desc3 = sd("2010/P1D", 0); - SegmentDescriptor desc4 = sd("2011/P1D", 0); - SegmentDescriptor desc5 = sd("2012/P1D", 0); - SegmentDescriptor desc6 = sd("2013/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2, desc3, desc4, desc5, desc6), publishedDescriptors()); + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2008/P1D", 0), + sdd("2009/P1D", 0), + sdd("2010/P1D", 0), + sdd("2011/P1D", 0), + sdd("2012/P1D", 0), + sdd("2013/P1D", 0) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "6")) @@ -2277,10 +2143,14 @@ public void testRunWithPauseAndResume() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>( STREAM, @@ -2288,11 +2158,6 @@ public void testRunWithPauseAndResume() throws Exception )), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 60_000L) @@ -2359,19 +2224,18 @@ public void testRunContextSequenceAheadOfStartingOffsets() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getThrownAway()); - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata(new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "4"))), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); } @Test(timeout = 5000L) @@ -2509,16 +2373,19 @@ public void testIncrementalHandOffReadsThroughEndOffsets() throws Exception Assert.assertEquals(0, normalReplica.getRunner().getRowIngestionMeters().getThrownAway()); // Check published metadata - final List descriptors = new ArrayList<>(); - descriptors.add(sd("2008/P1D", 0)); - descriptors.add(sd("2009/P1D", 0)); - descriptors.add(sd("2010/P1D", 0)); - descriptors.add(sd("2010/P1D", 1)); - descriptors.add(sd("2011/P1D", 0)); - descriptors.add(sd("2011/P1D", 1)); - descriptors.add(sd("2012/P1D", 0)); - descriptors.add(sd("2013/P1D", 0)); - assertEqualsExceptVersion(descriptors, publishedDescriptors()); + assertEqualsExceptVersion( + Arrays.asList( + sdd("2008/P1D", 0), + sdd("2009/P1D", 0), + sdd("2010/P1D", 0), + sdd("2010/P1D", 1), + sdd("2011/P1D", 0), + sdd("2011/P1D", 1), + sdd("2012/P1D", 0), + sdd("2013/P1D", 0) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>(STREAM, ImmutableMap.of(SHARD_ID1, "9")) @@ -2662,10 +2529,14 @@ public void testEndOfShard() throws Exception Assert.assertEquals(0, task.getRunner().getRowIngestionMeters().getUnparseable()); Assert.assertEquals(1, task.getRunner().getRowIngestionMeters().getThrownAway()); // EOS marker - // Check published metadata - SegmentDescriptor desc1 = sd("2010/P1D", 0); - SegmentDescriptor desc2 = sd("2011/P1D", 0); - assertEqualsExceptVersion(ImmutableList.of(desc1, desc2), publishedDescriptors()); + // Check published metadata and segments in deep storage + assertEqualsExceptVersion( + ImmutableList.of( + sdd("2010/P1D", 0, ImmutableList.of("c")), + sdd("2011/P1D", 0, ImmutableList.of("d", "e")) + ), + publishedDescriptors() + ); Assert.assertEquals( new KinesisDataSourceMetadata( new SeekableStreamEndSequenceNumbers<>( @@ -2675,51 +2546,6 @@ public void testEndOfShard() throws Exception ), metadataStorageCoordinator.getDataSourceMetadata(DATA_SCHEMA.getDataSource()) ); - - // Check segments in deep storage - final List publishedDescriptors = publishedDescriptors(); - Assert.assertEquals(ImmutableList.of("c"), readSegmentColumn("dim1", publishedDescriptors.get(0))); - Assert.assertEquals(ImmutableList.of("d", "e"), readSegmentColumn("dim1", publishedDescriptors.get(1))); - } - - private ListenableFuture runTask(final Task task) - { - try { - taskStorage.insert(task, TaskStatus.running(task.getId())); - } - catch (EntryExistsException e) { - // suppress - } - taskLockbox.syncFromStorage(); - final TaskToolbox toolbox = toolboxFactory.build(task); - synchronized (runningTasks) { - runningTasks.add(task); - } - return taskExec.submit( - () -> { - try { - task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); - if (task.isReady(toolbox.getTaskActionClient())) { - return task.run(toolbox); - } else { - throw new ISE("Task is not ready"); - } - } - catch (Throwable e) { - LOG.warn(e, "Task failed"); - return TaskStatus.failure(task.getId(), Throwables.getStackTraceAsString(e)); - } - } - ); - } - - - private TaskLock getLock(final Task task, final Interval interval) - { - return Iterables.find( - taskLockbox.findLocksForTask(task), - lock -> lock.getInterval().contains(interval) - ); } private KinesisIndexTask createTask( @@ -2997,169 +2823,6 @@ public void close() ); } - private void destroyToolboxFactory() - { - toolboxFactory = null; - taskStorage = null; - taskLockbox = null; - metadataStorageCoordinator = null; - } - - - private List publishedDescriptors() - { - return metadataStorageCoordinator.getUsedSegmentsForInterval( - DATA_SCHEMA.getDataSource(), - Intervals.of("0000/3000") - ).stream().map(DataSegment::toDescriptor).collect(Collectors.toList()); - } - - private void unlockAppenderatorBasePersistDirForTask(KinesisIndexTask task) - throws NoSuchMethodException, InvocationTargetException, IllegalAccessException - { - Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()).getClass() - .getDeclaredMethod( - "unlockBasePersistDirectory"); - unlockBasePersistDir.setAccessible(true); - unlockBasePersistDir.invoke(task.getAppenderator()); - } - - private File getSegmentDirectory() - { - return new File(directory, "segments"); - } - - - private List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException - { - File indexBasePath = new File( - StringUtils.format( - "%s/%s/%s_%s/%s/%d", - getSegmentDirectory(), - DATA_SCHEMA.getDataSource(), - descriptor.getInterval().getStart(), - descriptor.getInterval().getEnd(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ) - ); - - File outputLocation = new File( - directory, - StringUtils.format( - "%s_%s_%s_%s", - descriptor.getInterval().getStart(), - descriptor.getInterval().getEnd(), - descriptor.getVersion(), - descriptor.getPartitionNumber() - ) - ); - outputLocation.mkdir(); - CompressionUtils.unzip( - Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), - outputLocation, - Predicates.alwaysFalse(), - false - ); - IndexIO indexIO = new TestUtils().getTestIndexIO(); - QueryableIndex index = indexIO.loadIndex(outputLocation); - DictionaryEncodedColumn theColumn = (DictionaryEncodedColumn) index.getColumnHolder(column) - .getColumn(); - List values = new ArrayList<>(); - for (int i = 0; i < theColumn.length(); i++) { - int id = theColumn.getSingleValueRow(i); - String value = theColumn.lookupName(id); - values.add(value); - } - return values; - } - - public long countEvents(final Task task) - { - // Do a query. - TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() - .dataSource(DATA_SCHEMA.getDataSource()) - .aggregators( - ImmutableList.of( - new LongSumAggregatorFactory("rows", "rows") - ) - ).granularity(Granularities.ALL) - .intervals("0000/3000") - .build(); - - List> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList(); - - return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); - } - - private static List jb( - String timestamp, - String dim1, - String dim2, - String dimLong, - String dimFloat, - String met1 - ) - { - try { - return Collections.singletonList(new ObjectMapper().writeValueAsBytes( - ImmutableMap.builder() - .put("timestamp", timestamp) - .put("dim1", dim1) - .put("dim2", dim2) - .put("dimLong", dimLong) - .put("dimFloat", dimFloat) - .put("met1", met1) - .build() - )); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - - private SegmentDescriptor sd(final String intervalString, final int partitionNum) - { - final Interval interval = Intervals.of(intervalString); - return new SegmentDescriptor(interval, "fakeVersion", partitionNum); - } - - private void assertEqualsExceptVersion(List descriptors1, List descriptors2) - { - Assert.assertEquals(descriptors1.size(), descriptors2.size()); - final Comparator comparator = (s1, s2) -> { - final int intervalCompare = Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval()); - if (intervalCompare == 0) { - return Integer.compare(s1.getPartitionNumber(), s2.getPartitionNumber()); - } else { - return intervalCompare; - } - }; - - final List copy1 = new ArrayList<>(descriptors1); - final List copy2 = new ArrayList<>(descriptors2); - copy1.sort(comparator); - copy2.sort(comparator); - - for (int i = 0; i < copy1.size(); i++) { - Assert.assertEquals(copy1.get(i).getInterval(), copy2.get(i).getInterval()); - Assert.assertEquals(copy1.get(i).getPartitionNumber(), copy2.get(i).getPartitionNumber()); - } - } - - private IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException - { - Map taskReports = OBJECT_MAPPER.readValue( - reportsFile, - new TypeReference>() - { - } - ); - return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( - taskReports - ); - } - @JsonTypeName("index_kinesis") private static class TestableKinesisIndexTask extends KinesisIndexTask { diff --git a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java index 42e0135fc7cb..660ef35552b6 100644 --- a/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java +++ b/extensions-core/s3-extensions/src/test/java/org/apache/druid/storage/s3/S3DataSegmentArchiverTest.java @@ -91,6 +91,7 @@ public String getArchiveBaseKey() S3DataSegmentPuller.KEY, "source_key" )) + .size(0) .build(); @BeforeClass diff --git a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java index 50b7177392cd..8046f44c1279 100644 --- a/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java +++ b/extensions-core/stats/src/test/java/org/apache/druid/query/aggregation/variance/sql/VarianceSqlAggregatorTest.java @@ -150,6 +150,7 @@ public void setUp() throws Exception .interval(index.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index ); 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 d96dd51ecda8..7d3a04999aba 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 @@ -25,7 +25,8 @@ import org.apache.druid.common.utils.UUIDUtils; import org.apache.druid.indexer.hadoop.DatasourceIngestionSpec; import org.apache.druid.indexer.hadoop.WindowedDataSegment; -import org.apache.druid.indexer.path.UsedSegmentLister; +import org.apache.druid.indexer.path.UsedSegmentsRetriever; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.IngestionSpec; import org.apache.druid.timeline.DataSegment; @@ -38,6 +39,7 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -140,7 +142,7 @@ public HadoopIngestionSpec withTuningConfig(HadoopTuningConfig config) public static HadoopIngestionSpec updateSegmentListIfDatasourcePathSpecIsUsed( HadoopIngestionSpec spec, ObjectMapper jsonMapper, - UsedSegmentLister segmentLister + UsedSegmentsRetriever segmentsRetriever ) throws IOException { @@ -171,20 +173,21 @@ public static HadoopIngestionSpec updateSegmentListIfDatasourcePathSpecIsUsed( DatasourceIngestionSpec.class ); - List segmentsList = segmentLister.getUsedSegmentsForIntervals( + Collection usedVisibleSegments = segmentsRetriever.getUsedSegmentsForIntervals( ingestionSpecObj.getDataSource(), - ingestionSpecObj.getIntervals() + ingestionSpecObj.getIntervals(), + Segments.ONLY_VISIBLE ); if (ingestionSpecObj.getSegments() != null) { - //ensure that user supplied segment list matches with the segmentsList obtained from db + //ensure that user supplied segment list matches with the usedVisibleSegments obtained from db //this safety check lets users do test-n-set kind of batch delta ingestion where the delta //ingestion task would only run if current state of the system is same as when they submitted //the task. List userSuppliedSegmentsList = ingestionSpecObj.getSegments(); - if (segmentsList.size() == userSuppliedSegmentsList.size()) { - Set segmentsSet = new HashSet<>(segmentsList); + if (usedVisibleSegments.size() == userSuppliedSegmentsList.size()) { + Set segmentsSet = new HashSet<>(usedVisibleSegments); for (DataSegment userSegment : userSuppliedSegmentsList) { if (!segmentsSet.contains(userSegment)) { @@ -196,7 +199,8 @@ public static HadoopIngestionSpec updateSegmentListIfDatasourcePathSpecIsUsed( } } - final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(segmentsList); + final VersionedIntervalTimeline timeline = + VersionedIntervalTimeline.forSegments(usedVisibleSegments); final List windowedSegments = new ArrayList<>(); for (Interval interval : ingestionSpecObj.getIntervals()) { final List> timeLineSegments = timeline.lookup(interval); diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java index 5ee8d5e11de8..0861a915a7a3 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/IndexGeneratorJob.java @@ -821,7 +821,7 @@ public void doRun() metricNames, shardSpecForPublishing, -1, - -1 + 0 ); final DataSegment segment = JobHelper.serializeOutIndex( diff --git a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentLister.java b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java similarity index 75% rename from indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentLister.java rename to indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java index 5797ac6a423f..b4fe44f7bbcf 100644 --- a/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentLister.java +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/MetadataStoreBasedUsedSegmentsRetriever.java @@ -22,19 +22,21 @@ import com.google.common.base.Preconditions; import com.google.inject.Inject; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import java.util.Collection; import java.util.List; /** */ -public class MetadataStoreBasedUsedSegmentLister implements UsedSegmentLister +public class MetadataStoreBasedUsedSegmentsRetriever implements UsedSegmentsRetriever { private IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator; @Inject - public MetadataStoreBasedUsedSegmentLister(IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator) + public MetadataStoreBasedUsedSegmentsRetriever(IndexerMetadataStorageCoordinator indexerMetadataStorageCoordinator) { this.indexerMetadataStorageCoordinator = Preconditions.checkNotNull( indexerMetadataStorageCoordinator, @@ -43,8 +45,12 @@ public MetadataStoreBasedUsedSegmentLister(IndexerMetadataStorageCoordinator ind } @Override - public List getUsedSegmentsForIntervals(String dataSource, List intervals) + public Collection getUsedSegmentsForIntervals( + String dataSource, + List intervals, + Segments visibility + ) { - return indexerMetadataStorageCoordinator.getUsedSegmentsForIntervals(dataSource, intervals); + return indexerMetadataStorageCoordinator.getUsedSegmentsForIntervals(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 new file mode 100644 index 000000000000..428569f709c4 --- /dev/null +++ b/indexing-hadoop/src/main/java/org/apache/druid/indexer/path/UsedSegmentsRetriever.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexer.path; + +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.timeline.DataSegment; +import org.joda.time.Interval; + +import java.io.IOException; +import java.util.Collection; +import java.util.List; + +/** + */ +public interface UsedSegmentsRetriever +{ + /** + * Get 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. + * + * @param dataSource The datasource to query + * @param intervals The intervals for which used segments are to be returned + * @param visibility Whether only visible or visible as well as overshadowed segments should be returned. The + * visibility is considered within the specified intervals: that is, a segment which is visible + * outside of the specified intervals, but overshadowed on the specified intervals will not be + * returned if {@link Segments#ONLY_VISIBLE} is passed. See more precise description in the doc for + * {@link Segments}. + * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the + * 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. + */ + Collection getUsedSegmentsForIntervals(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 174d2817491c..dfe667e83880 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 @@ -29,7 +29,8 @@ import org.apache.druid.indexer.path.MultiplePathSpec; import org.apache.druid.indexer.path.PathSpec; import org.apache.druid.indexer.path.StaticPathSpec; -import org.apache.druid.indexer.path.UsedSegmentLister; +import org.apache.druid.indexer.path.UsedSegmentsRetriever; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -297,25 +298,27 @@ private HadoopDruidIndexerConfig testRunUpdateSegmentListIfDatasourcePathSpecIsU HadoopIngestionSpec.class ); - UsedSegmentLister segmentLister = EasyMock.createMock(UsedSegmentLister.class); + UsedSegmentsRetriever segmentsRetriever = EasyMock.createMock(UsedSegmentsRetriever.class); EasyMock.expect( - segmentLister.getUsedSegmentsForIntervals( + segmentsRetriever.getUsedSegmentsForIntervals( TEST_DATA_SOURCE, - Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL) : null) + Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL) : null), + Segments.ONLY_VISIBLE ) ).andReturn(ImmutableList.of(SEGMENT)); EasyMock.expect( - segmentLister.getUsedSegmentsForIntervals( + segmentsRetriever.getUsedSegmentsForIntervals( TEST_DATA_SOURCE2, - Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL2) : null) + Collections.singletonList(jobInterval != null ? jobInterval.overlap(TEST_DATA_SOURCE_INTERVAL2) : null), + Segments.ONLY_VISIBLE ) ).andReturn(ImmutableList.of(SEGMENT2)); - EasyMock.replay(segmentLister); + EasyMock.replay(segmentsRetriever); - spec = HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(spec, jsonMapper, segmentLister); + spec = HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed(spec, jsonMapper, segmentsRetriever); return HadoopDruidIndexerConfig.fromString(jsonMapper.writeValueAsString(spec)); } } diff --git a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java index a6535a4c54ff..d19e34211237 100644 --- a/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java +++ b/indexing-hadoop/src/test/java/org/apache/druid/indexer/path/DatasourcePathSpecTest.java @@ -154,8 +154,8 @@ public DatasourcePathSpecTest() @Test public void testSerde() throws Exception { - final UsedSegmentLister segmentList = EasyMock.createMock( - UsedSegmentLister.class + final UsedSegmentsRetriever segmentsRetriever = EasyMock.createMock( + UsedSegmentsRetriever.class ); Injector injector = Initialization.makeInjectorWithModules( @@ -166,7 +166,7 @@ public void testSerde() throws Exception @Override public void configure(Binder binder) { - binder.bind(UsedSegmentLister.class).toInstance(segmentList); + binder.bind(UsedSegmentsRetriever.class).toInstance(segmentsRetriever); JsonConfigProvider.bindInstance( binder, Key.get(DruidNode.class, Self.class), diff --git a/indexing-service/pom.xml b/indexing-service/pom.xml index 61ed2ea4d3f8..11be3ace6035 100644 --- a/indexing-service/pom.xml +++ b/indexing-service/pom.xml @@ -242,6 +242,11 @@ hamcrest-core test + + org.assertj + assertj-core + test + 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 96ce6aeae6d5..04a026b50fed 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 @@ -20,8 +20,9 @@ package org.apache.druid.indexing.appenderator; import com.google.common.collect.Iterables; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker; @@ -29,6 +30,7 @@ import org.joda.time.Interval; import java.io.IOException; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -62,8 +64,8 @@ public Set findUsedSegments(Set identifiers Iterables.transform(entry.getValue(), input -> input.getInterval()) ); - final List usedSegmentsForIntervals = taskActionClient.submit( - new SegmentListUsedAction(entry.getKey(), null, intervals) + final Collection usedSegmentsForIntervals = taskActionClient.submit( + new RetrieveUsedSegmentsAction(entry.getKey(), null, intervals, Segments.ONLY_VISIBLE) ); for (DataSegment segment : usedSegmentsForIntervals) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUnusedAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java similarity index 93% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUnusedAction.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java index 70c0dc5ef5c3..2c54106d4fd4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUnusedAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUnusedSegmentsAction.java @@ -29,7 +29,7 @@ import java.util.List; -public class SegmentListUnusedAction implements TaskAction> +public class RetrieveUnusedSegmentsAction implements TaskAction> { @JsonIgnore private final String dataSource; @@ -38,7 +38,7 @@ public class SegmentListUnusedAction implements TaskAction> private final Interval interval; @JsonCreator - public SegmentListUnusedAction( + public RetrieveUnusedSegmentsAction( @JsonProperty("dataSource") String dataSource, @JsonProperty("interval") Interval interval ) @@ -80,7 +80,7 @@ public boolean isAudited() @Override public String toString() { - return "SegmentListUnusedAction{" + + return getClass().getSimpleName() + "{" + "dataSource='" + dataSource + '\'' + ", interval=" + interval + '}'; diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java similarity index 57% rename from indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java rename to indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java index 461db0ae624f..60411195debf 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentListUsedAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsAction.java @@ -26,14 +26,28 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; +import javax.annotation.Nullable; import java.util.Collection; import java.util.List; +import java.util.Objects; -public class SegmentListUsedAction implements TaskAction> +/** + * This TaskAction returns a collection of segments which have data within the specified intervals 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. + * + * @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 + * a collection. Producing a {@link java.util.Set} would require an unnecessary copy of segments collection. + */ +public class RetrieveUsedSegmentsAction implements TaskAction> { @JsonIgnore private final String dataSource; @@ -41,11 +55,17 @@ public class SegmentListUsedAction implements TaskAction> @JsonIgnore private final List intervals; + @JsonIgnore + private final Segments visibility; + @JsonCreator - public SegmentListUsedAction( + public RetrieveUsedSegmentsAction( @JsonProperty("dataSource") String dataSource, @Deprecated @JsonProperty("interval") Interval interval, - @JsonProperty("intervals") Collection intervals + @JsonProperty("intervals") Collection intervals, + // When JSON object is deserialized, this parameter is optional for backward compatibility. + // Otherwise, it shouldn't be considered optional. + @JsonProperty("visibility") @Nullable Segments visibility ) { this.dataSource = dataSource; @@ -62,6 +82,9 @@ public SegmentListUsedAction( theIntervals = JodaUtils.condenseIntervals(intervals); } this.intervals = Preconditions.checkNotNull(theIntervals, "no intervals found"); + + // Defaulting to the former behaviour when visibility wasn't explicitly specified for backward compatibility + this.visibility = visibility != null ? visibility : Segments.ONLY_VISIBLE; } @JsonProperty @@ -76,16 +99,23 @@ public List getIntervals() return intervals; } + @JsonProperty + public Segments getVisibility() + { + return visibility; + } + @Override - public TypeReference> getReturnTypeReference() + public TypeReference> getReturnTypeReference() { - return new TypeReference>() {}; + return new TypeReference>() {}; } @Override - public List perform(Task task, TaskActionToolbox toolbox) + public Collection perform(Task task, TaskActionToolbox toolbox) { - return toolbox.getIndexerMetadataStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals); + return toolbox.getIndexerMetadataStorageCoordinator() + .getUsedSegmentsForIntervals(dataSource, intervals, visibility); } @Override @@ -104,29 +134,30 @@ public boolean equals(Object o) return false; } - SegmentListUsedAction that = (SegmentListUsedAction) o; + RetrieveUsedSegmentsAction that = (RetrieveUsedSegmentsAction) o; if (!dataSource.equals(that.dataSource)) { return false; } - return intervals.equals(that.intervals); - + if (!intervals.equals(that.intervals)) { + return false; + } + return visibility.equals(that.visibility); } @Override public int hashCode() { - int result = dataSource.hashCode(); - result = 31 * result + intervals.hashCode(); - return result; + return Objects.hash(dataSource, intervals, visibility); } @Override public String toString() { - return "SegmentListUsedAction{" + + return getClass().getSimpleName() + "{" + "dataSource='" + dataSource + '\'' + ", intervals=" + intervals + + ", visibility=" + visibility + '}'; } } 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 aecfcec36432..678144999e5c 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 @@ -30,6 +30,7 @@ import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.LockRequestForNewSegment; import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; @@ -189,7 +190,7 @@ public SegmentIdWithShardSpec perform( final Interval rowInterval = queryGranularity.bucket(timestamp); final Set usedSegmentsForRow = new HashSet<>( - msc.getUsedSegmentsForInterval(dataSource, rowInterval) + msc.getUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE) ); final SegmentIdWithShardSpec identifier = usedSegmentsForRow.isEmpty() ? @@ -208,7 +209,8 @@ public SegmentIdWithShardSpec perform( // overlapping with this row between when we called "mdc.getUsedSegmentsForInterval" and now. Check it again, // and if it's different, repeat. - if (!ImmutableSet.copyOf(msc.getUsedSegmentsForInterval(dataSource, rowInterval)).equals(usedSegmentsForRow)) { + if (!ImmutableSet.copyOf(msc.getUsedSegmentsForInterval(dataSource, rowInterval, Segments.ONLY_VISIBLE)) + .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/TaskAction.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java index e268188a464e..29be9c8dd809 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/TaskAction.java @@ -34,8 +34,10 @@ @JsonSubTypes.Type(name = "lockRelease", value = LockReleaseAction.class), @JsonSubTypes.Type(name = "segmentInsertion", value = SegmentInsertAction.class), @JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class), - @JsonSubTypes.Type(name = "segmentListUsed", value = SegmentListUsedAction.class), - @JsonSubTypes.Type(name = "segmentListUnused", value = SegmentListUnusedAction.class), + // Type name doesn't correspond to the name of the class for backward compatibility. + @JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class), + // Type name doesn't correspond to the name of the class for backward compatibility. + @JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class), @JsonSubTypes.Type(name = "segmentNuke", value = SegmentNukeAction.class), @JsonSubTypes.Type(name = "segmentMetadataUpdate", value = SegmentMetadataUpdateAction.class), @JsonSubTypes.Type(name = "segmentAllocate", value = SegmentAllocateAction.class), diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java index b8f48557ce2e..d32ddfb823a4 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AbstractBatchIndexTask.java @@ -20,6 +20,7 @@ package org.apache.druid.indexing.common.task; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.indexer.TaskStatus; @@ -29,7 +30,7 @@ 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.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; @@ -37,6 +38,7 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig; import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; import org.apache.druid.indexing.firehose.WindowedSegmentId; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularity; @@ -44,10 +46,9 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineObjectHolder; +import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; -import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpecFactory; import org.joda.time.Interval; import org.joda.time.Period; @@ -57,6 +58,7 @@ import javax.annotation.concurrent.GuardedBy; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -66,7 +68,6 @@ import java.util.SortedSet; import java.util.function.Consumer; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; /** * Abstract class for batch tasks like {@link IndexTask}. @@ -352,15 +353,12 @@ private LockGranularityDetermineResult determineSegmentGranularity(List segmentsToLock = timeline - .lookup(JodaUtils.umbrellaInterval(intervals)) - .stream() - .map(TimelineObjectHolder::getObject) - .flatMap(partitionHolder -> StreamSupport.stream(partitionHolder.spliterator(), false)) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); + Set segmentsToLock = timeline.findNonOvershadowedObjectsInInterval( + JodaUtils.umbrellaInterval(intervals), + Partitions.ONLY_COMPLETE + ); log.info("No segmentGranularity change detected and it's not perfect rollup. Using segment lock"); - return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, segmentsToLock); + return new LockGranularityDetermineResult(LockGranularity.SEGMENT, null, new ArrayList<>(segmentsToLock)); } } else { // Set useSegmentLock even though we don't get any locks. @@ -458,6 +456,9 @@ protected static Map> createShardSpecW * However, firehoseFactory is not IngestSegmentFirehoseFactory, it means this task will overwrite some segments * with data read from some input source outside of Druid. As a result, only the segments falling in intervalsToRead * should be locked. + * + * The order of segments within the returned list is unspecified, but each segment is guaranteed to appear in the list + * only once. */ protected static List findInputSegments( String dataSource, @@ -475,20 +476,22 @@ protected static List findInputSegments( "input interval" ); - return actionClient.submit( - new SegmentListUsedAction(dataSource, null, Collections.singletonList(inputInterval)) + return ImmutableList.copyOf( + actionClient.submit( + new RetrieveUsedSegmentsAction(dataSource, inputInterval, null, Segments.ONLY_VISIBLE) + ) ); } else { - final List inputSegmentIds = inputSegments.stream() - .map(WindowedSegmentId::getSegmentId) - .collect(Collectors.toList()); - final List dataSegmentsInIntervals = actionClient.submit( - new SegmentListUsedAction( + final List inputSegmentIds = + inputSegments.stream().map(WindowedSegmentId::getSegmentId).collect(Collectors.toList()); + final Collection dataSegmentsInIntervals = actionClient.submit( + new RetrieveUsedSegmentsAction( dataSource, null, inputSegments.stream() .flatMap(windowedSegmentId -> windowedSegmentId.getIntervals().stream()) - .collect(Collectors.toSet()) + .collect(Collectors.toSet()), + Segments.ONLY_VISIBLE ) ); return dataSegmentsInIntervals.stream() @@ -496,7 +499,11 @@ protected static List findInputSegments( .collect(Collectors.toList()); } } else { - return actionClient.submit(new SegmentListUsedAction(dataSource, null, intervalsToRead)); + return ImmutableList.copyOf( + actionClient.submit( + new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE) + ) + ); } } 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 01d2448a88db..a53e63106d73 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 @@ -25,7 +25,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; +import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; @@ -77,7 +77,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); + .submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { 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 115ff8103b08..16bedc3cac1e 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 @@ -50,7 +50,7 @@ import org.apache.druid.indexing.common.RetryPolicyFactory; import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexIOConfig; @@ -58,6 +58,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.Pair; @@ -83,7 +84,6 @@ import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; @@ -104,7 +104,6 @@ import java.util.TreeMap; import java.util.stream.Collectors; import java.util.stream.IntStream; -import java.util.stream.StreamSupport; public class CompactionTask extends AbstractBatchIndexTask { @@ -295,7 +294,9 @@ public boolean requireLockExistingSegments() public List findSegmentsToLock(TaskActionClient taskActionClient, List intervals) throws IOException { - return taskActionClient.submit(new SegmentListUsedAction(getDataSource(), null, intervals)); + return ImmutableList.copyOf( + taskActionClient.submit(new RetrieveUsedSegmentsAction(getDataSource(), null, intervals, Segments.ONLY_VISIBLE)) + ); } @Override @@ -784,17 +785,9 @@ static class SegmentProvider List checkAndGetSegments(TaskActionClient actionClient) throws IOException { - final List usedSegments = actionClient.submit( - new SegmentListUsedAction(dataSource, interval, null) + final List latestSegments = new ArrayList<>( + actionClient.submit(new RetrieveUsedSegmentsAction(dataSource, interval, null, Segments.ONLY_VISIBLE)) ); - final TimelineLookup timeline = VersionedIntervalTimeline.forSegments(usedSegments); - final List latestSegments = timeline - .lookup(interval) - .stream() - .map(TimelineObjectHolder::getObject) - .flatMap(partitionHolder -> StreamSupport.stream(partitionHolder.spliterator(), false)) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); if (!inputSpec.validateSegments(latestSegments)) { throw new ISE( 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 50261a375440..136a57a1220e 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 @@ -50,7 +50,7 @@ import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction; import org.apache.druid.indexing.common.config.TaskConfig; import org.apache.druid.indexing.common.stats.RowIngestionMeters; -import org.apache.druid.indexing.hadoop.OverlordActionBasedUsedSegmentLister; +import org.apache.druid.indexing.hadoop.OverlordActionBasedUsedSegmentsRetriever; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.JodaUtils; import org.apache.druid.java.util.common.StringUtils; @@ -322,7 +322,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception spec = HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed( spec, jsonMapper, - new OverlordActionBasedUsedSegmentLister(toolbox) + new OverlordActionBasedUsedSegmentsRetriever(toolbox) ); Object determinePartitionsInnerProcessingRunner = getForeignClassloaderObject( diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java index ea7ac46c984e..b7439d66de06 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/KillTask.java @@ -25,7 +25,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; +import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentNukeAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.actions.TaskLocks; @@ -79,7 +79,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(getDataSource(), getInterval())); + .submit(new RetrieveUnusedSegmentsAction(getDataSource(), getInterval())); if (!TaskLocks.isLockCoversSegments(taskLockMap, unusedSegments)) { throw new ISE( 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 6bdaef816c1a..047ec343c9ae 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 @@ -26,7 +26,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; +import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; @@ -85,7 +85,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); + .submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { 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 f3e98ae40289..4e3960a6dab6 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 @@ -25,7 +25,7 @@ import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUnusedAction; +import org.apache.druid.indexing.common.actions.RetrieveUnusedSegmentsAction; import org.apache.druid.indexing.common.actions.SegmentMetadataUpdateAction; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.logger.Logger; @@ -78,7 +78,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception // List unused segments final List unusedSegments = toolbox .getTaskActionClient() - .submit(new SegmentListUnusedAction(myLock.getDataSource(), myLock.getInterval())); + .submit(new RetrieveUnusedSegmentsAction(myLock.getDataSource(), myLock.getInterval())); // Verify none of these segments have versions > lock version for (final DataSegment unusedSegment : unusedSegments) { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java index c8211511fd02..8b59dabc2f37 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/firehose/IngestSegmentFirehoseFactory.java @@ -65,6 +65,7 @@ import java.io.File; import java.io.IOException; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -313,7 +314,7 @@ private List> getTimelineForInterval() // This call used to use the TaskActionClient, so for compatibility we use the same retry configuration // as TaskActionClient. final RetryPolicy retryPolicy = retryPolicyFactory.makeRetryPolicy(); - List usedSegments; + Collection usedSegments; while (true) { try { usedSegments = diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentLister.java b/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java similarity index 67% rename from indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentLister.java rename to indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java index c0ef8f768ff1..0ae705e1f30e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentLister.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/hadoop/OverlordActionBasedUsedSegmentsRetriever.java @@ -21,32 +21,38 @@ import com.google.common.base.Preconditions; import com.google.inject.Inject; -import org.apache.druid.indexer.path.UsedSegmentLister; +import org.apache.druid.indexer.path.UsedSegmentsRetriever; import org.apache.druid.indexing.common.TaskToolbox; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.timeline.DataSegment; import org.joda.time.Interval; import java.io.IOException; +import java.util.Collection; import java.util.List; /** */ -public class OverlordActionBasedUsedSegmentLister implements UsedSegmentLister +public class OverlordActionBasedUsedSegmentsRetriever implements UsedSegmentsRetriever { private final TaskToolbox toolbox; @Inject - public OverlordActionBasedUsedSegmentLister(TaskToolbox toolbox) + public OverlordActionBasedUsedSegmentsRetriever(TaskToolbox toolbox) { this.toolbox = Preconditions.checkNotNull(toolbox, "null task toolbox"); } @Override - public List getUsedSegmentsForIntervals(String dataSource, List intervals) throws IOException + public Collection getUsedSegmentsForIntervals( + String dataSource, + List intervals, + Segments visibility + ) throws IOException { return toolbox .getTaskActionClient() - .submit(new SegmentListUsedAction(dataSource, null, intervals)); + .submit(new RetrieveUsedSegmentsAction(dataSource, null, intervals, visibility)); } } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java index eb4d2e38f1d7..2be16ae57d30 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/ZkWorker.java @@ -31,6 +31,7 @@ import org.apache.druid.indexing.worker.TaskAnnouncement; import org.apache.druid.indexing.worker.Worker; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.joda.time.DateTime; import java.io.Closeable; @@ -60,19 +61,8 @@ public ZkWorker(Worker worker, PathChildrenCache statusCache, final ObjectMapper { this.worker = new AtomicReference<>(worker); this.statusCache = statusCache; - this.cacheConverter = new Function() - { - @Override - public TaskAnnouncement apply(ChildData input) - { - try { - return jsonMapper.readValue(input.getData(), TaskAnnouncement.class); - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - }; + this.cacheConverter = (ChildData input) -> + JacksonUtils.readValue(jsonMapper, input.getData(), TaskAnnouncement.class); } public void start() throws Exception diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java index 5dcb31f98f17..35a28be6cbdf 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/appenderator/ActionBasedUsedSegmentCheckerTest.java @@ -21,8 +21,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskActionClient; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker; @@ -43,7 +44,7 @@ public void testBasic() throws IOException final TaskActionClient taskActionClient = EasyMock.createMock(TaskActionClient.class); EasyMock.expect( taskActionClient.submit( - new SegmentListUsedAction("bar", null, ImmutableList.of(Intervals.of("2002/P1D"))) + new RetrieveUsedSegmentsAction("bar", Intervals.of("2002/P1D"), null, Segments.ONLY_VISIBLE) ) ).andReturn( ImmutableList.of( @@ -52,18 +53,25 @@ public void testBasic() throws IOException .interval(Intervals.of("2002/P1D")) .shardSpec(new LinearShardSpec(0)) .version("b") + .size(0) .build(), DataSegment.builder() .dataSource("bar") .interval(Intervals.of("2002/P1D")) .shardSpec(new LinearShardSpec(1)) .version("b") + .size(0) .build() ) ); EasyMock.expect( taskActionClient.submit( - new SegmentListUsedAction("foo", null, ImmutableList.of(Intervals.of("2000/P1D"), Intervals.of("2001/P1D"))) + new RetrieveUsedSegmentsAction( + "foo", + null, + ImmutableList.of(Intervals.of("2000/P1D"), Intervals.of("2001/P1D")), + Segments.ONLY_VISIBLE + ) ) ).andReturn( ImmutableList.of( @@ -72,24 +80,28 @@ public void testBasic() throws IOException .interval(Intervals.of("2000/P1D")) .shardSpec(new LinearShardSpec(0)) .version("a") + .size(0) .build(), DataSegment.builder() .dataSource("foo") .interval(Intervals.of("2000/P1D")) .shardSpec(new LinearShardSpec(1)) .version("a") + .size(0) .build(), DataSegment.builder() .dataSource("foo") .interval(Intervals.of("2001/P1D")) .shardSpec(new LinearShardSpec(1)) .version("b") + .size(0) .build(), DataSegment.builder() .dataSource("foo") .interval(Intervals.of("2002/P1D")) .shardSpec(new LinearShardSpec(1)) .version("b") + .size(0) .build() ) ); @@ -111,12 +123,14 @@ public void testBasic() throws IOException .interval(Intervals.of("2000/P1D")) .shardSpec(new LinearShardSpec(1)) .version("a") + .size(0) .build(), DataSegment.builder() .dataSource("bar") .interval(Intervals.of("2002/P1D")) .shardSpec(new LinearShardSpec(0)) .version("b") + .size(0) .build() ), segments diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java similarity index 89% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java index c5f677121edf..9f3d63823e78 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListActionsTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveSegmentsActionsTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -35,7 +36,7 @@ import java.util.HashSet; import java.util.Set; -public class SegmentListActionsTest +public class RetrieveSegmentsActionsTest { private static final Interval INTERVAL = Intervals.of("2017-10-01/2017-10-15"); @@ -94,21 +95,18 @@ private DataSegment createSegment(Interval interval, String version) } @Test - public void testSegmentListUsedAction() + public void testRetrieveUsedSegmentsAction() { - final SegmentListUsedAction action = new SegmentListUsedAction( - task.getDataSource(), - null, - ImmutableList.of(INTERVAL) - ); + final RetrieveUsedSegmentsAction action = + new RetrieveUsedSegmentsAction(task.getDataSource(), INTERVAL, null, Segments.ONLY_VISIBLE); final Set resultSegments = new HashSet<>(action.perform(task, actionTestKit.getTaskActionToolbox())); Assert.assertEquals(expectedUsedSegments, resultSegments); } @Test - public void testSegmentListUnusedAction() + public void testRetrieveUnusedSegmentsAction() { - final SegmentListUnusedAction action = new SegmentListUnusedAction(task.getDataSource(), INTERVAL); + final RetrieveUnusedSegmentsAction action = new RetrieveUnusedSegmentsAction(task.getDataSource(), INTERVAL); final Set resultSegments = new HashSet<>(action.perform(task, actionTestKit.getTaskActionToolbox())); Assert.assertEquals(expectedUnusedSegments, resultSegments); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsActionSerdeTest.java similarity index 69% rename from indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java rename to indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsActionSerdeTest.java index 3aa8319c30a1..0876092cac11 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentListUsedActionSerdeTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/RetrieveUsedSegmentsActionSerdeTest.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.segment.TestHelper; import org.joda.time.Interval; @@ -31,7 +32,7 @@ /** */ -public class SegmentListUsedActionSerdeTest +public class RetrieveUsedSegmentsActionSerdeTest { private static final ObjectMapper MAPPER = TestHelper.makeJsonMapper(); @@ -40,13 +41,11 @@ public void testSingleIntervalSerde() throws Exception { Interval interval = Intervals.of("2014/2015"); - SegmentListUsedAction expected = new SegmentListUsedAction( - "dataSource", - interval, - null - ); + RetrieveUsedSegmentsAction expected = + new RetrieveUsedSegmentsAction("dataSource", interval, null, Segments.ONLY_VISIBLE); - SegmentListUsedAction actual = MAPPER.readValue(MAPPER.writeValueAsString(expected), SegmentListUsedAction.class); + RetrieveUsedSegmentsAction actual = + MAPPER.readValue(MAPPER.writeValueAsString(expected), RetrieveUsedSegmentsAction.class); Assert.assertEquals(ImmutableList.of(interval), actual.getIntervals()); Assert.assertEquals(expected, actual); } @@ -55,13 +54,15 @@ public void testSingleIntervalSerde() throws Exception public void testMultiIntervalSerde() throws Exception { List intervals = ImmutableList.of(Intervals.of("2014/2015"), Intervals.of("2016/2017")); - SegmentListUsedAction expected = new SegmentListUsedAction( + RetrieveUsedSegmentsAction expected = new RetrieveUsedSegmentsAction( "dataSource", null, - intervals + intervals, + Segments.ONLY_VISIBLE ); - SegmentListUsedAction actual = MAPPER.readValue(MAPPER.writeValueAsString(expected), SegmentListUsedAction.class); + RetrieveUsedSegmentsAction actual = + MAPPER.readValue(MAPPER.writeValueAsString(expected), RetrieveUsedSegmentsAction.class); Assert.assertEquals(intervals, actual.getIntervals()); Assert.assertEquals(expected, actual); } @@ -70,8 +71,11 @@ public void testMultiIntervalSerde() throws Exception public void testOldJsonDeserialization() throws Exception { String jsonStr = "{\"type\": \"segmentListUsed\", \"dataSource\": \"test\", \"interval\": \"2014/2015\"}"; - SegmentListUsedAction actual = (SegmentListUsedAction) MAPPER.readValue(jsonStr, TaskAction.class); + RetrieveUsedSegmentsAction actual = (RetrieveUsedSegmentsAction) MAPPER.readValue(jsonStr, TaskAction.class); - Assert.assertEquals(new SegmentListUsedAction("test", Intervals.of("2014/2015"), null), actual); + Assert.assertEquals( + new RetrieveUsedSegmentsAction("test", Intervals.of("2014/2015"), null, Segments.ONLY_VISIBLE), + actual + ); } } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java index 5cabcf598cb6..7187bdbf07e3 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/SegmentAllocateActionTest.java @@ -631,12 +631,14 @@ public void testAddToExistingLinearShardSpecsSameGranularity() throws Exception .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new LinearShardSpec(1)) + .size(0) .build() ) ); @@ -694,12 +696,14 @@ public void testAddToExistingNumberedShardSpecsSameGranularity() throws Exceptio .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(0, 2)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(1, 2)) + .size(0) .build() ) ); @@ -755,12 +759,14 @@ public void testAddToExistingNumberedShardSpecsCoarserPreferredGranularity() thr .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(0, 2)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(1, 2)) + .size(0) .build() ) ); @@ -792,12 +798,14 @@ public void testAddToExistingNumberedShardSpecsFinerPreferredGranularity() throw .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(0, 2)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(1, 2)) + .size(0) .build() ) ); @@ -829,12 +837,14 @@ public void testCannotAddToExistingNumberedShardSpecsWithCoarserQueryGranularity .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(0, 2)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new NumberedShardSpec(1, 2)) + .size(0) .build() ) ); @@ -869,12 +879,14 @@ public void testCannotAddToExistingSingleDimensionShardSpecs() throws Exception .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new SingleDimensionShardSpec("foo", null, "bar", 0)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new SingleDimensionShardSpec("foo", "bar", null, 1)) + .size(0) .build() ) ); @@ -933,12 +945,14 @@ public void testWithShardSpecFactoryAndOvershadowingSegments() throws IOExceptio .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new HashBasedNumberedShardSpec(0, 2, ImmutableList.of("dim1"), objectMapper)) + .size(0) .build(), DataSegment.builder() .dataSource(DATA_SOURCE) .interval(Granularities.HOUR.bucket(PARTY_TIME)) .version(PARTY_TIME.toString()) .shardSpec(new HashBasedNumberedShardSpec(1, 2, ImmutableList.of("dim1"), objectMapper)) + .size(0) .build() ) ); 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 9fc905c278e5..a79b3ba327fc 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 @@ -27,6 +27,7 @@ import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.CriticalAction; import org.apache.druid.indexing.overlord.LockResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TimeChunkLockRequest; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; @@ -121,7 +122,7 @@ public void testSimple() throws Exception ImmutableSet.of(SEGMENT1, SEGMENT2), ImmutableSet.copyOf( actionTestKit.getMetadataStorageCoordinator() - .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL) + .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE) ) ); } 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 42417d680b06..f371deb597aa 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 @@ -28,6 +28,7 @@ import org.apache.druid.indexing.overlord.LockResult; import org.apache.druid.indexing.overlord.ObjectMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TimeChunkLockRequest; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.timeline.DataSegment; @@ -125,7 +126,7 @@ public void testTransactional() throws Exception ImmutableSet.of(SEGMENT1, SEGMENT2), ImmutableSet.copyOf( actionTestKit.getMetadataStorageCoordinator() - .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL) + .getUsedSegmentsForInterval(DATA_SOURCE, INTERVAL, Segments.ONLY_VISIBLE) ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java index 8b8f11ee8cf9..cc3a02546523 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/actions/TaskLocksTest.java @@ -75,18 +75,21 @@ private Set createTimeChunkedSegments() .interval(Intervals.of("2017-01-01/2017-01-02")) .version(DateTimes.nowUtc().toString()) .shardSpec(new LinearShardSpec(2)) + .size(0) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) .interval(Intervals.of("2017-01-02/2017-01-03")) .version(DateTimes.nowUtc().toString()) .shardSpec(new LinearShardSpec(2)) + .size(0) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) .interval(Intervals.of("2017-01-03/2017-01-04")) .version(DateTimes.nowUtc().toString()) .shardSpec(new LinearShardSpec(2)) + .size(0) .build() ); } @@ -100,30 +103,35 @@ private Set createNumberedPartitionedSegments() .interval(Intervals.of("2017-01-01/2017-01-02")) .version(version) .shardSpec(new NumberedShardSpec(0, 0)) + .size(0) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) .interval(Intervals.of("2017-01-01/2017-01-02")) .version(version) .shardSpec(new NumberedShardSpec(1, 0)) + .size(0) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) .interval(Intervals.of("2017-01-01/2017-01-02")) .version(version) .shardSpec(new NumberedShardSpec(2, 0)) + .size(0) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) .interval(Intervals.of("2017-01-01/2017-01-02")) .version(version) .shardSpec(new NumberedShardSpec(3, 0)) + .size(0) .build(), new DataSegment.Builder() .dataSource(task.getDataSource()) .interval(Intervals.of("2017-01-01/2017-01-02")) .version(version) .shardSpec(new NumberedShardSpec(4, 0)) + .size(0) .build() ); } 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 330fed26701d..6217dc1fa187 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 @@ -38,6 +38,7 @@ import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexSupervisorTask; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.SinglePhaseParallelIndexingTest.TestSupervisorTask; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.granularity.Granularities; @@ -64,6 +65,7 @@ import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -94,9 +96,9 @@ public CompactionTaskParallelRunTest(LockGranularity lockGranularity) coordinatorClient = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) { - return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals); + return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals, Segments.ONLY_VISIBLE); } @Override 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 9a7fa51bad8e..6deef192aa27 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 @@ -42,6 +42,7 @@ import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; import org.apache.druid.indexing.common.task.CompactionTask.Builder; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -83,6 +84,7 @@ import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; @@ -159,9 +161,9 @@ public CompactionTaskRunTest(LockGranularity lockGranularity) coordinatorClient = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) { - return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals); + return getStorageCoordinator().getUsedSegmentsForIntervals(dataSource, intervals, Segments.ONLY_VISIBLE); } }; segmentLoaderFactory = new SegmentLoaderFactory(getIndexIO(), getObjectMapper()); @@ -497,7 +499,8 @@ public void testCompactThenAppend() throws Exception final Set usedSegments = new HashSet<>( getStorageCoordinator().getUsedSegmentsForIntervals( DATA_SOURCE, - Collections.singletonList(Intervals.of("2014-01-01/2014-01-02")) + 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 3253fdc4f672..5f25458f4bd4 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 @@ -26,6 +26,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import org.apache.druid.client.coordinator.CoordinatorClient; @@ -50,7 +51,7 @@ import org.apache.druid.indexing.common.SegmentLoaderFactory; import org.apache.druid.indexing.common.TaskToolbox; import org.apache.druid.indexing.common.TestUtils; -import org.apache.druid.indexing.common.actions.SegmentListUsedAction; +import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction; import org.apache.druid.indexing.common.actions.TaskAction; import org.apache.druid.indexing.common.actions.TaskActionClient; import org.apache.druid.indexing.common.stats.RowIngestionMetersFactory; @@ -123,6 +124,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; @@ -1116,9 +1118,9 @@ private static class TestCoordinatorClient extends CoordinatorClient } @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) { - return new ArrayList<>(segmentMap.keySet()); + return ImmutableSet.copyOf(segmentMap.keySet()); } } @@ -1189,7 +1191,7 @@ private static class TestTaskActionClient implements TaskActionClient @Override public RetType submit(TaskAction taskAction) { - if (!(taskAction instanceof SegmentListUsedAction)) { + if (!(taskAction instanceof RetrieveUsedSegmentsAction)) { throw new ISE("action[%s] is not supported", taskAction); } return (RetType) segments; 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/KillTaskTest.java index 322c649be0e7..4db77719b198 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/KillTaskTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.apache.druid.indexer.TaskState; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskRunner; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; @@ -75,18 +76,19 @@ public void testKill() throws Exception Assert.assertEquals(TaskState.SUCCESS, taskRunner.run(task).get().getStatusCode()); - final List unusedSegments = getMetadataStorageCoordinator().getUnusedSegmentsForInterval( - DATA_SOURCE, - Intervals.of("2019/2020") - ); + final List unusedSegments = + getMetadataStorageCoordinator().getUnusedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020")); Assert.assertEquals(ImmutableList.of(newSegment(Intervals.of("2019-02-01/2019-03-01"), version)), unusedSegments); Assert.assertEquals( - ImmutableList.of( + ImmutableSet.of( newSegment(Intervals.of("2019-01-01/2019-02-01"), version), newSegment(Intervals.of("2019-04-01/2019-05-01"), version) ), - getMetadataStorageCoordinator().getUsedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020")) + ImmutableSet.copyOf( + getMetadataStorageCoordinator() + .getUsedSegmentsForInterval(DATA_SOURCE, Intervals.of("2019/2020"), Segments.ONLY_VISIBLE) + ) ); } 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 0c17b8c77abf..26747cbcc10a 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,6 +20,7 @@ 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.FiniteFirehoseFactory; import org.apache.druid.data.input.InputSplit; @@ -31,6 +32,7 @@ import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.TestAppenderatorsManager; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; @@ -41,8 +43,8 @@ import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; -import org.apache.druid.timeline.partition.PartitionChunk; import org.joda.time.Interval; import org.junit.After; import org.junit.Assert; @@ -57,12 +59,13 @@ import java.io.Writer; import java.nio.charset.StandardCharsets; import java.nio.file.Files; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.stream.Collectors; +import java.util.Set; @RunWith(Parameterized.class) public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSupervisorTaskTest @@ -187,21 +190,19 @@ private void testRunAndOverwrite(@Nullable Interval inputInterval, Granularity s runTestTask(inputInterval, Granularities.DAY); final Interval interval = inputInterval == null ? Intervals.ETERNITY : inputInterval; - final List allSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + final Collection oldSegments = + getStorageCoordinator().getUsedSegmentsForInterval("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 List newSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); - allSegments.addAll(newSegments); + final Collection newSegments = + getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE); + Set allSegments = ImmutableSet.builder().addAll(oldSegments).addAll(newSegments).build(); final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(allSegments); - final List visibles = timeline.lookup(interval) - .stream() - .flatMap(holder -> holder.getObject().stream()) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); - Assert.assertEquals(newSegments, visibles); + final Set visibles = timeline.findNonOvershadowedObjectsInInterval(interval, Partitions.ONLY_COMPLETE); + Assert.assertEquals(new HashSet<>(newSegments), visibles); } @Test @@ -329,18 +330,16 @@ public void testAppendToExisting() throws Exception { final Interval interval = Intervals.of("2017/2018"); runTestTask(interval, Granularities.DAY, true); - final List oldSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + final Collection oldSegments = + getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE); runTestTask(interval, Granularities.DAY, true); - final List newSegments = getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval); + final Collection newSegments = + getStorageCoordinator().getUsedSegmentsForInterval("dataSource", interval, Segments.ONLY_VISIBLE); Assert.assertTrue(newSegments.containsAll(oldSegments)); final VersionedIntervalTimeline timeline = VersionedIntervalTimeline.forSegments(newSegments); - final List visibles = timeline.lookup(interval) - .stream() - .flatMap(holder -> holder.getObject().stream()) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); - Assert.assertEquals(newSegments, visibles); + final Set visibles = timeline.findNonOvershadowedObjectsInInterval(interval, Partitions.ONLY_COMPLETE); + Assert.assertEquals(new HashSet<>(newSegments), visibles); } private ParallelIndexSupervisorTask newTask(@Nullable Interval interval, ParallelIndexIOConfig ioConfig) 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 55754ffbdd2d..37d7fcee9858 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 @@ -50,6 +50,7 @@ import org.apache.druid.indexing.common.task.NoopTask; import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.indexing.overlord.TaskLockbox; import org.apache.druid.indexing.overlord.TaskStorage; import org.apache.druid.java.util.common.IOE; @@ -141,13 +142,11 @@ public class IngestSegmentFirehoseFactoryTest private final Set published = new HashSet<>(); @Override - public List getUsedSegmentsForInterval(String dataSource, Interval interval) - { - return ImmutableList.copyOf(SEGMENT_SET); - } - - @Override - public List getUsedSegmentsForIntervals(String dataSource, List interval) + public List getUsedSegmentsForIntervals( + String dataSource, + List interval, + Segments visibility + ) { return ImmutableList.copyOf(SEGMENT_SET); } @@ -212,9 +211,9 @@ public static Collection constructorFeeder() throws IOException final CoordinatorClient cc = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) { - return ImmutableList.copyOf(SEGMENT_SET); + 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 ec250c4217d3..e29f0b985fed 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 @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.io.Files; import org.apache.commons.io.FileUtils; @@ -321,11 +322,11 @@ public static Collection constructorFeeder() final CoordinatorClient cc = new CoordinatorClient(null, null) { @Override - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) { // Expect the interval we asked for if (intervals.equals(ImmutableList.of(testCase.interval))) { - return ImmutableList.copyOf(testCase.segments); + return ImmutableSet.copyOf(testCase.segments); } else { throw new IllegalArgumentException("WTF"); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java index 40141412e8ab..7b74ac048049 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/RealtimeishTask.java @@ -103,6 +103,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception .dataSource("foo") .interval(interval1) .version(lock1.getVersion()) + .size(0) .build() ) ); @@ -122,6 +123,7 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception .dataSource("foo") .interval(interval2) .version(lock2.getVersion()) + .size(0) .build() ) ); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java index 74c61d563eef..5e4099eac7b0 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLifecycleTest.java @@ -958,11 +958,13 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception throw new ISE("Failed to get a lock"); } - final DataSegment segment = DataSegment.builder() - .dataSource("ds") - .interval(interval) - .version(lock.getVersion()) - .build(); + final DataSegment segment = DataSegment + .builder() + .dataSource("ds") + .interval(interval) + .version(lock.getVersion()) + .size(0) + .build(); toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); @@ -997,11 +999,13 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); - final DataSegment segment = DataSegment.builder() - .dataSource("ds") - .interval(Intervals.of("2012-01-01/P2D")) - .version(myLock.getVersion()) - .build(); + final DataSegment segment = DataSegment + .builder() + .dataSource("ds") + .interval(Intervals.of("2012-01-01/P2D")) + .version(myLock.getVersion()) + .size(0) + .build(); toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); @@ -1037,11 +1041,13 @@ public TaskStatus run(TaskToolbox toolbox) throws Exception { final TaskLock myLock = Iterables.getOnlyElement(toolbox.getTaskActionClient().submit(new LockListAction())); - final DataSegment segment = DataSegment.builder() - .dataSource("ds") - .interval(Intervals.of("2012-01-01/P1D")) - .version(myLock.getVersion() + "1!!!1!!") - .build(); + final DataSegment segment = DataSegment + .builder() + .dataSource("ds") + .interval(Intervals.of("2012-01-01/P1D")) + .version(myLock.getVersion() + "1!!!1!!") + .size(0) + .build(); toolbox.getTaskActionClient().submit(new SegmentInsertAction(ImmutableSet.of(segment))); return TaskStatus.success(getId()); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java new file mode 100644 index 000000000000..b8c2715b1384 --- /dev/null +++ b/indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskTestBase.java @@ -0,0 +1,404 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.seekablestream; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Predicates; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.io.Files; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.data.input.impl.FloatDimensionSchema; +import org.apache.druid.data.input.impl.JSONParseSpec; +import org.apache.druid.data.input.impl.LongDimensionSchema; +import org.apache.druid.data.input.impl.StringDimensionSchema; +import org.apache.druid.data.input.impl.StringInputRowParser; +import org.apache.druid.data.input.impl.TimestampSpec; +import org.apache.druid.indexer.TaskStatus; +import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReportData; +import org.apache.druid.indexing.common.LockGranularity; +import org.apache.druid.indexing.common.TaskReport; +import org.apache.druid.indexing.common.TaskToolbox; +import org.apache.druid.indexing.common.TaskToolboxFactory; +import org.apache.druid.indexing.common.TestUtils; +import org.apache.druid.indexing.common.task.Task; +import org.apache.druid.indexing.common.task.Tasks; +import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.Segments; +import org.apache.druid.indexing.overlord.TaskLockbox; +import org.apache.druid.indexing.overlord.TaskStorage; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.granularity.Granularities; +import org.apache.druid.java.util.common.guava.Comparators; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.common.parsers.JSONPathSpec; +import org.apache.druid.metadata.EntryExistsException; +import org.apache.druid.query.Druids; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.Result; +import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.query.aggregation.AggregatorFactory; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; +import org.apache.druid.query.aggregation.LongSumAggregatorFactory; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.query.timeseries.TimeseriesResultValue; +import org.apache.druid.segment.DimensionHandlerUtils; +import org.apache.druid.segment.IndexIO; +import org.apache.druid.segment.QueryableIndex; +import org.apache.druid.segment.column.DictionaryEncodedColumn; +import org.apache.druid.segment.indexing.DataSchema; +import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; +import org.apache.druid.segment.realtime.appenderator.AppenderatorImpl; +import org.apache.druid.timeline.DataSegment; +import org.apache.druid.utils.CompressionUtils; +import org.assertj.core.api.Assertions; +import org.easymock.EasyMockSupport; +import org.joda.time.Interval; +import org.junit.Assert; + +import java.io.File; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +public class SeekableStreamIndexTaskTestBase extends EasyMockSupport +{ + protected static final ObjectMapper OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); + protected static final DataSchema DATA_SCHEMA = new DataSchema( + "test_ds", + OBJECT_MAPPER.convertValue( + new StringInputRowParser( + new JSONParseSpec( + new TimestampSpec("timestamp", "iso", null), + new DimensionsSpec( + Arrays.asList( + new StringDimensionSchema("dim1"), + new StringDimensionSchema("dim1t"), + new StringDimensionSchema("dim2"), + new LongDimensionSchema("dimLong"), + new FloatDimensionSchema("dimFloat") + ), + null, + null + ), + new JSONPathSpec(true, ImmutableList.of()), + ImmutableMap.of() + ), + StandardCharsets.UTF_8.name() + ), + Map.class + ), + new AggregatorFactory[]{ + new DoubleSumAggregatorFactory("met1sum", "met1"), + new CountAggregatorFactory("rows") + }, + new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), + null, + OBJECT_MAPPER + ); + protected static final Logger LOG = new Logger(SeekableStreamIndexTaskTestBase.class); + protected static ListeningExecutorService taskExec; + + protected final List runningTasks = new ArrayList<>(); + protected final LockGranularity lockGranularity; + protected File directory; + protected File reportsFile; + protected TaskToolboxFactory toolboxFactory; + protected TaskStorage taskStorage; + protected TaskLockbox taskLockbox; + protected IndexerMetadataStorageCoordinator metadataStorageCoordinator; + + public SeekableStreamIndexTaskTestBase( + LockGranularity lockGranularity + ) + { + this.lockGranularity = lockGranularity; + } + + protected static byte[] jb( + String timestamp, + String dim1, + String dim2, + String dimLong, + String dimFloat, + String met1 + ) + { + try { + return new ObjectMapper().writeValueAsBytes( + ImmutableMap.builder() + .put("timestamp", timestamp) + .put("dim1", dim1) + .put("dim2", dim2) + .put("dimLong", dimLong) + .put("dimFloat", dimFloat) + .put("met1", met1) + .build() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + protected static List jbl( + String timestamp, + String dim1, + String dim2, + String dimLong, + String dimFloat, + String met1 + ) + { + return Collections.singletonList(jb(timestamp, dim1, dim2, dimLong, dimFloat, met1)); + } + + protected File getSegmentDirectory() + { + return new File(directory, "segments"); + } + + protected List readSegmentColumn(final String column, final SegmentDescriptor descriptor) throws IOException + { + File indexBasePath = new File( + StringUtils.format( + "%s/%s/%s_%s/%s/%d", + getSegmentDirectory(), + DATA_SCHEMA.getDataSource(), + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + + File outputLocation = new File( + directory, + StringUtils.format( + "%s_%s_%s_%s", + descriptor.getInterval().getStart(), + descriptor.getInterval().getEnd(), + descriptor.getVersion(), + descriptor.getPartitionNumber() + ) + ); + outputLocation.mkdir(); + CompressionUtils.unzip( + Files.asByteSource(new File(indexBasePath.listFiles()[0], "index.zip")), + outputLocation, + Predicates.alwaysFalse(), + false + ); + IndexIO indexIO = new TestUtils().getTestIndexIO(); + QueryableIndex index = indexIO.loadIndex(outputLocation); + DictionaryEncodedColumn theColumn = + (DictionaryEncodedColumn) index.getColumnHolder(column).getColumn(); + List values = new ArrayList<>(); + for (int i = 0; i < theColumn.length(); i++) { + int id = theColumn.getSingleValueRow(i); + String value = theColumn.lookupName(id); + values.add(value); + } + return values; + } + + protected SegmentDescriptor sd(final String intervalString, final int partitionNum) + { + final Interval interval = Intervals.of(intervalString); + return new SegmentDescriptor(interval, "fakeVersion", partitionNum); + } + + protected void assertEqualsExceptVersion( + List expectedDescriptors, + List actualDescriptors + ) throws IOException + { + Assert.assertEquals(expectedDescriptors.size(), actualDescriptors.size()); + final Comparator comparator = (s1, s2) -> { + final int intervalCompare = Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval()); + if (intervalCompare == 0) { + return Integer.compare(s1.getPartitionNumber(), s2.getPartitionNumber()); + } else { + return intervalCompare; + } + }; + + final List expectedDescsCopy = new ArrayList<>(expectedDescriptors); + final List actualDescsCopy = new ArrayList<>(actualDescriptors); + expectedDescsCopy.sort( + Comparator.comparing(SegmentDescriptorAndExpectedDim1Values::getSegmentDescriptor, comparator) + ); + actualDescsCopy.sort(comparator); + + for (int i = 0; i < expectedDescsCopy.size(); i++) { + SegmentDescriptorAndExpectedDim1Values expectedDesc = expectedDescsCopy.get(i); + SegmentDescriptor actualDesc = actualDescsCopy.get(i); + Assert.assertEquals( + expectedDesc.segmentDescriptor.getInterval(), + actualDesc.getInterval() + ); + Assert.assertEquals( + expectedDesc.segmentDescriptor.getPartitionNumber(), + actualDesc.getPartitionNumber() + ); + if (expectedDesc.expectedDim1Values.isEmpty()) { + continue; // Treating empty expectedDim1Values as a signal that checking the dim1 column value is not needed. + } + Assertions.assertThat(readSegmentColumn("dim1", actualDesc)).isIn(expectedDesc.expectedDim1Values); + } + } + + /** "sdd" stands for "Segment Descriptor and expected Dim1 values" */ + protected SegmentDescriptorAndExpectedDim1Values sdd( + String interval, + int partitionNum, + List... expectedDim1Values + ) + { + return new SegmentDescriptorAndExpectedDim1Values(interval, partitionNum, expectedDim1Values); + } + + protected IngestionStatsAndErrorsTaskReportData getTaskReportData() throws IOException + { + Map taskReports = OBJECT_MAPPER.readValue( + reportsFile, + new TypeReference>() + { + } + ); + return IngestionStatsAndErrorsTaskReportData.getPayloadFromTaskReports( + taskReports + ); + } + + protected ListenableFuture runTask(final Task task) + { + try { + taskStorage.insert(task, TaskStatus.running(task.getId())); + } + catch (EntryExistsException e) { + // suppress + } + taskLockbox.syncFromStorage(); + final TaskToolbox toolbox = toolboxFactory.build(task); + synchronized (runningTasks) { + runningTasks.add(task); + } + return taskExec.submit( + () -> { + try { + task.addToContext(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockGranularity == LockGranularity.TIME_CHUNK); + if (task.isReady(toolbox.getTaskActionClient())) { + return task.run(toolbox); + } else { + throw new ISE("Task is not ready"); + } + } + catch (Throwable e) { + LOG.warn(e, "Task failed"); + return TaskStatus.failure(task.getId(), Throwables.getStackTraceAsString(e)); + } + } + ); + } + + protected long countEvents(final Task task) + { + // Do a query. + TimeseriesQuery query = Druids.newTimeseriesQueryBuilder() + .dataSource(DATA_SCHEMA.getDataSource()) + .aggregators( + ImmutableList.of( + new LongSumAggregatorFactory("rows", "rows") + ) + ).granularity(Granularities.ALL) + .intervals("0000/3000") + .build(); + + List> results = task.getQueryRunner(query).run(QueryPlus.wrap(query)).toList(); + + return results.isEmpty() ? 0L : DimensionHandlerUtils.nullToZero(results.get(0).getValue().getLongMetric("rows")); + } + + protected void unlockAppenderatorBasePersistDirForTask(SeekableStreamIndexTask task) + throws NoSuchMethodException, InvocationTargetException, IllegalAccessException + { + Method unlockBasePersistDir = ((AppenderatorImpl) task.getAppenderator()) + .getClass() + .getDeclaredMethod("unlockBasePersistDirectory"); + unlockBasePersistDir.setAccessible(true); + unlockBasePersistDir.invoke(task.getAppenderator()); + } + + protected List publishedDescriptors() + { + return metadataStorageCoordinator + .getUsedSegmentsForInterval(DATA_SCHEMA.getDataSource(), Intervals.of("0000/3000"), Segments.ONLY_VISIBLE) + .stream() + .map(DataSegment::toDescriptor) + .collect(Collectors.toList()); + } + + protected void destroyToolboxFactory() + { + toolboxFactory = null; + taskStorage = null; + taskLockbox = null; + metadataStorageCoordinator = null; + } + + protected class SegmentDescriptorAndExpectedDim1Values + { + final SegmentDescriptor segmentDescriptor; + final Set> expectedDim1Values; + + protected SegmentDescriptorAndExpectedDim1Values( + String interval, + int partitionNum, + List... expectedDim1Values + ) + { + segmentDescriptor = sd(interval, partitionNum); + this.expectedDim1Values = ImmutableSet.copyOf(Arrays.asList(expectedDim1Values)); + } + + public SegmentDescriptor getSegmentDescriptor() + { + return segmentDescriptor; + } + } +} diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java index 1013f2210fb0..d270ea81ee95 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/test/TestIndexerMetadataStorageCoordinator.java @@ -26,6 +26,7 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Pair; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; @@ -73,12 +74,6 @@ public boolean insertDataSourceMetadata(String dataSource, DataSourceMetadata da { return false; } - - @Override - public List getUsedSegmentsForInterval(String dataSource, Interval interval) - { - return ImmutableList.of(); - } @Override public List> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval) @@ -87,7 +82,7 @@ public List> getUsedSegmentAndCreatedDateForInterval(S } @Override - public List getUsedSegmentsForIntervals(String dataSource, List intervals) + public List getUsedSegmentsForIntervals(String dataSource, List intervals, Segments visibility) { return ImmutableList.of(); } diff --git a/pom.xml b/pom.xml index 4f1fb16064b9..cfde7f5f59d0 100644 --- a/pom.xml +++ b/pom.xml @@ -1038,6 +1038,12 @@ test + + org.assertj + assertj-core + 3.13.2 + test + com.ircclouds.irc irc-api diff --git a/server/pom.xml b/server/pom.xml index 518d0e930a2e..4be28f7c3c80 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -416,6 +416,11 @@ httpcore test + + org.assertj + assertj-core + test + diff --git a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java index 48cabb6e16e9..6cf33bbe48a3 100644 --- a/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java +++ b/server/src/main/java/org/apache/druid/client/coordinator/CoordinatorClient.java @@ -35,6 +35,7 @@ import javax.annotation.Nullable; import javax.ws.rs.core.MediaType; +import java.util.Collection; import java.util.List; public class CoordinatorClient @@ -126,7 +127,7 @@ public List fetchServerView(String dataSource, Interva } } - public List getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) + public Collection getDatabaseSegmentDataSourceSegments(String dataSource, List intervals) { try { StringFullResponseHolder response = druidLeaderClient.go( diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java index 9eacec56bebe..97205a8981e4 100644 --- a/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/indexing/overlord/IndexerMetadataStorageCoordinator.java @@ -27,6 +27,7 @@ import javax.annotation.Nullable; import java.io.IOException; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Set; @@ -36,41 +37,66 @@ public interface IndexerMetadataStorageCoordinator { /** - * Get all segments which may include any data in the interval and are flagged as used. + * Get all published segments which may include any data in the interval and are marked as used. * - * @param dataSource The datasource to query - * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, end is exclusive - * - * @return The DataSegments which include data in the requested interval. These segments may contain data outside the requested interval. + * The order of segments within the returned collection is unspecified, but each segment is guaranteed to appear in + * the collection only once. * - * @throws IOException + * @param dataSource The datasource to query + * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, + * end is exclusive + * @param visibility Whether only visible or visible as well as overshadowed segments should be returned. The + * visibility is considered within the specified interval: that is, a segment which is visible + * outside of the specified interval, but overshadowed within the specified interval will not be + * returned if {@link Segments#ONLY_VISIBLE} is passed. See more precise description in the doc for + * {@link Segments}. + * @return The DataSegments which include data in the requested interval. These segments may contain data outside the + * requested interval. + * + * @implNote This method doesn't return a {@link Set} because there may be an expectation that {@code Set.contains()} + * is O(1) operation, while it's not the case for the returned collection unless it copies all segments into a new + * {@link java.util.HashSet} or {@link com.google.common.collect.ImmutableSet} which may in turn be unnecessary in + * other use cases. So clients should perform such copy themselves if they need {@link Set} semantics. */ - default List getUsedSegmentsForInterval(String dataSource, Interval interval) + default Collection getUsedSegmentsForInterval(String dataSource, Interval interval, Segments visibility) { - return getUsedSegmentsForIntervals(dataSource, Collections.singletonList(interval)); + return getUsedSegmentsForIntervals(dataSource, Collections.singletonList(interval), visibility); } /** - * Get all used segments and the created_date of these segments in a given datasource and interval + * Get all published segments which are marked as used and the created_date of these segments in a given datasource + * and interval. * * @param dataSource The datasource to query - * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, end is exclusive + * @param interval The interval for which all applicable and used datasources are requested. Start is inclusive, + * end is exclusive * * @return The DataSegments and the related created_date of segments which include data in the requested interval */ - List> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval); + Collection> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval); /** - * Get all segments which may include any data in the interval and are flagged as used. + * Get all published 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. * * @param dataSource The datasource to query * @param intervals The intervals for which all applicable and used datasources are requested. - * - * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the requested interval. - * - * @throws IOException + * @param visibility Whether only visible or visible as well as overshadowed segments should be returned. The + * visibility is considered within the specified intervals: that is, a segment which is visible + * outside of the specified intervals, but overshadowed on the specified intervals will not be + * returned if {@link Segments#ONLY_VISIBLE} is passed. See more precise description in the doc for + * {@link Segments}. + * @return The DataSegments which include data in the requested intervals. These segments may contain data outside the + * requested interval. + * + * @implNote This method doesn't return a {@link Set} because there may be an expectation that {@code Set.contains()} + * is O(1) operation, while it's not the case for the returned collection unless it copies all segments into a new + * {@link java.util.HashSet} or {@link com.google.common.collect.ImmutableSet} which may in turn be unnecessary in + * other use cases. So clients should perform such copy themselves if they need {@link Set} semantics. */ - List getUsedSegmentsForIntervals(String dataSource, List intervals); + Collection getUsedSegmentsForIntervals(String dataSource, List intervals, Segments visibility); /** * Attempts to insert a set of segments to the metadata storage. Returns the set of segments actually added (segments @@ -192,12 +218,14 @@ SegmentPublishResult announceHistoricalSegments( void deleteSegments(Set segments); /** - * Get all segments which include ONLY data within the given interval and are not flagged as used. + * Get all published segments which include ONLY data within the given interval and are not marked as used. * * @param dataSource The datasource the segments belong to - * @param interval Filter the data segments to ones that include data in this interval exclusively. Start is inclusive, end is exclusive + * @param interval Filter the data segments to ones that include data in this interval exclusively. Start is + * inclusive, end is exclusive * - * @return DataSegments which include ONLY data within the requested interval and are not flagged as used. Data segments NOT returned here may include data in the interval + * @return DataSegments which include ONLY data within the requested interval and are not marked as used. + * Data segments NOT returned here may include data in the interval */ List getUnusedSegmentsForInterval(String dataSource, Interval interval); } diff --git a/server/src/main/java/org/apache/druid/indexing/overlord/Segments.java b/server/src/main/java/org/apache/druid/indexing/overlord/Segments.java new file mode 100644 index 000000000000..5346d0ae08d9 --- /dev/null +++ b/server/src/main/java/org/apache/druid/indexing/overlord/Segments.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.indexing.overlord; + +/** + * This enum is used as a parameter for several methods in {@link IndexerMetadataStorageCoordinator}, specifying whether + * only visible segments, or visible as well as overshadowed segments should be included in results. + * + * Visibility (and overshadowness - these terms are antonyms) may be defined on an interval (or a series of intervals). + * Consider the following example: + * + * |----| I + * |----| S' + * |-------| S + * + * Here, I denotes an interval in question, S and S' are segments. S' is newer (has a higher version) than S. + * Segment S is overshadowed (by S') on the interval I, though it's visible (non-overshadowed) outside of I: more + * specifically, it's visible on the interval [end of S', end of S]. + * + * A segment is considered visible on a series of intervals if it's visible on any of the intervals in the series. A + * segment is considered (fully) overshadowed on a series of intervals if it's overshadowed (= non-visible) on all of + * the intervals in the series. + * + * If not specified otherwise, visibility (or overshadowness) should be assumed on the interval (-inf, +inf). + */ +public enum Segments +{ + /** Specifies that only visible segments should be included in results. */ + ONLY_VISIBLE, + /** Specifies that visible as well as overshadowed segments should be included in results. */ + INCLUDING_OVERSHADOWED +} diff --git a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java index 44e2196ce301..dbb690ecc15c 100644 --- a/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java +++ b/server/src/main/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinator.java @@ -32,16 +32,19 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NoneShardSpec; @@ -59,22 +62,20 @@ import org.skife.jdbi.v2.TransactionStatus; import org.skife.jdbi.v2.exceptions.CallbackFailedException; import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; import org.skife.jdbi.v2.util.ByteArrayMapper; import org.skife.jdbi.v2.util.StringMapper; import javax.annotation.Nullable; import java.io.IOException; import java.sql.ResultSet; -import java.sql.SQLException; import java.util.ArrayList; +import java.util.Collection; import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.stream.Collectors; import java.util.stream.StreamSupport; /** @@ -115,24 +116,21 @@ public void start() } @Override - public List getUsedSegmentsForIntervals(final String dataSource, final List intervals) + public Collection getUsedSegmentsForIntervals( + final String dataSource, + final List intervals, + Segments visibility + ) { return connector.retryWithHandle( handle -> { - final VersionedIntervalTimeline timeline = getTimelineForIntervalsWithHandle( - handle, - dataSource, - intervals - ); - - return intervals - .stream() - .flatMap((Interval interval) -> timeline.lookup(interval).stream()) - .flatMap(timelineObjectHolder -> { - return StreamSupport.stream(timelineObjectHolder.getObject().payloads().spliterator(), false); - }) - .distinct() - .collect(Collectors.toList()); + if (visibility == Segments.ONLY_VISIBLE) { + final VersionedIntervalTimeline timeline = + getTimelineForIntervalsWithHandle(handle, dataSource, intervals); + return timeline.findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE); + } else { + return getAllUsedSegmentsForIntervalsWithHandle(handle, dataSource, intervals); + } } ); } @@ -177,6 +175,36 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi final String dataSource, final List intervals ) + { + Query> sql = createUsedSegmentsSqlQueryForIntervals(handle, dataSource, intervals); + + try (final ResultIterator dbSegments = sql.map(ByteArrayMapper.FIRST).iterator()) { + return VersionedIntervalTimeline.forSegments( + Iterators.transform(dbSegments, payload -> JacksonUtils.readValue(jsonMapper, payload, DataSegment.class)) + ); + } + } + + private Collection getAllUsedSegmentsForIntervalsWithHandle( + final Handle handle, + final String dataSource, + final List intervals + ) + { + return createUsedSegmentsSqlQueryForIntervals(handle, dataSource, intervals) + .map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), DataSegment.class)) + .list(); + } + + /** + * Creates a query to the metadata store which selects payload from the segments table for all segments which are + * marked as used and whose interval intersects (not just abuts) with any of the intervals given to this method. + */ + private Query> createUsedSegmentsSqlQueryForIntervals( + Handle handle, + String dataSource, + List intervals + ) { if (intervals == null || intervals.isEmpty()) { throw new IAE("null/empty intervals"); @@ -186,7 +214,7 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi sb.append("SELECT payload FROM %s WHERE used = true AND dataSource = ? AND ("); for (int i = 0; i < intervals.size(); i++) { sb.append( - StringUtils.format("(start <= ? AND %1$send%1$s >= ?)", connector.getQuoteString()) + StringUtils.format("(start < ? AND %1$send%1$s > ?)", connector.getQuoteString()) ); if (i == intervals.size() - 1) { sb.append(")"); @@ -208,22 +236,7 @@ private VersionedIntervalTimeline getTimelineForIntervalsWi .bind(2 * i + 1, interval.getEnd().toString()) .bind(2 * i + 2, interval.getStart().toString()); } - - try (final ResultIterator dbSegments = sql.map(ByteArrayMapper.FIRST).iterator()) { - return VersionedIntervalTimeline.forSegments( - Iterators.transform( - dbSegments, - payload -> { - try { - return jsonMapper.readValue(payload, DataSegment.class); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - ) - ); - } + return sql; } /** @@ -840,12 +853,7 @@ public DataSourceMetadata getDataSourceMetadata(final String dataSource) return null; } - try { - return jsonMapper.readValue(bytes, DataSourceMetadata.class); - } - catch (Exception e) { - throw new RuntimeException(e); - } + return JacksonUtils.readValue(jsonMapper, bytes, DataSourceMetadata.class); } /** @@ -1141,13 +1149,8 @@ public List fold( StatementContext statementContext ) { - try { - accumulator.add(jsonMapper.readValue(payload, DataSegment.class)); - return accumulator; - } - catch (Exception e) { - throw new RuntimeException(e); - } + accumulator.add(JacksonUtils.readValue(jsonMapper, payload, DataSegment.class)); + return accumulator; } } ); @@ -1160,7 +1163,10 @@ public List fold( } @Override - public List> getUsedSegmentAndCreatedDateForInterval(String dataSource, Interval interval) + public Collection> getUsedSegmentAndCreatedDateForInterval( + String dataSource, + Interval interval + ) { return connector.retryWithHandle( handle -> handle.createQuery( @@ -1173,23 +1179,10 @@ public List> getUsedSegmentAndCreatedDateForInterval(S .bind("dataSource", dataSource) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) - .map(new ResultSetMapper>() - { - @Override - public Pair map(int index, ResultSet r, StatementContext ctx) - throws SQLException - { - try { - return new Pair<>( - jsonMapper.readValue(r.getBytes("payload"), DataSegment.class), - r.getString("created_date") - ); - } - catch (IOException e) { - throw new RuntimeException(e); - } - } - }) + .map((int index, ResultSet r, StatementContext ctx) -> new Pair<>( + JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), DataSegment.class), + r.getString("created_date") + )) .list() ); } diff --git a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java index 550dd49b0bf9..fb623acce329 100644 --- a/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/MetadataSegmentManager.java @@ -115,8 +115,7 @@ int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) /** * Retrieves all data source names for which there are segment in the database, regardless of whether those segments - * are used or not. Data source names in the returned collection are unique. If there are no segments in the database, - * returns an empty collection. + * are used or not. If there are no segments in the database, returns an empty set. * * Performance warning: this method makes a query into the database. * @@ -125,7 +124,7 @@ int markAsUsedNonOvershadowedSegments(String dataSource, Set segmentIds) * are no used segments belonging to it, while {@link #getImmutableDataSourcesWithAllUsedSegments} won't return * such a data source. */ - Collection retrieveAllDataSourceNames(); + Set retrieveAllDataSourceNames(); /** * Returns top N unused segment intervals with the end time no later than the specified maxEndTime when ordered by diff --git a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java index 4f22478cd36a..368918043be1 100644 --- a/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java +++ b/server/src/main/java/org/apache/druid/metadata/SQLMetadataSegmentManager.java @@ -39,6 +39,7 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -65,6 +66,7 @@ import java.sql.SQLException; import java.util.ArrayList; import java.util.Collection; +import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; @@ -503,20 +505,17 @@ private int doMarkAsUsedNonOvershadowedSegments(String dataSourceName, @Nullable } query = query .map((int index, ResultSet resultSet, StatementContext context) -> { - try { - DataSegment segment = jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); - if (resultSet.getBoolean("used")) { - usedSegmentsOverlappingInterval.add(segment); - } else { - if (interval == null || interval.contains(segment.getInterval())) { - unusedSegmentsInInterval.add(segment); - } + DataSegment segment = + JacksonUtils.readValue(jsonMapper, resultSet.getBytes("payload"), DataSegment.class); + if (resultSet.getBoolean("used")) { + usedSegmentsOverlappingInterval.add(segment); + } else { + if (interval == null || interval.contains(segment.getInterval())) { + unusedSegmentsInInterval.add(segment); } - return null; - } - catch (IOException e) { - throw new RuntimeException(e); } + //noinspection ReturnOfNull: intentional, consume() call below doesn't use the results. + return null; }); // Consume the query results to ensure usedSegmentsOverlappingInterval and unusedSegmentsInInterval are // populated. @@ -670,14 +669,9 @@ private Iterator retrieveUsedSegmentsOverlappingIntervals( .bind("dataSource", dataSource) .bind("start", interval.getStart().toString()) .bind("end", interval.getEnd().toString()) - .map((int index, ResultSet resultSet, StatementContext context) -> { - try { - return jsonMapper.readValue(resultSet.getBytes("payload"), DataSegment.class); - } - catch (IOException e) { - throw new RuntimeException(e); - } - }) + .map((int index, ResultSet resultSet, StatementContext context) -> + JacksonUtils.readValue(jsonMapper, resultSet.getBytes("payload"), DataSegment.class) + ) .iterator(); return StreamSupport.stream(segmentResultIterable.spliterator(), false); }) @@ -879,14 +873,14 @@ public Iterable iterateAllUsedSegments() } @Override - public Collection retrieveAllDataSourceNames() + public Set retrieveAllDataSourceNames() { return connector.getDBI().withHandle( handle -> handle .createQuery(StringUtils.format("SELECT DISTINCT(datasource) FROM %s", getSegmentsTable())) .fold( - new ArrayList<>(), - (List druidDataSources, + new HashSet<>(), + (Set druidDataSources, Map stringObjectMap, FoldController foldController, StatementContext statementContext) -> { diff --git a/server/src/main/java/org/apache/druid/server/audit/SQLAuditManager.java b/server/src/main/java/org/apache/druid/server/audit/SQLAuditManager.java index 7c6c65e90915..fc68c84727c2 100644 --- a/server/src/main/java/org/apache/druid/server/audit/SQLAuditManager.java +++ b/server/src/main/java/org/apache/druid/server/audit/SQLAuditManager.java @@ -28,6 +28,7 @@ import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.metadata.MetadataStorageTablesConfig; @@ -37,13 +38,9 @@ import org.skife.jdbi.v2.Handle; import org.skife.jdbi.v2.IDBI; import org.skife.jdbi.v2.Query; -import org.skife.jdbi.v2.StatementContext; import org.skife.jdbi.v2.tweak.HandleCallback; -import org.skife.jdbi.v2.tweak.ResultSetMapper; import java.io.IOException; -import java.sql.ResultSet; -import java.sql.SQLException; import java.util.List; import java.util.Map; @@ -124,39 +121,20 @@ public List fetchAuditHistory(final String key, final String type, I { final Interval theInterval = getIntervalOrDefault(interval); return dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery( + (Handle handle) -> handle + .createQuery( StringUtils.format( - "SELECT payload FROM %s WHERE audit_key = :audit_key and type = :type and created_date between :start_date and :end_date ORDER BY created_date", + "SELECT payload FROM %s WHERE audit_key = :audit_key and type = :type and " + + "created_date between :start_date and :end_date ORDER BY created_date", getAuditTable() ) - ).bind("audit_key", key) - .bind("type", type) - .bind("start_date", theInterval.getStart().toString()) - .bind("end_date", theInterval.getEnd().toString()) - .map( - new ResultSetMapper() - { - @Override - public AuditEntry map(int index, ResultSet r, StatementContext ctx) - throws SQLException - { - try { - return jsonMapper.readValue(r.getBytes("payload"), AuditEntry.class); - } - catch (IOException e) { - throw new SQLException(e); - } - } - } - ) - .list(); - } - } + ) + .bind("audit_key", key) + .bind("type", type) + .bind("start_date", theInterval.getStart().toString()) + .bind("end_date", theInterval.getEnd().toString()) + .map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), AuditEntry.class)) + .list() ); } @@ -185,39 +163,19 @@ public List fetchAuditHistory(final String type, Interval interval) { final Interval theInterval = getIntervalOrDefault(interval); return dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - return handle.createQuery( + (Handle handle) -> handle + .createQuery( StringUtils.format( - "SELECT payload FROM %s WHERE type = :type and created_date between :start_date and :end_date ORDER BY created_date", + "SELECT payload FROM %s WHERE type = :type and created_date between :start_date and " + + ":end_date ORDER BY created_date", getAuditTable() ) ) - .bind("type", type) - .bind("start_date", theInterval.getStart().toString()) - .bind("end_date", theInterval.getEnd().toString()) - .map( - new ResultSetMapper() - { - @Override - public AuditEntry map(int index, ResultSet r, StatementContext ctx) - throws SQLException - { - try { - return jsonMapper.readValue(r.getBytes("payload"), AuditEntry.class); - } - catch (IOException e) { - throw new SQLException(e); - } - } - } - ) - .list(); - } - } + .bind("type", type) + .bind("start_date", theInterval.getStart().toString()) + .bind("end_date", theInterval.getEnd().toString()) + .map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), AuditEntry.class)) + .list() ); } @@ -247,37 +205,18 @@ private List fetchAuditHistoryLastEntries(final String key, final St final String theQueryString = queryString; return dbi.withHandle( - new HandleCallback>() - { - @Override - public List withHandle(Handle handle) - { - Query> query = handle.createQuery(theQueryString); - if (key != null) { - query.bind("audit_key", key); - } - return query.bind("type", type) - .setMaxRows(theLimit) - .map( - new ResultSetMapper() - { - @Override - public AuditEntry map(int index, ResultSet r, StatementContext ctx) - throws SQLException - { - try { - return jsonMapper.readValue(r.getBytes("payload"), AuditEntry.class); - } - catch (IOException e) { - throw new SQLException(e); - } - } - } - ) - .list(); + (Handle handle) -> { + Query> query = handle.createQuery(theQueryString); + if (key != null) { + query.bind("audit_key", key); } + return query + .bind("type", type) + .setMaxRows(theLimit) + .map((index, r, ctx) -> JacksonUtils.readValue(jsonMapper, r.getBytes("payload"), AuditEntry.class)) + .list(); } - ); + ); } } diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/CompactionSegmentIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/CompactionSegmentIterator.java index d3216dc16366..448b78c2dea9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/CompactionSegmentIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/CompactionSegmentIterator.java @@ -25,6 +25,10 @@ import java.util.Iterator; import java.util.List; +/** + * Segments in the lists which are the elements of this iterator are sorted according to the natural segment order + * (see {@link DataSegment#compareTo}). + */ public interface CompactionSegmentIterator extends Iterator> { long UNKNOWN_REMAINING_SEGMENT_SIZE = -1L; diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java index d627741b10ae..a556ed3e57c1 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorLogger.java @@ -34,12 +34,10 @@ import org.apache.druid.server.coordinator.ServerHolder; import org.apache.druid.server.coordinator.rules.LoadRule; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.PartitionChunk; import java.util.List; import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.stream.StreamSupport; /** */ @@ -313,10 +311,7 @@ public DruidCoordinatorRuntimeParams run(DruidCoordinatorRuntimeParams params) .getUsedSegmentsTimelinesPerDataSource() .values() .stream() - .flatMap(timeline -> timeline.getAllTimelineEntries().values().stream()) - .flatMap(entryMap -> entryMap.values().stream()) - .flatMap(entry -> StreamSupport.stream(entry.getPartitionHolder().spliterator(), false)) - .map(PartitionChunk::getObject); + .flatMap(timeline -> timeline.iterateAllObjects().stream()); allSegments .collect(Collectors.groupingBy(DataSegment::getDataSource)) diff --git a/server/src/main/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstIterator.java b/server/src/main/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstIterator.java index 1ade74036688..1790abd467d9 100644 --- a/server/src/main/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstIterator.java +++ b/server/src/main/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstIterator.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; -import com.google.common.collect.Lists; import it.unimi.dsi.fastutil.objects.Object2LongOpenHashMap; import org.apache.druid.indexer.partitions.DynamicPartitionsSpec; import org.apache.druid.indexer.partitions.PartitionsSpec; @@ -34,6 +33,7 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.CompactionState; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.TimelineObjectHolder; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.PartitionChunk; @@ -44,11 +44,11 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Map.Entry; import java.util.NoSuchElementException; import java.util.Objects; import java.util.PriorityQueue; @@ -87,33 +87,24 @@ public class NewestSegmentFirstIterator implements CompactionSegmentIterator this.dataSources = dataSources; this.timelineIterators = new HashMap<>(dataSources.size()); - for (Entry> entry : dataSources.entrySet()) { - final String dataSource = entry.getKey(); - final VersionedIntervalTimeline timeline = entry.getValue(); + dataSources.forEach((String dataSource, VersionedIntervalTimeline timeline) -> { final DataSourceCompactionConfig config = compactionConfigs.get(dataSource); if (config != null && !timeline.isEmpty()) { - final List searchIntervals = findInitialSearchInterval( - timeline, - config.getSkipOffsetFromLatest(), - skipIntervals.get(dataSource) - ); + final List searchIntervals = + findInitialSearchInterval(timeline, config.getSkipOffsetFromLatest(), skipIntervals.get(dataSource)); if (!searchIntervals.isEmpty()) { timelineIterators.put(dataSource, new CompactibleTimelineObjectHolderCursor(timeline, searchIntervals)); } } - } - - for (Entry entry : compactionConfigs.entrySet()) { - final String dataSourceName = entry.getKey(); - final DataSourceCompactionConfig config = entry.getValue(); + }); + compactionConfigs.forEach((String dataSourceName, DataSourceCompactionConfig config) -> { if (config == null) { throw new ISE("Unknown dataSource[%s]", dataSourceName); } - updateQueue(dataSourceName, config); - } + }); } @Override @@ -211,17 +202,28 @@ private static class CompactibleTimelineObjectHolderCursor implements Iterator timeline .lookup(interval) .stream() - .filter(holder -> { - final List> chunks = Lists.newArrayList(holder.getObject().iterator()); - final long partitionBytes = chunks.stream().mapToLong(chunk -> chunk.getObject().getSize()).sum(); - return !chunks.isEmpty() - && partitionBytes > 0 - && interval.contains(chunks.get(0).getObject().getInterval()); - }) + .filter(holder -> isCompactibleHolder(interval, holder)) ) .collect(Collectors.toList()); } + private boolean isCompactibleHolder(Interval interval, TimelineObjectHolder holder) + { + final Iterator> chunks = holder.getObject().iterator(); + if (!chunks.hasNext()) { + return false; // There should be at least one chunk for a holder to be compactible. + } + PartitionChunk firstChunk = chunks.next(); + if (!interval.contains(firstChunk.getObject().getInterval())) { + return false; + } + long partitionBytes = firstChunk.getObject().getSize(); + while (partitionBytes == 0 && chunks.hasNext()) { + partitionBytes += chunks.next().getObject().getSize(); + } + return partitionBytes > 0; + } + @Override public boolean hasNext() { @@ -384,26 +386,29 @@ private static List findInitialSearchInterval( final List searchIntervals = new ArrayList<>(); for (Interval lookupInterval : filteredInterval) { - final List> holders = timeline.lookup( - new Interval(lookupInterval.getStart(), lookupInterval.getEnd()) - ); - - final List segments = holders + final List segments = timeline + .findNonOvershadowedObjectsInInterval(lookupInterval, Partitions.ONLY_COMPLETE) .stream() - .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) - .map(PartitionChunk::getObject) + // findNonOvershadowedObjectsInInterval() may return segments merely intersecting with lookupInterval, while + // we are interested only in segments fully lying within lookupInterval here. .filter(segment -> lookupInterval.contains(segment.getInterval())) - .sorted((s1, s2) -> Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval())) .collect(Collectors.toList()); - if (!segments.isEmpty()) { - searchIntervals.add( - new Interval( - segments.get(0).getInterval().getStart(), - segments.get(segments.size() - 1).getInterval().getEnd() - ) - ); + if (segments.isEmpty()) { + continue; } + + DateTime searchStart = segments + .stream() + .map(segment -> segment.getId().getIntervalStart()) + .min(Comparator.naturalOrder()) + .orElseThrow(AssertionError::new); + DateTime searchEnd = segments + .stream() + .map(segment -> segment.getId().getIntervalEnd()) + .max(Comparator.naturalOrder()) + .orElseThrow(AssertionError::new); + searchIntervals.add(new Interval(searchStart, searchEnd)); } return searchIntervals; diff --git a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java index f4cbcfa3088d..6eca39248c1c 100644 --- a/server/src/main/java/org/apache/druid/server/http/MetadataResource.java +++ b/server/src/main/java/org/apache/druid/server/http/MetadataResource.java @@ -29,6 +29,7 @@ import org.apache.druid.client.ImmutableDruidDataSource; import org.apache.druid.guice.annotations.Json; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.metadata.MetadataSegmentManager; import org.apache.druid.server.JettyUtils; import org.apache.druid.server.http.security.DatasourceResourceFilter; @@ -93,7 +94,7 @@ public Response getDataSources( { final boolean includeUnused = JettyUtils.getQueryParam(uriInfo, "includeUnused", "includeDisabled") != null; Collection druidDataSources = null; - final Set dataSourceNamesPreAuth; + final TreeSet dataSourceNamesPreAuth; if (includeUnused) { dataSourceNamesPreAuth = new TreeSet<>(segmentsMetadata.retrieveAllDataSourceNames()); } else { @@ -104,7 +105,7 @@ public Response getDataSources( .collect(Collectors.toCollection(TreeSet::new)); } - final Set dataSourceNamesPostAuth = new TreeSet<>(); + final TreeSet dataSourceNamesPostAuth = new TreeSet<>(); Function> raGenerator = datasourceName -> Collections.singletonList(AuthorizationUtils.DATASOURCE_READ_RA_GENERATOR.apply(datasourceName)); @@ -259,7 +260,8 @@ public Response getUsedSegmentsInDataSourceForIntervals( List intervals ) { - List segments = metadataStorageCoordinator.getUsedSegmentsForIntervals(dataSourceName, intervals); + Collection segments = metadataStorageCoordinator + .getUsedSegmentsForIntervals(dataSourceName, intervals, Segments.INCLUDING_OVERSHADOWED); Response.ResponseBuilder builder = Response.status(Response.Status.OK); if (full != null) { diff --git a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java index 73900de02baf..45dae0af0aa9 100644 --- a/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SegmentListerResource.java @@ -248,8 +248,8 @@ public void applyDataSegmentChangeRequests( } final ResponseContext context = createContext(req.getHeader("Accept")); - final ListenableFuture> future = loadDropRequestHandler - .processBatch(changeRequestList); + final ListenableFuture> future = + loadDropRequestHandler.processBatch(changeRequestList); final AsyncContext asyncContext = req.startAsync(); diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 84534d28e7d4..ee490de46f25 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -181,6 +181,7 @@ private void addToTimeline(Interval interval, String version) .interval(interval) .version(version) .shardSpec(NoneShardSpec.instance()) + .size(0) .build(), new TierSelectorStrategy() { diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 242795084917..9b1726bb8fc7 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2555,7 +2555,7 @@ private MyDataSegment() null, NoneShardSpec.instance(), null, - -1 + 0 ); } diff --git a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java index 7e1ae4e2bf40..9ba446659633 100644 --- a/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java +++ b/server/src/test/java/org/apache/druid/client/client/BatchServerInventoryViewTest.java @@ -384,6 +384,7 @@ private DataSegment makeSegment(int offset) ) ) .version(DateTimes.nowUtc().toString()) + .size(0) .build(); } diff --git a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java index 76d637c6df6d..1ea7f5fbcf8d 100644 --- a/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/metadata/IndexerSQLMetadataStorageCoordinatorTest.java @@ -23,16 +23,17 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; import org.apache.druid.indexing.overlord.DataSourceMetadata; import org.apache.druid.indexing.overlord.ObjectMetadata; import org.apache.druid.indexing.overlord.SegmentPublishResult; +import org.apache.druid.indexing.overlord.Segments; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpecFactory; import org.apache.druid.timeline.partition.LinearShardSpec; @@ -41,9 +42,9 @@ import org.apache.druid.timeline.partition.NumberedOverwritingShardSpecFactory; import org.apache.druid.timeline.partition.NumberedShardSpec; import org.apache.druid.timeline.partition.NumberedShardSpecFactory; -import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.PartitionIds; import org.apache.druid.timeline.partition.ShardSpecFactory; +import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.joda.time.Interval; import org.junit.Assert; @@ -57,12 +58,12 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; public class IndexerSQLMetadataStorageCoordinatorTest { @@ -497,7 +498,8 @@ public void testSimpleUsedList() throws IOException ImmutableSet.copyOf( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - defaultSegment.getInterval() + defaultSegment.getInterval(), + Segments.ONLY_VISIBLE ) ) ); @@ -509,47 +511,41 @@ public void testMultiIntervalUsedList() throws IOException coordinator.announceHistoricalSegments(SEGMENTS); coordinator.announceHistoricalSegments(ImmutableSet.of(defaultSegment3)); - Assert.assertEquals( - SEGMENTS, - ImmutableSet.copyOf( - coordinator.getUsedSegmentsForIntervals( - defaultSegment.getDataSource(), - ImmutableList.of(defaultSegment.getInterval()) - ) + Assertions.assertThat( + coordinator.getUsedSegmentsForIntervals( + defaultSegment.getDataSource(), + ImmutableList.of(defaultSegment.getInterval()), + Segments.ONLY_VISIBLE ) - ); + ).containsOnlyOnce(SEGMENTS.toArray(new DataSegment[0])); - Assert.assertEquals( - ImmutableSet.of(defaultSegment3), - ImmutableSet.copyOf( - coordinator.getUsedSegmentsForIntervals( - defaultSegment.getDataSource(), - ImmutableList.of(defaultSegment3.getInterval()) - ) + Assertions.assertThat( + coordinator.getUsedSegmentsForIntervals( + defaultSegment.getDataSource(), + ImmutableList.of(defaultSegment3.getInterval()), + Segments.ONLY_VISIBLE ) - ); + ).containsOnlyOnce(defaultSegment3); - Assert.assertEquals( - ImmutableSet.of(defaultSegment, defaultSegment2, defaultSegment3), - ImmutableSet.copyOf( - coordinator.getUsedSegmentsForIntervals( - defaultSegment.getDataSource(), - ImmutableList.of(defaultSegment.getInterval(), defaultSegment3.getInterval()) - ) + Assertions.assertThat( + coordinator.getUsedSegmentsForIntervals( + defaultSegment.getDataSource(), + ImmutableList.of(defaultSegment.getInterval(), defaultSegment3.getInterval()), + Segments.ONLY_VISIBLE ) - ); + ).containsOnlyOnce(defaultSegment, defaultSegment2, defaultSegment3); //case to check no duplication if two intervals overlapped with the interval of same segment. - Assert.assertEquals( - ImmutableList.of(defaultSegment3), + Assertions.assertThat( coordinator.getUsedSegmentsForIntervals( defaultSegment.getDataSource(), ImmutableList.of( Intervals.of("2015-01-03T00Z/2015-01-03T05Z"), Intervals.of("2015-01-03T09Z/2015-01-04T00Z") - ) + ), + Segments.ONLY_VISIBLE ) - ); + ).containsOnlyOnce(defaultSegment3); } @Test @@ -576,7 +572,8 @@ public void testUsedOverlapLow() throws IOException Set actualSegments = ImmutableSet.copyOf( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - Intervals.of("2014-12-31T23:59:59.999Z/2015-01-01T00:00:00.001Z") // end is exclusive + Intervals.of("2014-12-31T23:59:59.999Z/2015-01-01T00:00:00.001Z"), // end is exclusive + Segments.ONLY_VISIBLE ) ); Assert.assertEquals( @@ -595,7 +592,8 @@ public void testUsedOverlapHigh() throws IOException ImmutableSet.copyOf( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - Intervals.of("2015-1-1T23:59:59.999Z/2015-02-01T00Z") + Intervals.of("2015-1-1T23:59:59.999Z/2015-02-01T00Z"), + Segments.ONLY_VISIBLE ) ) ); @@ -608,7 +606,8 @@ public void testUsedOutOfBoundsLow() throws IOException Assert.assertTrue( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - new Interval(defaultSegment.getInterval().getStart().minus(1), defaultSegment.getInterval().getStart()) + new Interval(defaultSegment.getInterval().getStart().minus(1), defaultSegment.getInterval().getStart()), + Segments.ONLY_VISIBLE ).isEmpty() ); } @@ -621,7 +620,8 @@ public void testUsedOutOfBoundsHigh() throws IOException Assert.assertTrue( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - new Interval(defaultSegment.getInterval().getEnd(), defaultSegment.getInterval().getEnd().plusDays(10)) + new Interval(defaultSegment.getInterval().getEnd(), defaultSegment.getInterval().getEnd().plusDays(10)), + Segments.ONLY_VISIBLE ).isEmpty() ); } @@ -635,7 +635,8 @@ public void testUsedWithinBoundsEnd() throws IOException ImmutableSet.copyOf( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - defaultSegment.getInterval().withEnd(defaultSegment.getInterval().getEnd().minusMillis(1)) + defaultSegment.getInterval().withEnd(defaultSegment.getInterval().getEnd().minusMillis(1)), + Segments.ONLY_VISIBLE ) ) ); @@ -650,7 +651,8 @@ public void testUsedOverlapEnd() throws IOException ImmutableSet.copyOf( coordinator.getUsedSegmentsForInterval( defaultSegment.getDataSource(), - defaultSegment.getInterval().withEnd(defaultSegment.getInterval().getEnd().plusMillis(1)) + defaultSegment.getInterval().withEnd(defaultSegment.getInterval().getEnd().plusMillis(1)), + Segments.ONLY_VISIBLE ) ) ); @@ -1001,15 +1003,8 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce Assert.assertEquals(toBeAnnounced, announced); } - final VersionedIntervalTimeline timeline = VersionedIntervalTimeline - .forSegments(coordinator.getUsedSegmentsForInterval(dataSource, interval)); - - final List visibleSegments = timeline - .lookup(interval) - .stream() - .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()); + final Collection visibleSegments = + coordinator.getUsedSegmentsForInterval(dataSource, interval, Segments.ONLY_VISIBLE); Assert.assertEquals(1, visibleSegments.size()); Assert.assertEquals( @@ -1030,7 +1025,7 @@ public void testAllocatePendingSegmentsWithOvershadowingSegments() throws IOExce 0, 10L ), - visibleSegments.get(0) + Iterables.getOnlyElement(visibleSegments) ); } diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerEmptyTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerEmptyTest.java index 1e3f5d8e25bc..1ca5d84de29f 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerEmptyTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerEmptyTest.java @@ -80,7 +80,7 @@ public void testPollEmpty() sqlSegmentsMetadata.poll(); Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( - ImmutableList.of(), + ImmutableSet.of(), sqlSegmentsMetadata.retrieveAllDataSourceNames() ); Assert.assertEquals( diff --git a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java index a1852be765e3..bbfcf137b844 100644 --- a/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java +++ b/server/src/test/java/org/apache/druid/metadata/SQLMetadataSegmentManagerTest.java @@ -154,7 +154,7 @@ public void testPoll() sqlSegmentsMetadata.poll(); Assert.assertTrue(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( - ImmutableList.of("wikipedia"), + ImmutableSet.of("wikipedia"), sqlSegmentsMetadata.retrieveAllDataSourceNames() ); Assert.assertEquals( @@ -228,7 +228,7 @@ private DataSegment pollThenStopThenStartIntro() throws IOException sqlSegmentsMetadata.stopPollingDatabasePeriodically(); Assert.assertFalse(sqlSegmentsMetadata.isPollingDatabasePeriodically()); Assert.assertEquals( - ImmutableList.of("wikipedia"), + ImmutableSet.of("wikipedia"), sqlSegmentsMetadata.retrieveAllDataSourceNames() ); DataSegment newSegment = createNewSegment1("wikipedia2"); diff --git a/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java b/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java index c61e9637e94c..b6a0d85cc1dd 100644 --- a/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java +++ b/server/src/test/java/org/apache/druid/segment/loading/LocalDataSegmentPusherTest.java @@ -59,7 +59,7 @@ public class LocalDataSegmentPusherTest null, NoneShardSpec.instance(), null, - -1 + 0 ); DataSegment dataSegment2 = new DataSegment( "ds", @@ -70,7 +70,7 @@ public class LocalDataSegmentPusherTest null, NoneShardSpec.instance(), null, - -1 + 0 ); @Before diff --git a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java index effc42f30bf8..ee36f0ebd677 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/coordination/BatchDataSegmentAnnouncerTest.java @@ -444,6 +444,7 @@ private DataSegment makeSegment(int offset) .dimensions(ImmutableList.of("dim1", "dim2")) .metrics(ImmutableList.of("met1", "met2")) .loadSpec(ImmutableMap.of("type", "local")) + .size(0) .build(); } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java index f450bf3b051c..dfe41851bd65 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/DruidCoordinatorCleanupOvershadowedTest.java @@ -55,14 +55,17 @@ public class DruidCoordinatorCleanupOvershadowedTest private DataSegment segmentV0 = new DataSegment.Builder().dataSource("test") .interval(new Interval(start, start.plusHours(1))) .version("0") + .size(0) .build(); private DataSegment segmentV1 = new DataSegment.Builder().dataSource("test") .interval(new Interval(start, start.plusHours(1))) .version("1") + .size(0) .build(); private DataSegment segmentV2 = new DataSegment.Builder().dataSource("test") .interval(new Interval(start, start.plusHours(1))) .version("2") + .size(0) .build(); @Test diff --git a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java index ce681f318700..c5623b39809c 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/helper/NewestSegmentFirstPolicyTest.java @@ -28,10 +28,11 @@ import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import org.apache.druid.timeline.DataSegment; +import org.apache.druid.timeline.Partitions; import org.apache.druid.timeline.VersionedIntervalTimeline; import org.apache.druid.timeline.partition.NumberedShardSpec; -import org.apache.druid.timeline.partition.PartitionChunk; import org.apache.druid.timeline.partition.ShardSpec; +import org.assertj.core.api.Assertions; import org.joda.time.Interval; import org.joda.time.Period; import org.junit.Assert; @@ -40,9 +41,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.List; import java.util.stream.Collectors; -import java.util.stream.StreamSupport; public class NewestSegmentFirstPolicyTest { @@ -311,31 +312,19 @@ public void testClearSegmentsToCompactWhenSkippingSegments() Collections.emptyMap() ); - final List expectedSegmentsToCompact = new ArrayList<>(); - expectedSegmentsToCompact.addAll( - timeline - .lookup(Intervals.of("2017-12-03/2017-12-04")) - .stream() - .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()) + final List expectedSegmentsToCompact = new ArrayList<>( + timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-12-03/2017-12-04"), Partitions.ONLY_COMPLETE) ); - Assert.assertTrue(iterator.hasNext()); - Assert.assertEquals(expectedSegmentsToCompact, iterator.next()); - - expectedSegmentsToCompact.clear(); - expectedSegmentsToCompact.addAll( - timeline - .lookup(Intervals.of("2017-12-01/2017-12-02")) - .stream() - .flatMap(holder -> StreamSupport.stream(holder.getObject().spliterator(), false)) - .map(PartitionChunk::getObject) - .collect(Collectors.toList()) + expectedSegmentsToCompact.sort(Comparator.naturalOrder()); + + final List expectedSegmentsToCompact2 = new ArrayList<>( + timeline.findNonOvershadowedObjectsInInterval(Intervals.of("2017-12-01/2017-12-02"), Partitions.ONLY_COMPLETE) ); - Assert.assertTrue(iterator.hasNext()); - Assert.assertEquals(expectedSegmentsToCompact, iterator.next()); + expectedSegmentsToCompact2.sort(Comparator.naturalOrder()); - Assert.assertFalse(iterator.hasNext()); + Assertions.assertThat(iterator) + .toIterable() + .containsExactly(expectedSegmentsToCompact, expectedSegmentsToCompact2); } @Test @@ -519,18 +508,17 @@ private static VersionedIntervalTimeline createTimeline( final String version = DateTimes.nowUtc().toString(); final List orderedSpecs = Arrays.asList(specs); - orderedSpecs.sort((s1, s2) -> Comparators.intervalsByStartThenEnd().compare(s1.totalInterval, s2.totalInterval)); - Collections.reverse(orderedSpecs); + orderedSpecs.sort(Comparator.comparing(s -> s.totalInterval, Comparators.intervalsByStartThenEnd().reversed())); for (SegmentGenerateSpec spec : orderedSpecs) { - Interval remaininInterval = spec.totalInterval; + Interval remainingInterval = spec.totalInterval; - while (!Intervals.isEmpty(remaininInterval)) { + while (!Intervals.isEmpty(remainingInterval)) { final Interval segmentInterval; - if (remaininInterval.toDuration().isLongerThan(spec.segmentPeriod.toStandardDuration())) { - segmentInterval = new Interval(spec.segmentPeriod, remaininInterval.getEnd()); + if (remainingInterval.toDuration().isLongerThan(spec.segmentPeriod.toStandardDuration())) { + segmentInterval = new Interval(spec.segmentPeriod, remainingInterval.getEnd()); } else { - segmentInterval = remaininInterval; + segmentInterval = remainingInterval; } for (int i = 0; i < spec.numSegmentsPerShard; i++) { @@ -549,7 +537,7 @@ private static VersionedIntervalTimeline createTimeline( segments.add(segment); } - remaininInterval = SegmentCompactorUtil.removeIntervalFromEnd(remaininInterval, segmentInterval); + remainingInterval = SegmentCompactorUtil.removeIntervalFromEnd(remainingInterval, segmentInterval); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRuleTest.java index b4aa17d7b45d..630bc32f46d1 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropBeforeRuleTest.java @@ -32,10 +32,12 @@ public class PeriodDropBeforeRuleTest { - private static final DataSegment.Builder BUILDER = DataSegment.builder() - .dataSource("test") - .version(DateTimes.of("2012-12-31T01:00:00").toString()) - .shardSpec(NoneShardSpec.instance()); + private static final DataSegment.Builder BUILDER = DataSegment + .builder() + .dataSource("test") + .version(DateTimes.of("2012-12-31T01:00:00").toString()) + .shardSpec(NoneShardSpec.instance()) + .size(0); @Test public void testSerde() throws Exception diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropRuleTest.java index 06054a1af876..ac8eb336a848 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodDropRuleTest.java @@ -32,10 +32,12 @@ */ public class PeriodDropRuleTest { - private static final DataSegment.Builder BUILDER = DataSegment.builder() - .dataSource("test") - .version(DateTimes.of("2012-12-31T01:00:00").toString()) - .shardSpec(NoneShardSpec.instance()); + private static final DataSegment.Builder BUILDER = DataSegment + .builder() + .dataSource("test") + .version(DateTimes.of("2012-12-31T01:00:00").toString()) + .shardSpec(NoneShardSpec.instance()) + .size(0); @Test public void testAppliesToAll() diff --git a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java index 91c7510429f1..84b72614000f 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java +++ b/server/src/test/java/org/apache/druid/server/coordinator/rules/PeriodLoadRuleTest.java @@ -37,10 +37,12 @@ */ public class PeriodLoadRuleTest { - private static final DataSegment.Builder BUILDER = DataSegment.builder() - .dataSource("test") - .version(DateTimes.nowUtc().toString()) - .shardSpec(NoneShardSpec.instance()); + private static final DataSegment.Builder BUILDER = DataSegment + .builder() + .dataSource("test") + .version(DateTimes.nowUtc().toString()) + .shardSpec(NoneShardSpec.instance()) + .size(0); @Test public void testAppliesToAll() diff --git a/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java b/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java index 3188a193ef10..3e9c6b64ffef 100644 --- a/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java +++ b/services/src/main/java/org/apache/druid/cli/CliInternalHadoopIndexer.java @@ -36,7 +36,7 @@ import org.apache.druid.indexer.JobHelper; import org.apache.druid.indexer.Jobby; import org.apache.druid.indexer.MetadataStorageUpdaterJobHandler; -import org.apache.druid.indexer.path.MetadataStoreBasedUsedSegmentLister; +import org.apache.druid.indexer.path.MetadataStoreBasedUsedSegmentsRetriever; import org.apache.druid.indexer.updater.MetadataStorageUpdaterJobSpec; import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator; import org.apache.druid.java.util.common.logger.Logger; @@ -112,7 +112,7 @@ public void run() HadoopIngestionSpec.updateSegmentListIfDatasourcePathSpecIsUsed( config.getSchema(), HadoopDruidIndexerConfig.JSON_MAPPER, - new MetadataStoreBasedUsedSegmentLister( + new MetadataStoreBasedUsedSegmentsRetriever( injector.getInstance(IndexerMetadataStorageCoordinator.class) ) ) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java index 133cce05820e..da567c359459 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/DruidSchemaTest.java @@ -152,6 +152,7 @@ public void setUp() throws Exception .interval(Intervals.of("2000/P1Y")) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index1 ).add( @@ -160,6 +161,7 @@ public void setUp() throws Exception .interval(Intervals.of("2001/P1Y")) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index2 ).add( @@ -168,6 +170,7 @@ public void setUp() throws Exception .interval(index2.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index2 ); diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 44ad2ca39b39..61c7861194f6 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -647,6 +647,7 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .interval(index1.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index1 ).add( @@ -655,6 +656,7 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .interval(index2.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index2 ).add( @@ -663,6 +665,7 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .interval(forbiddenIndex.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), forbiddenIndex ).add( @@ -671,6 +674,7 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .interval(indexNumericDims.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), indexNumericDims ).add( @@ -679,6 +683,7 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( .interval(index4.getDataInterval()) .version("1") .shardSpec(new LinearShardSpec(0)) + .size(0) .build(), index4 );