Fix missing intervals after compacting intervals (#5092)

* Fix missing intervals after compacting intervals

* fix build
This commit is contained in:
Jihoon Son 2017-11-17 04:42:38 +09:00 committed by Gian Merlino
parent b34d471aa2
commit 93459f748f
3 changed files with 100 additions and 6 deletions

View File

@ -277,7 +277,7 @@ public class IndexTask extends AbstractTask
.filter(entry -> entry.getKey().contains(interval))
.map(Entry::getValue)
.findFirst()
.orElse(null);
.orElseThrow(() -> new ISE("Cannot find a version for interval[%s]", interval));
}
private static boolean isGuaranteedRollup(IndexIOConfig ioConfig, IndexTuningConfig tuningConfig)

View File

@ -20,6 +20,7 @@
package io.druid.indexing.common.task;
import com.google.common.base.Preconditions;
import com.google.common.collect.Lists;
import io.druid.indexing.common.TaskLock;
import io.druid.indexing.common.TaskLockType;
import io.druid.indexing.common.actions.LockTryAcquireAction;
@ -64,21 +65,20 @@ public class Tasks
public static SortedSet<Interval> computeCompactIntervals(SortedSet<Interval> intervals)
{
final SortedSet<Interval> compactIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
List<Interval> toBeAccumulated = null;
List<Interval> toBeAccumulated = new ArrayList<>();
for (Interval interval : intervals) {
if (toBeAccumulated == null) {
toBeAccumulated = new ArrayList<>();
if (toBeAccumulated.size() == 0) {
toBeAccumulated.add(interval);
} else {
if (toBeAccumulated.get(toBeAccumulated.size() - 1).abuts(interval)) {
toBeAccumulated.add(interval);
} else {
compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
toBeAccumulated = null;
toBeAccumulated = Lists.newArrayList(interval);
}
}
}
if (toBeAccumulated != null) {
if (toBeAccumulated.size() > 0) {
compactIntervals.add(JodaUtils.umbrellaInterval(toBeAccumulated));
}
return compactIntervals;

View File

@ -0,0 +1,94 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.indexing.common.task;
import io.druid.java.util.common.Intervals;
import io.druid.java.util.common.StringUtils;
import io.druid.java.util.common.guava.Comparators;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import java.util.Iterator;
import java.util.SortedSet;
import java.util.TreeSet;
public class TasksTest
{
@Test
public void testComputeCompactIntervals()
{
final SortedSet<Interval> inputIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
for (int m = 1; m < 13; m++) {
for (int d = 1; d < 10; d++) {
inputIntervals.add(getInterval(m, d, m, d + 1));
}
for (int d = 12; d < 20; d++) {
inputIntervals.add(getInterval(m, d, m, d + 1));
}
inputIntervals.add(getInterval(m, 22, m, 23));
for (int d = 25; d < 28; d++) {
inputIntervals.add(getInterval(m, d, m, d + 1));
}
if (m == 1 || m == 3 || m == 5 || m == 7 || m == 8 || m == 10) {
inputIntervals.add(getInterval(m, 31, m + 1, 1));
}
}
inputIntervals.add(Intervals.of("2017-12-31/2018-01-01"));
final SortedSet<Interval> compactIntervals = Tasks.computeCompactIntervals(inputIntervals);
final Iterator<Interval> compactIntervalIterator = compactIntervals.iterator();
Assert.assertTrue(compactIntervalIterator.hasNext());
Interval compactInterval = compactIntervalIterator.next();
final SortedSet<Interval> checkedIntervals = new TreeSet<>(Comparators.intervalsByStartThenEnd());
for (Interval inputInterval : inputIntervals) {
if (!compactInterval.contains(inputInterval)) {
if (compactIntervalIterator.hasNext()) {
compactInterval = compactIntervalIterator.next();
Assert.assertTrue(compactInterval.contains(inputInterval));
}
}
checkedIntervals.add(inputInterval);
}
Assert.assertFalse(compactIntervalIterator.hasNext());
Assert.assertEquals(inputIntervals, checkedIntervals);
}
private static Interval getInterval(int startMonth, int startDay, int endMonth, int endDay)
{
return Intervals.of(
StringUtils.format(
"2017-%02d-%02d/2017-%02d-%02d",
startMonth,
startDay,
endMonth,
endDay
)
);
}
}