Batch ingestion replace (#12137)

* Tombstone support for replace functionality

* A used segment interval is the interval of a current used segment that overlaps any of the input intervals for the spec

* Update compaction test to match replace behavior

* Adapt ITAutoCompactionTest to work with tombstones rather than dropping segments. Add support for tombstones in the broker.

* Style plus simple queriableindex test

* Add segment cache loader tombstone test

* Add more tests

* Add a method to the LogicalSegment to test whether it has any data

* Test filter with some empty logical segments

* Refactor more compaction/dropexisting tests

* Code coverage

* Support for all empty segments

* Skip tombstones when looking-up broker's timeline. Discard changes made to tool chest to avoid empty segments since they will no longer have empty segments after lookup because we are skipping over them.

* Fix null ptr when segment does not have a queriable index

* Add support for empty replace interval (all input data has been filtered out)

* Fixed coverage & style

* Find tombstone versions from lock versions

* Test failures & style

* Interner was making this fail since the two segments were consider equal due to their id's being equal

* Cleanup tombstone version code

* Force timeChunkLock whenever replace (i.e. dropExisting=true) is being used

* Reject replace spec when input intervals are empty

* Documentation

* Style and unit test

* Restore test code deleted by mistake

* Allocate forces TIME_CHUNK locking and uses lock versions. TombstoneShardSpec added.

* Unused imports. Dead code. Test coverage.

* Coverage.

* Prevent killer from throwing an exception for tombstones. This is the killer used in the peon for killing segments.

* Fix OmniKiller + more test coverage.

* Tombstones are now marked using a shard spec

* Drop a segment factory.json in the segment cache for tombstones

* Style

* Style + coverage

* style

* Add TombstoneLoadSpec.class to mapper in test

* Update core/src/main/java/org/apache/druid/segment/loading/TombstoneLoadSpec.java

Typo

Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>

* Update docs/configuration/index.md

Missing

Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>

* Typo

* Integrated replace with an existing test since the replace part was redundant and more importantly, the test file was very close or exceeding the 10 min default "no output" CI Travis threshold.

* Range does not work with multi-dim

Co-authored-by: Jonathan Wei <jon-wei@users.noreply.github.com>
This commit is contained in:
Agustin Gonzalez 2022-03-08 20:07:02 -07:00 committed by GitHub
parent dae53ae36a
commit abe76ccb90
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
70 changed files with 2727 additions and 348 deletions

View File

@ -0,0 +1,66 @@
/*
* 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.initialization;
import com.fasterxml.jackson.core.Version;
import com.fasterxml.jackson.databind.Module;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.segment.loading.TombstoneLoadSpec;
import java.util.List;
public class TombstoneDataStorageModule implements DruidModule
{
public static final String SCHEME = "tombstone";
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new Module()
{
@Override
public String getModuleName()
{
return "DruidTombstoneStorage-" + System.identityHashCode(this);
}
@Override
public Version version()
{
return Version.unknownVersion();
}
@Override
public void setupModule(SetupContext context)
{
context.registerSubtypes(TombstoneLoadSpec.class);
}
}
);
}
@Override
public void configure(Binder binder)
{
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.segment.loading;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.io.Files;
import org.apache.druid.initialization.TombstoneDataStorageModule;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
@JsonTypeName(TombstoneDataStorageModule.SCHEME)
public class TombstoneLoadSpec implements LoadSpec
{
@Override
public LoadSpecResult loadSegment(File destDir) throws SegmentLoadingException
{
try {
return new LoadSpecResult(writeFactoryFile(destDir));
}
catch (IOException e) {
throw new SegmentLoadingException(
"Failed to create factory.json for tombstone in dir [%s]",
destDir.getAbsolutePath()
);
}
}
@VisibleForTesting
public static int writeFactoryFile(File destDir) throws IOException
{
final String factoryJSONString = "{\"type\":\"tombstoneSegmentFactory\"}";
final File factoryJson = new File(destDir, "factory.json");
factoryJson.createNewFile();
Files.write(factoryJSONString.getBytes(StandardCharsets.UTF_8), factoryJson);
return factoryJSONString.length();
}
}

View File

@ -56,6 +56,9 @@ import java.util.stream.Collectors;
@PublicApi
public class DataSegment implements Comparable<DataSegment>, Overshadowable<DataSegment>
{
public static final String TOMBSTONE_LOADSPEC_TYPE = "tombstone";
/*
* The difference between this class and org.apache.druid.segment.Segment is that this class contains the segment
* metadata only, while org.apache.druid.segment.Segment represents the actual body of segment data, queryable.
@ -211,6 +214,7 @@ public class DataSegment implements Comparable<DataSegment>, Overshadowable<Data
)
{
this.id = SegmentId.of(dataSource, interval, version, shardSpec);
// prune loadspec if needed
this.loadSpec = pruneSpecsHolder.pruneLoadSpec ? PRUNED_LOAD_SPEC : prepareLoadSpec(loadSpec);
// Deduplicating dimensions and metrics lists as a whole because they are very likely the same for the same
// dataSource
@ -343,6 +347,11 @@ public class DataSegment implements Comparable<DataSegment>, Overshadowable<Data
return id;
}
public boolean isTombstone()
{
return getShardSpec().getType().equals(ShardSpec.Type.TOMBSTONE);
}
@Override
public boolean overshadows(DataSegment other)
{
@ -596,4 +605,11 @@ public class DataSegment implements Comparable<DataSegment>, Overshadowable<Data
);
}
}
@Override
public boolean hasData()
{
return !isTombstone();
}
}

View File

@ -41,7 +41,24 @@ import org.joda.time.Interval;
@PublicApi
public interface LogicalSegment
{
// With the addition of tombstones the broker and other places may need to be aware of
// them...
// The reason this is an ENUM and not simply boolean is that when we add awareness to the broker
// that tombstones should have been loaded, but they are not we might want to introduce a third value:
// MISSING
enum Status
{
READY, /* It has data and it is ready */
EMPTY /* It has no data */
}
Interval getInterval();
Interval getTrueInterval();
default Status getStatus()
{
return Status.READY;
}
}

View File

@ -75,4 +75,9 @@ public interface Overshadowable<T extends Overshadowable>
* {@link VersionedIntervalTimeline}.
*/
short getAtomicUpdateGroupSize();
default boolean hasData()
{
return true;
}
}

View File

@ -106,4 +106,13 @@ public class TimelineObjectHolder<VersionType, ObjectType extends Overshadowable
", object=" + object +
'}';
}
@Override
public LogicalSegment.Status getStatus()
{
if (object.hasData()) {
return Status.READY;
}
return Status.EMPTY;
}
}

View File

@ -105,11 +105,20 @@ public class VersionedIntervalTimeline<VersionType, ObjectType extends Overshado
private final Comparator<? super VersionType> versionComparator;
// Set this to true if the client needs to skip tombstones upon lookup (like the broker)
private boolean skipObjectsWithNoData = false;
public VersionedIntervalTimeline(Comparator<? super VersionType> versionComparator)
{
this.versionComparator = versionComparator;
}
public VersionedIntervalTimeline(Comparator<? super VersionType> versionComparator, boolean skipObjectsWithNoData)
{
this(versionComparator);
this.skipObjectsWithNoData = skipObjectsWithNoData;
}
public static void addSegments(
VersionedIntervalTimeline<String, DataSegment> timeline,
Iterator<DataSegment> segments
@ -743,7 +752,9 @@ public class VersionedIntervalTimeline<VersionType, ObjectType extends Overshado
Interval timelineInterval = entry.getKey();
TimelineEntry val = entry.getValue();
if (timelineInterval.overlaps(interval)) {
// exclude empty partition holders (i.e. tombstones) since they do not add value
// for higher level code...they have no data rows...
if ((!skipObjectsWithNoData || val.partitionHolder.hasData()) && timelineInterval.overlaps(interval)) {
retVal.add(
new TimelineObjectHolder<>(
timelineInterval,

View File

@ -161,4 +161,16 @@ public class PartitionHolder<T extends Overshadowable<T>> implements Iterable<Pa
"overshadowableManager=" + overshadowableManager +
'}';
}
public boolean hasData()
{
// it has data as long as one of the payloads has data, otherwise it does not
for (T payload : payloads()) {
if (payload.hasData()) {
return true;
}
}
return false;
}
}

View File

@ -34,6 +34,7 @@ import java.util.Map;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes({
@JsonSubTypes.Type(name = ShardSpec.Type.NONE, value = NoneShardSpec.class),
@JsonSubTypes.Type(name = ShardSpec.Type.TOMBSTONE, value = TombstoneShardSpec.class),
@JsonSubTypes.Type(name = ShardSpec.Type.SINGLE, value = SingleDimensionShardSpec.class),
@JsonSubTypes.Type(name = ShardSpec.Type.RANGE, value = DimensionRangeShardSpec.class),
@JsonSubTypes.Type(name = ShardSpec.Type.LINEAR, value = LinearShardSpec.class),
@ -168,5 +169,7 @@ public interface ShardSpec
String BUCKET_HASH = "bucket_hash";
String BUCKET_SINGLE_DIM = "bucket_single_dim";
String BUCKET_RANGE = "bucket_range";
String TOMBSTONE = "tombstone";
}
}

View File

@ -0,0 +1,107 @@
/*
* 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.partition;
import com.google.common.base.Objects;
public class TombstonePartitionedChunk<T> implements PartitionChunk<T>
{
private final T object;
public static <T> TombstonePartitionedChunk<T> make(T obj)
{
return new TombstonePartitionedChunk<T>(obj);
}
public TombstonePartitionedChunk(T object)
{
this.object = object;
}
@Override
public T getObject()
{
return object;
}
@Override
public boolean abuts(final PartitionChunk<T> other)
{
return false;
}
@Override
public boolean isStart()
{
return true;
}
@Override
public boolean isEnd()
{
return true;
}
@Override
public int getChunkNumber()
{
return 0;
}
@Override
public int compareTo(PartitionChunk<T> other)
{
if (other instanceof TombstonePartitionedChunk) {
return 0;
} else {
throw new IllegalArgumentException("Cannot compare against something that is not a TombstonePartitionedChunk.");
}
}
@Override
@SuppressWarnings("unchecked")
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return compareTo((TombstonePartitionedChunk<T>) o) == 0;
}
@Override
public int hashCode()
{
return Objects.hashCode(0);
}
@Override
public String toString()
{
return "TombstonePartitionedChunk{" +
"chunkNumber=" + 0 +
", chunks=" + 1 +
", object=" + object +
'}';
}
}

View File

@ -0,0 +1,114 @@
/*
* 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.partition;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.RangeSet;
import org.apache.druid.data.input.InputRow;
import java.util.List;
import java.util.Map;
import java.util.Objects;
/**
* A shard spec to represent tombstones. Its partition number is always zero and contains 1 core partitions.
*/
public class TombstoneShardSpec implements ShardSpec
{
public static TombstoneShardSpec INSTANCE = new TombstoneShardSpec();
@JsonProperty("partitionNum")
@Override
public int getPartitionNum()
{
return 0;
}
@Override
public ShardSpecLookup getLookup(final List<? extends ShardSpec> shardSpecs)
{
return createLookup(shardSpecs);
}
static ShardSpecLookup createLookup(List<? extends ShardSpec> shardSpecs)
{
return (long timestamp, InputRow row) -> shardSpecs.get(0);
}
@Override
public List<String> getDomainDimensions()
{
return ImmutableList.of();
}
@Override
public boolean possibleInDomain(Map<String, RangeSet<String>> domain)
{
return true;
}
@Override
@JsonProperty("partitions")
public int getNumCorePartitions()
{
return 1;
}
@Override
public <T> PartitionChunk<T> createChunk(T obj)
{
return TombstonePartitionedChunk.make(obj);
}
@Override
public String getType()
{
return Type.TOMBSTONE;
}
@Override
public String toString()
{
return "TombstoneShardSpec{" +
"partitionNum=" + 0 +
", partitions=" + 1 +
'}';
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
return true;
}
@Override
public int hashCode()
{
return Objects.hash(0);
}
}

View File

@ -37,6 +37,7 @@ import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.ShardSpec;
import org.apache.druid.timeline.partition.ShardSpecLookup;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
@ -364,6 +365,52 @@ public class DataSegmentTest
Assert.assertEquals(segment1, segment2.withLastCompactionState(compactionState));
}
@Test
public void testTombstoneType()
{
final DataSegment segment1 = DataSegment.builder()
.dataSource("foo")
.interval(Intervals.of("2012-01-01/2012-01-02"))
.version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(new TombstoneShardSpec())
.loadSpec(Collections.singletonMap(
"type",
DataSegment.TOMBSTONE_LOADSPEC_TYPE
))
.size(0)
.build();
Assert.assertTrue(segment1.isTombstone());
Assert.assertFalse(segment1.hasData());
final DataSegment segment2 = DataSegment.builder()
.dataSource("foo")
.interval(Intervals.of("2012-01-01/2012-01-02"))
.version(DateTimes.of("2012-01-01T11:22:33.444Z").toString())
.shardSpec(getShardSpec(7))
.loadSpec(Collections.singletonMap(
"type",
"foo"
))
.size(0)
.build();
Assert.assertFalse(segment2.isTombstone());
Assert.assertTrue(segment2.hasData());
final DataSegment segment3 = DataSegment.builder()
.dataSource("foo")
.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.assertFalse(segment3.isTombstone());
Assert.assertTrue(segment3.hasData());
}
private DataSegment makeDataSegment(String dataSource, String interval, String version)
{
return DataSegment.builder()

View File

@ -0,0 +1,75 @@
/*
* 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 org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
public class LogicalSegmentTest
{
@Test
public void getStatusReady()
{
LogicalSegment logicalSegment = new LogicalSegment()
{
@Override
public Interval getInterval()
{
return null;
}
@Override
public Interval getTrueInterval()
{
return null;
}
};
Assert.assertEquals(logicalSegment.getStatus(), LogicalSegment.Status.READY);
}
@Test
public void getStatusEmpty()
{
LogicalSegment emptyLogicalSegment = new LogicalSegment()
{
@Override
public Interval getInterval()
{
return null;
}
@Override
public Interval getTrueInterval()
{
return null;
}
@Override
public Status getStatus()
{
return Status.EMPTY;
}
};
Assert.assertEquals(emptyLogicalSegment.getStatus(), LogicalSegment.Status.EMPTY);
}
}

View File

@ -0,0 +1,66 @@
/*
* 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 org.junit.Assert;
import org.junit.Test;
public class OvershadowableTest
{
@Test
public void hasData()
{
Overshadowable overshadowable = new Overshadowable()
{
@Override
public int getStartRootPartitionId()
{
return 0;
}
@Override
public int getEndRootPartitionId()
{
return 0;
}
@Override
public String getVersion()
{
return null;
}
@Override
public short getMinorVersion()
{
return 0;
}
@Override
public short getAtomicUpdateGroupSize()
{
return 0;
}
};
Assert.assertTrue(overshadowable.hasData());
}
}

View File

@ -22,7 +22,9 @@ package org.apache.druid.timeline.partition;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.druid.data.input.StringTuple;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.timeline.DataSegment;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
@ -161,5 +163,21 @@ public class PartitionHolderCompletenessTest
holder.add(shardSpec.createChunk(new OvershadowableInteger("version", shardSpec.getPartitionNum(), 0)));
}
Assert.assertTrue(holder.isComplete());
Assert.assertTrue(holder.hasData());
}
@Test
public void testHasNoData()
{
final DataSegment tombstone = DataSegment.builder()
.dataSource("foo")
.version("1")
.interval(Intervals.of("2021-01-01/P1D"))
.shardSpec(new TombstoneShardSpec())
.size(1)
.build();
final PartitionChunk<DataSegment> partitionChunk = new TombstonePartitionedChunk<>(tombstone);
final PartitionHolder<DataSegment> partitionHolder = new PartitionHolder<DataSegment>(partitionChunk);
Assert.assertFalse(partitionHolder.hasData());
}
}

View File

@ -0,0 +1,100 @@
/*
* 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.partition;
import org.junit.Assert;
import org.junit.Test;
public class TombstonePartitionedChunkTest
{
final TombstoneShardSpec tombstoneShardSpec = new TombstoneShardSpec();
final TombstonePartitionedChunk tombstonePartitionedChunk =
TombstonePartitionedChunk.make(tombstoneShardSpec);
@Test
public void make()
{
Assert.assertTrue(TombstonePartitionedChunk.make(tombstoneShardSpec) != null);
}
@Test
public void getObject()
{
Assert.assertEquals(tombstoneShardSpec, tombstonePartitionedChunk.getObject());
}
@Test
public void abuts()
{
Assert.assertFalse(tombstonePartitionedChunk.abuts(tombstonePartitionedChunk));
}
@Test
public void isStart()
{
Assert.assertTrue(tombstonePartitionedChunk.isStart());
}
@Test
public void isEnd()
{
Assert.assertTrue(tombstonePartitionedChunk.isEnd());
}
@Test
public void getChunkNumber()
{
Assert.assertEquals(0, tombstonePartitionedChunk.getChunkNumber());
}
@Test
public void compareTo()
{
Assert.assertEquals(0, tombstonePartitionedChunk.compareTo(
TombstonePartitionedChunk.make(new Object())));
Exception exception = Assert.assertThrows(
IllegalArgumentException.class,
() -> tombstonePartitionedChunk.compareTo(
new NumberedPartitionChunk<Object>(0, 1, new Object()))
);
Assert.assertEquals("Cannot compare against something that is not a TombstonePartitionedChunk.",
exception.getMessage());
}
@Test
public void equalsTest()
{
TombstonePartitionedChunk aCopy = tombstonePartitionedChunk;
Assert.assertTrue(tombstonePartitionedChunk.equals(aCopy));
Assert.assertFalse(tombstonePartitionedChunk.equals(null));
Assert.assertFalse(tombstonePartitionedChunk.equals(new Object()));
Assert.assertTrue(tombstonePartitionedChunk.equals(TombstonePartitionedChunk.make(new Object())));
}
// make jacoco happy:
@Test
public void minutia()
{
Assert.assertTrue(tombstonePartitionedChunk.hashCode() > 0);
Assert.assertNotNull(tombstonePartitionedChunk.toString());
}
}

View File

@ -0,0 +1,89 @@
/*
* 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.partition;
import org.junit.Assert;
import org.junit.Test;
import java.util.Collections;
import static org.junit.Assert.assertEquals;
public class TombstoneShardSpecTest
{
final TombstoneShardSpec tombstoneShardSpec = new TombstoneShardSpec();
@Test
public void getPartitionNum()
{
assertEquals(0, tombstoneShardSpec.getPartitionNum());
}
@Test
public void getLookup()
{
ShardSpecLookup shardSpecLookup = tombstoneShardSpec.getLookup(Collections.singletonList(tombstoneShardSpec));
Assert.assertEquals(tombstoneShardSpec, shardSpecLookup.getShardSpec(1, null));
}
@Test
public void getDomainDimensions()
{
Assert.assertTrue(tombstoneShardSpec.getDomainDimensions().isEmpty());
}
@Test
public void possibleInDomain()
{
Assert.assertTrue(tombstoneShardSpec.possibleInDomain(Collections.emptyMap()));
}
@Test
public void getNumCorePartitions()
{
assertEquals(1, tombstoneShardSpec.getNumCorePartitions());
}
@Test
public void getType()
{
Assert.assertEquals(ShardSpec.Type.TOMBSTONE, tombstoneShardSpec.getType());
}
@Test
public void createChunk()
{
Assert.assertTrue(tombstoneShardSpec.createChunk(new Object()) != null);
}
// just to increase branch coverage
@Test
public void equalsTest()
{
TombstoneShardSpec tombstoneShardSpecOther = tombstoneShardSpec;
Assert.assertTrue(tombstoneShardSpec.equals(tombstoneShardSpecOther));
tombstoneShardSpecOther = null;
Assert.assertFalse(tombstoneShardSpec.equals(tombstoneShardSpecOther));
TombstoneShardSpec newTombostoneShardSepc = new TombstoneShardSpec();
Assert.assertTrue(tombstoneShardSpec.equals(newTombostoneShardSepc));
}
}

View File

@ -1026,7 +1026,7 @@ The below is a list of the supported configurations for auto compaction.
|Property|Description|Default|Required|
|--------|-----------|-------|--------|
|`dropExisting`|If `true`, then the generated compaction task drops (mark unused) all existing segments fully contained by the umbrella interval of the compacted segments when the task publishes new segments. If compaction fails, Druid does not drop or mark unused any segments. WARNING: this functionality is still in beta and can result in temporary data unavailability for data within the compacted `interval`. Note that changing this config does not cause intervals to be compacted again.|false|no|
|`dropExisting`|If `true` the compaction task replaces all existing segments fully contained by the umbrella interval of the compacted segments when the task publishes new segments and tombstones. If compaction fails, Druid does not publish any segments or tombstones. WARNING: this functionality is still in beta. Note that changing this config does not cause intervals to be compacted again.|false|no|
### Overlord

View File

@ -33,7 +33,7 @@ There are several cases to consider compaction for segment optimization:
By default, compaction does not modify the underlying data of the segments. However, there are cases when you may want to modify data during compaction to improve query performance:
- If, after ingestion, you realize that data for the time interval is sparse, you can use compaction to increase the segment granularity.
- Over time you don't need fine-grained granularity for older data so you want use compaction to change older segments to a coarser query granularity. This reduces the storage space required for older data. For example from `minute` to `hour`, or `hour` to `day`. You cannot go from coarser granularity to finer granularity.
- Over time you don't need fine-grained granularity for older data so you want use compaction to change older segments to a coarser query granularity. This reduces the storage space required for older data. For example from `minute` to `hour`, or `hour` to `day`.
- You can change the dimension order to improve sorting and reduce segment size.
- You can remove unused columns in compaction or implement an aggregation metric for older data.
- You can change segment rollup from dynamic partitioning with best-effort rollup to hash or range partitioning with perfect rollup. For more information on rollup, see [perfect vs best-effort rollup](./rollup.md#perfect-rollup-vs-best-effort-rollup).
@ -54,10 +54,10 @@ See [Setting up a manual compaction task](#setting-up-manual-compaction) for mor
## Data handling with compaction
During compaction, Druid overwrites the original set of segments with the compacted set. Druid also locks the segments for the time interval being compacted to ensure data consistency. By default, compaction tasks do not modify the underlying data. You can configure the compaction task to change the query granularity or add or remove dimensions in the compaction task. This means that the only changes to query results should be the result of intentional, not automatic, changes.
You can set `dropExisting` in `ioConfig` to "true" in the compaction task to configure Druid to mark all existing segments fully contained by the interval of the compaction task as unused (drop the segments). See the suggestion for reindexing with finer granularity under [Implementation considerations](native-batch.md#implementation-considerations) for an example.
> WARNING: `dropExisting` in `ioConfig` is a beta feature. It may cause data within the compaction task interval to become temporarily unavailable.
You can set `dropExisting` in `ioConfig` to "true" in the compaction task to configure Druid to replace all existing segments fully contained by the interval. See the suggestion for reindexing with finer granularity under [Implementation considerations](native-batch.md#implementation-considerations) for an example.
> WARNING: `dropExisting` in `ioConfig` is a beta feature.
If an ingestion task needs to write data to a segment for a time interval locked for compaction, by default the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjustment the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information. Another option is to set the compaction task to higher priority than the ingestion task.
If an ingestion task needs to write data to a segment for a time interval locked for compaction, by default the ingestion task supersedes the compaction task and the compaction task fails without finishing. For manual compaction tasks you can adjust the input spec interval to avoid conflicts between ingestion and compaction. For automatic compaction, you can set the `skipOffsetFromLatest` key to adjust the auto compaction starting point from the current time to reduce the chance of conflicts between ingestion and compaction. See [Compaction dynamic configuration](../configuration/index.md#compaction-dynamic-configuration) for more information. Another option is to set the compaction task to higher priority than the ingestion task.
### Segment granularity handling
@ -161,7 +161,7 @@ The compaction `ioConfig` requires specifying `inputSpec` as follows:
|-----|-----------|-------|--------|
|`type`|Task type: `compact`|none|Yes|
|`inputSpec`|Specification of the target [intervals](#interval-inputspec) or [segments](#segments-inputspec).|none|Yes|
|`dropExisting`|If `true`, when the task publishes newly compacted segments the compaction, it marks all existing segments fully contained by either of the following as unused (drops the segments):<br>- the `interval` in the `interval` type `inputSpec`.<br>- the umbrella interval of the `segments` in the `segment` type `inputSpec`.<br>If compaction fails, Druid does not drop or mark unused any segments.<br>**WARNING**: `dropExisting` in `ioConfig` is a beta feature. It may cause data within the compaction task interval to become temporarily unavailable.|false|no|
|`dropExisting`|If `true` the task replaces all existing segments fully contained by either of the following:<br>- the `interval` in the `interval` type `inputSpec`.<br>- the umbrella interval of the `segments` in the `segment` type `inputSpec`.<br>If compaction fails, Druid does change any of the existing segments.<br>**WARNING**: `dropExisting` in `ioConfig` is a beta feature. |false|no|
Druid supports two supported `inputSpec` formats:

View File

@ -40,7 +40,7 @@ A sample task is shown below:
"format" : "auto"
},
"dimensionsSpec" : {
"dimensions": ["country", "page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent",,"region","city"],
"dimensions": ["country", "page","language","user","unpatrolled","newPage","robot","anonymous","namespace","continent","region","city"],
"dimensionExclusions" : []
},
"metricsSpec" : [
@ -120,7 +120,7 @@ that range if there's some stray data with unexpected timestamps.
|type|The task type, this should always be "index".|none|yes|
|inputFormat|[`inputFormat`](./data-formats.md#input-format) to specify how to parse input data.|none|yes|
|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This means that you can append new segments to any datasource regardless of its original partitioning scheme. You must use the `dynamic` partitioning type for the appended segments. If you specify a different partitioning type, the task fails with an error.|false|no|
|dropExisting|If `true` and `appendToExisting` is `false` and the `granularitySpec` contains an`interval`, then the ingestion task drops (mark unused) all existing segments fully contained by the specified `interval` when the task publishes new segments. If ingestion fails, Druid does not drop or mark unused any segments. In the case of misconfiguration where either `appendToExisting` is `true` or `interval` is not specified in `granularitySpec`, Druid does not drop any segments even if `dropExisting` is `true`. WARNING: this functionality is still in beta and can result in temporary data unavailability for data within the specified `interval`.|false|no|
|dropExisting|If this setting is `false` then ingestion proceeds as usual. Set this to `true` and `appendToExisting` to `false` to enforce true "replace" functionality as described next. If `true` and `appendToExisting` is `false` and the `granularitySpec` contains at least one`interval`, then the ingestion task will create regular segments for time chunk intervals with input data and `tombstones` for all other time chunks with no data. The task will publish the data segments and the tombstone segments together when the it publishes new segments. The net effect of the data segments and the tombstones is to completely adhere to a "replace" semantics where the input data contained in the `granularitySpec` intervals replaces all existing data in the intervals even for time chunks that would be empty in the case that no input data was associated with them. In the extreme case when the input data set that falls in the `granularitySpec` intervals is empty all existing data in the interval will be replaced with an empty data set (i.e. with nothing -- all existing data will be covered by `tombstones`). If ingestion fails, no segments and tombstones will be published. The following two combinations are not supported and will make the ingestion fail with an error: `dropExisting` is `true` and `interval` is not specified in `granularitySpec` or `appendToExisting` is true and `dropExisting` is `true`. WARNING: this functionality is still in beta and even though we are not aware of any bugs, use with caution.|false|no|
### `tuningConfig`

View File

@ -81,12 +81,12 @@ The `maxNumConcurrentSubTasks` in the `tuningConfig` determines the number of co
#### Replacing or appending data
By default, batch ingestion replaces all data in the intervals in your `granularitySpec`' for any segment that it writes to. If you want to add to the segment instead, set the `appendToExisting` flag in the `ioConfig`. Batch ingestion only replaces data in segments where it actively adds data. If there are segments in the intervals for your `granularitySpec` that have do not have data from a task, they remain unchanged. If any existing segments partially overlap with the intervals in the `granularitySpec`, the portion of those segments outside the interval for the new spec remain visible.
#### Dropping existing segments
You can set `dropExisting` flag in the `ioConfig` to true if you want the ingestion task to drop all existing segments that start and end within the intervals for your `granularitySpec`. This applies whether or not the new data covers all existing segments. `dropExisting` only applies when `appendToExisting` is false and the `granularitySpec` contains an `interval`. WARNING: this functionality is still in beta and can result in temporary data unavailability for data within the specified `interval`.
#### Fully replacing existing segments using tombstones
You can set `dropExisting` flag in the `ioConfig` to true if you want the ingestion task to replace all existing segments that start and end within the intervals for your `granularitySpec`. This applies whether or not the new data covers all existing segments. `dropExisting` only applies when `appendToExisting` is false and the `granularitySpec` contains an `interval`. WARNING: this functionality is still in beta.
The following examples demonstrate when to set the `dropExisting` property to true in the `ioConfig`:
Consider an existing segment with an interval of 2020-01-01 to 2021-01-01 and `YEAR` `segmentGranularity`. You want to overwrite the whole interval of 2020-01-01 to 2021-01-01 with new data using the finer segmentGranularity of `MONTH`. If the replacement data does not have a record within every months from 2020-01-01 to 2021-01-01 Druid cannot drop the original `YEAR` segment even if it does include all the replacement data. Set `dropExisting` to true in this case to drop the original segment at `YEAR` `segmentGranularity` since you no longer need it.<br><br>
Consider an existing segment with an interval of 2020-01-01 to 2021-01-01 and `YEAR` `segmentGranularity`. You want to overwrite the whole interval of 2020-01-01 to 2021-01-01 with new data using the finer segmentGranularity of `MONTH`. If the replacement data does not have a record within every months from 2020-01-01 to 2021-01-01 Druid cannot drop the original `YEAR` segment even if it does include all the replacement data. Set `dropExisting` to true in this case to replace the original segment at `YEAR` `segmentGranularity` since you no longer need it.<br><br>
Imagine you want to re-ingest or overwrite a datasource and the new data does not contain some time intervals that exist in the datasource. For example, a datasource contains the following data at `MONTH` segmentGranularity:
- **January**: 1 record
- **February**: 10 records
@ -102,7 +102,7 @@ Unless you set `dropExisting` to true, the result after ingestion with overwrite
* **February**: 10 records
* **March**: 9 records
This is incorrect since the new data has 0 records for January. Set `dropExisting` to true to drop the unneeded January segment.
This may not be what it is expected since the new data has 0 records for January. Set `dropExisting` to true to replace the unneeded January segment with a tombstone.
## Parallel indexing example
@ -208,7 +208,7 @@ When defining the `granularitySpec` for index parallel, consider the defining `i
|type|The task type. Set to the value to `index_parallel`.|none|yes|
|inputFormat|[`inputFormat`](./data-formats.md#input-format) to specify how to parse input data.|none|yes|
|appendToExisting|Creates segments as additional shards of the latest version, effectively appending to the segment set instead of replacing it. This means that you can append new segments to any datasource regardless of its original partitioning scheme. You must use the `dynamic` partitioning type for the appended segments. If you specify a different partitioning type, the task fails with an error.|false|no|
|dropExisting|If `true` and `appendToExisting` is `false` and the `granularitySpec` contains an`interval`, then the ingestion task drops (mark unused) all existing segments fully contained by the specified `interval` when the task publishes new segments. If ingestion fails, Druid does not drop or mark unused any segments. In the case of misconfiguration where either `appendToExisting` is `true` or `interval` is not specified in `granularitySpec`, Druid does not drop any segments even if `dropExisting` is `true`. WARNING: this functionality is still in beta and can result in temporary data unavailability for data within the specified `interval`.|false|no|
|dropExisting|If `true` and `appendToExisting` is `false` and the `granularitySpec` contains an`interval`, then the ingestion task replaces all existing segments fully contained by the specified `interval` when the task publishes new segments. If ingestion fails, Druid does not change any existing segment. In the case of misconfiguration where either `appendToExisting` is `true` or `interval` is not specified in `granularitySpec`, Druid does not replace any segments even if `dropExisting` is `true`. WARNING: this functionality is still in beta.|false|no|
### `tuningConfig`

View File

@ -36,6 +36,7 @@ import org.apache.druid.indexing.common.LockGranularity;
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.LockListAction;
import org.apache.druid.indexing.common.actions.RetrieveUsedSegmentsAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
@ -43,12 +44,16 @@ import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.indexing.firehose.WindowedSegmentId;
import org.apache.druid.indexing.input.InputRowSchemas;
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.JodaUtils;
import org.apache.druid.java.util.common.NonnullPair;
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.granularity.Granularity;
@ -64,12 +69,15 @@ import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.IngestionSpec;
import org.apache.druid.segment.indexing.TuningConfig;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.timeline.CompactionState;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.VersionedIntervalTimeline;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
import org.joda.time.Period;
@ -79,6 +87,7 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@ -115,6 +124,9 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
private final int maxAllowedLockCount;
// Store lock versions
Map<Interval, String> intervalToVersion = new HashMap<>();
protected AbstractBatchIndexTask(String id, String dataSource, Map<String, Object> context)
{
super(id, dataSource, context);
@ -296,8 +308,10 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
);
final boolean useSharedLock = ioConfig.isAppendToExisting() && getContextValue(Tasks.USE_SHARED_LOCK, false);
// Respect task context value most.
if (forceTimeChunkLock) {
log.info("[%s] is set to true in task context. Use timeChunk lock", Tasks.FORCE_TIME_CHUNK_LOCK_KEY);
if (forceTimeChunkLock || ioConfig.isDropExisting()) {
log.info("forceTimeChunkLock[%s] or isDropExisting[%s] is set to true. Use timeChunk lock",
forceTimeChunkLock, ioConfig.isDropExisting()
);
taskLockHelper = new TaskLockHelper(false, useSharedLock);
if (!intervals.isEmpty()) {
return tryTimeChunkLock(client, intervals);
@ -431,6 +445,7 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", cur));
}
locksAcquired++;
intervalToVersion.put(cur, lock.getVersion());
}
return true;
}
@ -704,6 +719,89 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
);
}
}
@Nullable
public static String findVersion(Map<Interval, String> versions, Interval interval)
{
return versions.entrySet().stream()
.filter(entry -> entry.getKey().contains(interval))
.map(Map.Entry::getValue)
.findFirst()
.orElse(null);
}
public static NonnullPair<Interval, String> findIntervalAndVersion(
TaskToolbox toolbox,
IngestionSpec<?, ?> ingestionSpec,
DateTime timestamp
) throws IOException
{
// This method is called whenever subtasks need to allocate a new segment via the supervisor task.
// As a result, this code is never called in the Overlord. For now using the materialized intervals
// here is ok for performance reasons
GranularitySpec granularitySpec = ingestionSpec.getDataSchema().getGranularitySpec();
final Set<Interval> materializedBucketIntervals = granularitySpec.materializedBucketIntervals();
// List locks whenever allocating a new segment because locks might be revoked and no longer valid.
final List<TaskLock> locks = toolbox
.getTaskActionClient()
.submit(new LockListAction());
final TaskLock revokedLock = locks.stream().filter(TaskLock::isRevoked).findAny().orElse(null);
if (revokedLock != null) {
throw new ISE("Lock revoked: [%s]", revokedLock);
}
final Map<Interval, String> versions = locks
.stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
Interval interval;
String version;
if (!materializedBucketIntervals.isEmpty()) {
// If granularity spec has explicit intervals, we just need to find the version associated to the interval.
// This is because we should have gotten all required locks up front when the task starts up.
final Optional<Interval> maybeInterval = granularitySpec.bucketInterval(timestamp);
if (!maybeInterval.isPresent()) {
throw new IAE("Could not find interval for timestamp [%s]", timestamp);
}
interval = maybeInterval.get();
if (!materializedBucketIntervals.contains(interval)) {
throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec);
}
version = AbstractBatchIndexTask.findVersion(versions, interval);
if (version == null) {
throw new ISE("Cannot find a version for interval[%s]", interval);
}
} else {
// We don't have explicit intervals. We can use the segment granularity to figure out what
// interval we need, but we might not have already locked it.
interval = granularitySpec.getSegmentGranularity().bucket(timestamp);
version = AbstractBatchIndexTask.findVersion(versions, interval);
if (version == null) {
if (ingestionSpec.getTuningConfig() instanceof ParallelIndexTuningConfig) {
final int maxAllowedLockCount = ((ParallelIndexTuningConfig) ingestionSpec.getTuningConfig())
.getMaxAllowedLockCount();
if (maxAllowedLockCount >= 0 && locks.size() >= maxAllowedLockCount) {
throw new MaxAllowedLocksExceededException(maxAllowedLockCount);
}
}
// We don't have a lock for this interval, so we should lock it now.
final TaskLock lock = Preconditions.checkNotNull(
toolbox.getTaskActionClient().submit(
new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)
),
"Cannot acquire a lock for interval[%s]",
interval
);
if (lock.isRevoked()) {
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
}
version = lock.getVersion();
}
}
return new NonnullPair<>(interval, version);
}
private static class LockGranularityDetermineResult
{
@ -724,4 +822,42 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
this.segments = segments;
}
}
/**
* Get the version from the locks for a given timestamp. This will work if the locks were acquired upfront
* @param timestamp
* @return The interval andversion if n interval that contains an interval was found or null otherwise
*/
@Nullable
Pair<Interval, String> lookupVersion(DateTime timestamp)
{
java.util.Optional<Map.Entry<Interval, String>> intervalAndVersion = intervalToVersion.entrySet()
.stream()
.filter(e -> e.getKey()
.contains(
timestamp))
.findFirst();
if (!intervalAndVersion.isPresent()) {
return null;
}
return new Pair(intervalAndVersion.get().getKey(), intervalAndVersion.get().getValue());
}
protected SegmentIdWithShardSpec allocateNewSegmentForTombstone(
IngestionSpec ingestionSchema,
DateTime timestamp,
TaskToolbox toolbox
)
{
// Since tombstones are derived from inputIntervals, inputIntervals cannot be empty for replace, and locks are
// all acquired upfront then the following stream query should always find the version
Pair<Interval, String> intervalAndVersion = lookupVersion(timestamp);
return new SegmentIdWithShardSpec(
ingestionSchema.getDataSchema().getDataSource(),
intervalAndVersion.lhs,
intervalAndVersion.rhs,
new TombstoneShardSpec()
);
}
}

View File

@ -700,11 +700,15 @@ public class CompactionTask extends AbstractBatchIndexTask
LockGranularity lockGranularityInUse
) throws IOException, SegmentLoadingException
{
final List<DataSegment> usedSegments = segmentProvider.findSegments(toolbox.getTaskActionClient());
segmentProvider.checkSegments(lockGranularityInUse, usedSegments);
final Map<DataSegment, File> segmentFileMap = toolbox.fetchSegments(usedSegments);
final List<DataSegment> usedSegmentsMinusTombstones =
segmentProvider.findSegments(toolbox.getTaskActionClient())
.stream()
.filter(dataSegment -> !dataSegment.isTombstone()) // skip tombstones
.collect(Collectors.toList());
segmentProvider.checkSegments(lockGranularityInUse, usedSegmentsMinusTombstones);
final Map<DataSegment, File> segmentFileMap = toolbox.fetchSegments(usedSegmentsMinusTombstones);
final List<TimelineObjectHolder<String, DataSegment>> timelineSegments = VersionedIntervalTimeline
.forSegments(usedSegments)
.forSegments(usedSegmentsMinusTombstones)
.lookup(segmentProvider.interval);
return new NonnullPair<>(segmentFileMap, timelineSegments);
}

View File

@ -58,6 +58,7 @@ import org.apache.druid.indexing.common.actions.SegmentTransactionalInsertAction
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.stats.TaskRealtimeMetricsMonitor;
import org.apache.druid.indexing.common.task.batch.parallel.PartialHashSegmentGenerateTask;
import org.apache.druid.indexing.common.task.batch.parallel.TombstoneHelper;
import org.apache.druid.indexing.common.task.batch.parallel.iterator.DefaultIndexTaskInputRowIteratorBuilder;
import org.apache.druid.indexing.common.task.batch.partition.CompletePartitionAnalysis;
import org.apache.druid.indexing.common.task.batch.partition.HashPartitionAnalysis;
@ -92,6 +93,7 @@ import org.apache.druid.segment.realtime.appenderator.Appenderator;
import org.apache.druid.segment.realtime.appenderator.AppenderatorConfig;
import org.apache.druid.segment.realtime.appenderator.BaseAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.BatchAppenderatorDriver;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.segment.realtime.appenderator.SegmentsAndCommitMetadata;
import org.apache.druid.segment.realtime.appenderator.TransactionalSegmentPublisher;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
@ -883,10 +885,6 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
throw new UOE("[%s] secondary partition type is not supported", partitionsSpec.getType());
}
Set<DataSegment> segmentsFoundForDrop = null;
if (ingestionSchema.getIOConfig().isDropExisting()) {
segmentsFoundForDrop = getUsedSegmentsWithinInterval(toolbox, getDataSource(), ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals());
}
final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) ->
toolbox.getTaskActionClient()
@ -912,7 +910,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
try (final BatchAppenderatorDriver driver = BatchAppenderators.newDriver(appenderator, toolbox, segmentAllocator)) {
driver.startJob();
InputSourceProcessor.process(
SegmentsAndCommitMetadata pushed = InputSourceProcessor.process(
dataSchema,
driver,
partitionsSpec,
@ -942,9 +940,36 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
ingestionSchema
);
Set<DataSegment> tombStones = Collections.emptySet();
if (ingestionSchema.getIOConfig().isDropExisting()) {
TombstoneHelper tombstoneHelper = new TombstoneHelper(pushed.getSegments(),
ingestionSchema.getDataSchema(),
toolbox.getTaskActionClient());
List<Interval> tombstoneIntervals = tombstoneHelper.computeTombstoneIntervals();
// now find the versions for the tombstone intervals
Map<Interval, SegmentIdWithShardSpec> tombstonesAndVersions = new HashMap<>();
for (Interval interval : tombstoneIntervals) {
SegmentIdWithShardSpec segmentIdWithShardSpec = allocateNewSegmentForTombstone(
ingestionSchema,
interval.getStart(),
toolbox
);
tombstonesAndVersions.put(interval, segmentIdWithShardSpec);
}
tombStones = tombstoneHelper.computeTombstones(tombstonesAndVersions);
log.debugSegments(tombStones, "To publish tombstones");
}
// Probably we can publish atomicUpdateGroup along with segments.
final SegmentsAndCommitMetadata published =
awaitPublish(driver.publishAll(inputSegments, segmentsFoundForDrop, publisher, annotateFunction), pushTimeout);
awaitPublish(driver.publishAll(
inputSegments,
null,
tombStones,
publisher,
annotateFunction
), pushTimeout);
appenderator.close();
// Try to wait for segments to be loaded by the cluster if the tuning config specifies a non-zero value
@ -1034,6 +1059,11 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) {
throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
}
if (ioConfig.isDropExisting() && dataSchema.getGranularitySpec().inputIntervals().isEmpty()) {
throw new IAE("GranularitySpec's intervals cannot be empty when setting dropExisting to true.");
}
if (ioConfig.getInputSource() != null && ioConfig.getInputSource().needsFormat()) {
Checks.checkOneNotNullOrEmpty(
ImmutableList.of(

View File

@ -53,4 +53,10 @@ public class ParallelIndexIOConfig extends IndexIOConfig
{
this(firehoseFactory, null, null, appendToExisting, null);
}
@Deprecated
public ParallelIndexIOConfig(FirehoseFactory firehoseFactory, @Nullable Boolean appendToExisting, boolean dropExisting)
{
this(firehoseFactory, null, null, appendToExisting, dropExisting);
}
}

View File

@ -453,8 +453,6 @@ public abstract class ParallelIndexPhaseRunner<SubTaskType extends Task, SubTask
return tuningConfig;
}
@VisibleForTesting
TaskToolbox getToolbox()
{
return toolbox;

View File

@ -220,6 +220,11 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
this.ingestionSchema = ingestionSchema;
this.baseSubtaskSpecName = baseSubtaskSpecName == null ? getId() : baseSubtaskSpecName;
if (ingestionSchema.getIOConfig().isDropExisting() &&
ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals().isEmpty()) {
throw new ISE("GranularitySpec's intervals cannot be empty when setting dropExisting to true.");
}
if (isGuaranteedRollup(ingestionSchema.getIOConfig(), ingestionSchema.getTuningConfig())) {
checkPartitionsSpecForForceGuaranteedRollup(ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec());
}
@ -1084,9 +1089,29 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
ingestionSchema
);
Set<DataSegment> segmentsFoundForDrop = null;
Set<DataSegment> tombStones;
if (ingestionSchema.getIOConfig().isDropExisting()) {
segmentsFoundForDrop = getUsedSegmentsWithinInterval(toolbox, getDataSource(), ingestionSchema.getDataSchema().getGranularitySpec().inputIntervals());
TombstoneHelper tombstoneHelper = new TombstoneHelper(
newSegments,
ingestionSchema.getDataSchema(),
toolbox.getTaskActionClient()
);
List<Interval> tombstoneIntervals = tombstoneHelper.computeTombstoneIntervals();
if (!tombstoneIntervals.isEmpty()) {
Map<Interval, SegmentIdWithShardSpec> tombstonesAnShards = new HashMap<>();
for (Interval interval : tombstoneIntervals) {
SegmentIdWithShardSpec segmentIdWithShardSpec = allocateNewSegmentForTombstone(
ingestionSchema,
interval.getStart(),
toolbox
);
tombstonesAnShards.put(interval, segmentIdWithShardSpec);
}
tombStones = tombstoneHelper.computeTombstones(tombstonesAnShards);
newSegments.addAll(tombStones);
LOG.debugSegments(tombStones, "To publish tombstones");
}
}
final TransactionalSegmentPublisher publisher = (segmentsToBeOverwritten, segmentsToDrop, segmentsToPublish, commitMetadata) ->
@ -1095,7 +1120,10 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
);
final boolean published =
newSegments.isEmpty()
|| publisher.publishSegments(oldSegments, segmentsFoundForDrop, newSegments, annotateFunction, null).isSuccess();
|| publisher.publishSegments(oldSegments,
Collections.emptySet(),
newSegments, annotateFunction,
null).isSuccess();
if (published) {
LOG.info("Published [%d] segments", newSegments.size());
@ -1261,15 +1289,6 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
}
}
@Nullable
public static String findVersion(Map<Interval, String> versions, Interval interval)
{
return versions.entrySet().stream()
.filter(entry -> entry.getKey().contains(interval))
.map(Entry::getValue)
.findFirst()
.orElse(null);
}
static InputFormat getInputFormat(ParallelIndexIngestionSpec ingestionSchema)
{

View File

@ -32,6 +32,7 @@ import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.LockListAction;
import org.apache.druid.indexing.common.actions.SurrogateAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
import org.apache.druid.indexing.common.task.ClientBasedTaskInfoProvider;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.java.util.common.FileUtils;
@ -283,7 +284,7 @@ abstract class PartialSegmentMergeTask<S extends ShardSpec> extends PerfectRollu
getDataSource(),
interval,
Preconditions.checkNotNull(
ParallelIndexSupervisorTask.findVersion(intervalToVersion, interval),
AbstractBatchIndexTask.findVersion(intervalToVersion, interval),
"version for interval[%s]",
interval
),

View File

@ -20,8 +20,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
@ -29,19 +27,12 @@ import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.SplittableInputSource;
import org.apache.druid.indexing.common.Counters;
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.LockListAction;
import org.apache.druid.indexing.common.actions.TimeChunkLockTryAcquireAction;
import org.apache.druid.indexing.common.task.AbstractBatchIndexTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException;
import org.apache.druid.indexing.common.task.batch.parallel.TaskMonitor.SubTaskCompleteEvent;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.NonnullPair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.BuildingNumberedShardSpec;
@ -55,10 +46,8 @@ import java.util.HashMap;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.stream.Collectors;
/**
* An implementation of {@link ParallelIndexTaskRunner} to support best-effort roll-up. This runner can submit and
@ -299,71 +288,9 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner
return segmentIdHolder.getValue();
}
private NonnullPair<Interval, String> findIntervalAndVersion(DateTime timestamp) throws IOException
NonnullPair<Interval, String> findIntervalAndVersion(DateTime timestamp) throws IOException
{
final GranularitySpec granularitySpec = getIngestionSchema().getDataSchema().getGranularitySpec();
// This method is called whenever subtasks need to allocate a new segment via the supervisor task.
// As a result, this code is never called in the Overlord. For now using the materialized intervals
// here is ok for performance reasons
final Set<Interval> materializedBucketIntervals = granularitySpec.materializedBucketIntervals();
// List locks whenever allocating a new segment because locks might be revoked and no longer valid.
final List<TaskLock> locks = getToolbox()
.getTaskActionClient()
.submit(new LockListAction());
final TaskLock revokedLock = locks.stream().filter(TaskLock::isRevoked).findAny().orElse(null);
if (revokedLock != null) {
throw new ISE("Lock revoked: [%s]", revokedLock);
}
final Map<Interval, String> versions = locks
.stream()
.collect(Collectors.toMap(TaskLock::getInterval, TaskLock::getVersion));
Interval interval;
String version;
if (!materializedBucketIntervals.isEmpty()) {
// If granularity spec has explicit intervals, we just need to find the version associated to the interval.
// This is because we should have gotten all required locks up front when the task starts up.
final Optional<Interval> maybeInterval = granularitySpec.bucketInterval(timestamp);
if (!maybeInterval.isPresent()) {
throw new IAE("Could not find interval for timestamp [%s]", timestamp);
}
interval = maybeInterval.get();
if (!materializedBucketIntervals.contains(interval)) {
throw new ISE("Unspecified interval[%s] in granularitySpec[%s]", interval, granularitySpec);
}
version = ParallelIndexSupervisorTask.findVersion(versions, interval);
if (version == null) {
throw new ISE("Cannot find a version for interval[%s]", interval);
}
} else {
// We don't have explicit intervals. We can use the segment granularity to figure out what
// interval we need, but we might not have already locked it.
interval = granularitySpec.getSegmentGranularity().bucket(timestamp);
version = ParallelIndexSupervisorTask.findVersion(versions, interval);
if (version == null) {
final int maxAllowedLockCount = getIngestionSchema().getTuningConfig().getMaxAllowedLockCount();
if (maxAllowedLockCount >= 0 && locks.size() >= maxAllowedLockCount) {
throw new MaxAllowedLocksExceededException(maxAllowedLockCount);
}
// We don't have a lock for this interval, so we should lock it now.
final TaskLock lock = Preconditions.checkNotNull(
getToolbox().getTaskActionClient().submit(
new TimeChunkLockTryAcquireAction(TaskLockType.EXCLUSIVE, interval)
),
"Cannot acquire a lock for interval[%s]",
interval
);
if (lock.isRevoked()) {
throw new ISE(StringUtils.format("Lock for interval [%s] was revoked.", interval));
}
version = lock.getVersion();
}
}
return new NonnullPair<>(interval, version);
return AbstractBatchIndexTask.findIntervalAndVersion(getToolbox(), ingestionSchema, timestamp);
}
@Override

View File

@ -0,0 +1,182 @@
/*
* 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.common.task.batch.parallel;
import com.google.common.base.Preconditions;
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.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.ShardSpec;
import org.joda.time.Interval;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
public class TombstoneHelper
{
private final DataSchema dataSchema;
private final TaskActionClient taskActionClient;
private final Collection<DataSegment> pushedSegments;
public TombstoneHelper(
Collection<DataSegment> pushedSegments,
DataSchema dataSchema,
TaskActionClient taskActionClient
)
{
Preconditions.checkNotNull(pushedSegments, "pushedSegments");
Preconditions.checkNotNull(dataSchema, "dataSchema");
Preconditions.checkNotNull(taskActionClient, "taskActionClient");
this.dataSchema = dataSchema;
this.taskActionClient = taskActionClient;
this.pushedSegments = pushedSegments;
}
private List<Interval> getCondensedPushedSegmentsIntervals()
{
List<Interval> pushedSegmentsIntervals = new ArrayList<>();
for (DataSegment pushedSegment : pushedSegments) {
pushedSegmentsIntervals.add(pushedSegment.getInterval());
}
return JodaUtils.condenseIntervals(pushedSegmentsIntervals);
}
public Set<DataSegment> computeTombstones(Map<Interval, SegmentIdWithShardSpec> tombstoneIntervalsAndVersions)
{
Set<DataSegment> retVal = new HashSet<>();
String dataSource = dataSchema.getDataSource();
for (Map.Entry<Interval, SegmentIdWithShardSpec> tombstoneIntervalAndVersion : tombstoneIntervalsAndVersions.entrySet()) {
// now we have all the metadata to create the tombstone:
DataSegment tombstone =
createTombstoneForTimeChunkInterval(
dataSource,
tombstoneIntervalAndVersion.getValue().getVersion(),
tombstoneIntervalAndVersion.getValue().getShardSpec(),
tombstoneIntervalAndVersion.getKey()
);
retVal.add(tombstone);
}
return retVal;
}
public List<Interval> computeTombstoneIntervals() throws IOException
{
List<Interval> retVal = new ArrayList<>();
GranularitySpec granularitySpec = dataSchema.getGranularitySpec();
List<Interval> pushedSegmentsIntervals = getCondensedPushedSegmentsIntervals();
List<Interval> intervalsForUsedSegments = getCondensedUsedIntervals();
for (Interval timeChunkInterval : granularitySpec.sortedBucketIntervals()) {
// is it an empty time chunk?
boolean isEmpty = true;
for (Interval pushedSegmentCondensedInterval : pushedSegmentsIntervals) {
if (timeChunkInterval.overlaps(pushedSegmentCondensedInterval)) {
isEmpty = false;
break;
}
}
if (isEmpty) {
// this timeChunkInterval has no data, it is empty, thus it is a candidate for tombstone
// now check if it actually might overshadow a used segment
for (Interval usedSegmentInterval : intervalsForUsedSegments) {
if (timeChunkInterval.overlaps(usedSegmentInterval)) {
// yes it does overshadow...
retVal.add(timeChunkInterval);
break;
}
}
}
}
return retVal;
}
private DataSegment createTombstoneForTimeChunkInterval(String dataSource, String version, ShardSpec shardSpec, Interval timeChunkInterval)
{
// and the loadSpec is different too:
Map<String, Object> tombstoneLoadSpec = new HashMap<>();
// since loadspec comes from prototype it is guaranteed to be non-null
tombstoneLoadSpec.put("type", DataSegment.TOMBSTONE_LOADSPEC_TYPE);
tombstoneLoadSpec.put("path", null); // tombstones do not have any backing file
// Several fields do not apply to tombstones...
DataSegment.Builder dataSegmentBuilder =
DataSegment.builder()
.dataSource(dataSource)
.interval(timeChunkInterval) // interval is different
.version(version)
.shardSpec(shardSpec)
.loadSpec(tombstoneLoadSpec) // load spec is special for tombstone
.size(1); // in case coordinator segment balancing chokes with zero size
return dataSegmentBuilder.build();
}
/**
* Helper method to prune required tombstones. Only tombstones that cover used intervals will be created
* since those that not cover used intervals will be redundant.
* @return Intervals corresponding to used segments that overlap with any of the spec's input intervals
* @throws IOException If used segments cannot be retrieved
*/
public List<Interval> getCondensedUsedIntervals() throws IOException
{
List<Interval> retVal = new ArrayList<>();
List<Interval> condensedInputIntervals = JodaUtils.condenseIntervals(dataSchema.getGranularitySpec().inputIntervals());
if (!condensedInputIntervals.isEmpty()) {
Collection<DataSegment> usedSegmentsInInputInterval =
taskActionClient.submit(new RetrieveUsedSegmentsAction(
dataSchema.getDataSource(), null,
condensedInputIntervals,
Segments.ONLY_VISIBLE
));
for (DataSegment usedSegment : usedSegmentsInInputInterval) {
for (Interval condensedInputInterval : condensedInputIntervals) {
if (condensedInputInterval.overlaps(usedSegment.getInterval())) {
retVal.add(usedSegment.getInterval());
break;
}
}
}
}
return JodaUtils.condenseIntervals(retVal);
}
}

View File

@ -235,6 +235,7 @@ public class DruidInputSource extends AbstractInputSource implements SplittableI
//noinspection ConstantConditions
return FluentIterable
.from(partitionHolder)
.filter(chunk -> !chunk.getObject().isTombstone())
.transform(chunk -> new DruidSegmentInputEntity(segmentCacheManager, chunk.getObject(), holder.getInterval()));
}).iterator();

View File

@ -716,12 +716,17 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
final Set<DataSegment> compactedSegments = runTask(compactionTask);
usedSegments = getCoordinatorClient().fetchUsedSegmentsInDataSourceForIntervals(DATA_SOURCE, ImmutableList.of(INTERVAL_TO_INDEX));
// All the HOUR segments got dropped even if we do not have all MINUTES segments fully covering the 3 HOURS interval.
// All the HOUR segments got covered by tombstones even if we do not have all MINUTES segments fully covering the 3 HOURS interval.
// In fact, we only have 3 minutes of data out of the 3 hours interval.
Assert.assertEquals(3, usedSegments.size());
Assert.assertEquals(180, usedSegments.size());
int tombstonesCount = 0;
for (DataSegment segment : usedSegments) {
Assert.assertTrue(Granularities.MINUTE.isAligned(segment.getInterval()));
if (segment.isTombstone()) {
tombstonesCount++;
}
}
Assert.assertEquals(177, tombstonesCount);
}
@Test

View File

@ -119,6 +119,7 @@ import java.util.Set;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Future;
import java.util.stream.Collectors;
@RunWith(Parameterized.class)
public class CompactionTaskRunTest extends IngestionTestBase
@ -952,23 +953,52 @@ public class CompactionTaskRunTest extends IngestionTestBase
return;
}
// This creates HOUR segments with intervals of
// The following task creates (several, more than three, last time I checked, six) HOUR segments with intervals of
// - 2014-01-01T00:00:00/2014-01-01T01:00:00
// - 2014-01-01T01:00:00/2014-01-01T02:00:00
// - 2014-01-01T02:00:00/2014-01-01T03:00:00
runIndexTask();
// The six segments are:
// three rows in hour 00:
// 2014-01-01T00:00:00.000Z_2014-01-01T01:00:00.000Z with two rows
// 2014-01-01T00:00:00.000Z_2014-01-01T01:00:00.000Z_1 with one row
// three rows in hour 01:
// 2014-01-01T01:00:00.000Z_2014-01-01T02:00:00.000Z with two rows
// 2014-01-01T01:00:00.000Z_2014-01-01T02:00:00.000Z_1 with one row
// four rows in hour 02:
// 2014-01-01T02:00:00.000Z_2014-01-01T03:00:00.000Z with two rows
// 2014-01-01T02:00:00.000Z_2014-01-01T03:00:00.000Z_1 with two rows
// there are 10 rows total in data set
final Interval compactionPartialInterval = Intervals.of("2014-01-01T01:00:00/2014-01-01T02:00:00");
// Segments that did not belong in the compaction interval are expected unchanged
final Set<DataSegment> expectedSegments = new HashSet<>();
expectedSegments.addAll(
getStorageCoordinator().retrieveUsedSegmentsForIntervals(
DATA_SOURCE,
Collections.singletonList(Intervals.of("2014-01-01T02:00:00/2014-01-01T03:00:00")),
Segments.ONLY_VISIBLE
)
// maxRowsPerSegment is set to 2 inside the runIndexTask methods
Pair<TaskStatus, List<DataSegment>> result = runIndexTask();
Assert.assertEquals(6, result.rhs.size());
final Builder builder = new Builder(
DATA_SOURCE,
segmentCacheManagerFactory,
RETRY_POLICY_FACTORY
);
// Setup partial compaction:
// Change the granularity from HOUR to MINUTE through compaction for hour 01, there are three rows in the compaction interval,
// all three in the same timestamp (see TEST_ROWS), this should generate one segments (task will now use
// the default rows per segments since compaction's tuning config is null) in same minute and
// 59 tombstones to completely overshadow the existing hour 01 segment. Since the segments outside the
// compaction interval should remanin unchanged there should be a total of 1 + (2 + 59) + 2 = 64 segments
// **** PARTIAL COMPACTION: hour -> minute ****
final Interval compactionPartialInterval = Intervals.of("2014-01-01T01:00:00/2014-01-01T02:00:00");
final CompactionTask partialCompactionTask = builder
.segmentGranularity(Granularities.MINUTE)
// Set dropExisting to true
.inputSpec(new CompactionIntervalSpec(compactionPartialInterval, null), true)
.build();
final Pair<TaskStatus, List<DataSegment>> partialCompactionResult = runTask(partialCompactionTask);
Assert.assertTrue(partialCompactionResult.lhs.isSuccess());
// Segments that did not belong in the compaction interval (hours 00 and 02) are expected unchanged
// add 2 unchanged segments for hour 00:
final Set<DataSegment> expectedSegments = new HashSet<>();
expectedSegments.addAll(
getStorageCoordinator().retrieveUsedSegmentsForIntervals(
DATA_SOURCE,
@ -976,27 +1006,19 @@ public class CompactionTaskRunTest extends IngestionTestBase
Segments.ONLY_VISIBLE
)
);
final Builder builder = new Builder(
DATA_SOURCE,
segmentCacheManagerFactory,
RETRY_POLICY_FACTORY
// add 2 unchanged segments for hour 02:
expectedSegments.addAll(
getStorageCoordinator().retrieveUsedSegmentsForIntervals(
DATA_SOURCE,
Collections.singletonList(Intervals.of("2014-01-01T02:00:00/2014-01-01T03:00:00")),
Segments.ONLY_VISIBLE
)
);
final CompactionTask partialCompactionTask = builder
.segmentGranularity(Granularities.MINUTE)
// Set dropExisting to true
.inputSpec(new CompactionIntervalSpec(compactionPartialInterval, null), true)
.build();
final Pair<TaskStatus, List<DataSegment>> partialCompactionResult = runTask(partialCompactionTask);
Assert.assertTrue(partialCompactionResult.lhs.isSuccess());
// New segments that was compacted are expected. However, old segments of the compacted interval should be drop
// regardless of the new segments fully overshadow the old segments or not. Hence, we do not expect old segments
// of the 2014-01-01T01:00:00/2014-01-01T02:00:00 interval post-compaction
expectedSegments.addAll(partialCompactionResult.rhs);
Assert.assertEquals(64, expectedSegments.size());
// New segments that were compacted are expected. However, old segments of the compacted interval should be
// overshadowed by the new tombstones (59) being created for all minutes other than 01:01
final Set<DataSegment> segmentsAfterPartialCompaction = new HashSet<>(
getStorageCoordinator().retrieveUsedSegmentsForIntervals(
DATA_SOURCE,
@ -1004,18 +1026,40 @@ public class CompactionTaskRunTest extends IngestionTestBase
Segments.ONLY_VISIBLE
)
);
Assert.assertEquals(expectedSegments, segmentsAfterPartialCompaction);
final List<DataSegment> realSegmentsAfterPartialCompaction =
segmentsAfterPartialCompaction.stream()
.filter(s -> !s.isTombstone())
.collect(Collectors.toList());
final List<DataSegment> tombstonesAfterPartialCompaction =
segmentsAfterPartialCompaction.stream()
.filter(s -> s.isTombstone())
.collect(Collectors.toList());
Assert.assertEquals(59, tombstonesAfterPartialCompaction.size());
Assert.assertEquals(5, realSegmentsAfterPartialCompaction.size());
Assert.assertEquals(64, segmentsAfterPartialCompaction.size());
// Setup full compaction:
// Full Compaction with null segmentGranularity meaning that the original segmentGranularity is preserved.
// For the intervals, 2014-01-01T00:00:00.000Z/2014-01-01T01:00:00.000Z and 2014-01-01T02:00:00.000Z/2014-01-01T03:00:00.000Z
// the original segmentGranularity is HOUR from the initial ingestion.
// For the interval, 2014-01-01T01:00:00.000Z/2014-01-01T01:01:00.000Z, the original segmentGranularity is
// MINUTE from the partial compaction done earlier.
// Again since the tuningconfig for the compaction is null, the maxRowsPerSegment is the default so
// for hour 00 one real HOUR segment will be generated;
// for hour 01, one real minute segment plus 59 minute tombstones;
// and hour 02 one real HOUR segment for a total of 1 + (1+59) + 1 = 62 total segments
final CompactionTask fullCompactionTask = builder
.segmentGranularity(null)
// Set dropExisting to true
.inputSpec(new CompactionIntervalSpec(Intervals.of("2014-01-01/2014-01-02"), null), true)
.build();
// **** FULL COMPACTION ****
final Pair<TaskStatus, List<DataSegment>> fullCompactionResult = runTask(fullCompactionTask);
Assert.assertTrue(fullCompactionResult.lhs.isSuccess());
final List<DataSegment> segmentsAfterFullCompaction = new ArrayList<>(
getStorageCoordinator().retrieveUsedSegmentsForIntervals(
DATA_SOURCE,
@ -1026,25 +1070,33 @@ public class CompactionTaskRunTest extends IngestionTestBase
segmentsAfterFullCompaction.sort(
(s1, s2) -> Comparators.intervalsByStartThenEnd().compare(s1.getInterval(), s2.getInterval())
);
Assert.assertEquals(62, segmentsAfterFullCompaction.size());
final List<DataSegment> tombstonesAfterFullCompaction =
segmentsAfterFullCompaction.stream()
.filter(s -> s.isTombstone())
.collect(Collectors.toList());
Assert.assertEquals(59, tombstonesAfterFullCompaction.size());
final List<DataSegment> realSegmentsAfterFullCompaction =
segmentsAfterFullCompaction.stream()
.filter(s -> !s.isTombstone())
.collect(Collectors.toList());
Assert.assertEquals(3, realSegmentsAfterFullCompaction.size());
Assert.assertEquals(3, segmentsAfterFullCompaction.size());
// Full Compaction with null segmentGranularity meaning that the original segmentGrnaularity is perserved
// For the intervals, 2014-01-01T00:00:00.000Z/2014-01-01T01:00:00.000Z and 2014-01-01T02:00:00.000Z/2014-01-01T03:00:00.000Z
// the original segmentGranularity is HOUR from the initial ingestion.
// For the interval, 2014-01-01T01:00:00.000Z/2014-01-01T01:01:00.000Z, the original segmentGranularity is
// MINUTE from the partial compaction done earlier.
Assert.assertEquals(
Intervals.of("2014-01-01T00:00:00.000Z/2014-01-01T01:00:00.000Z"),
segmentsAfterFullCompaction.get(0).getInterval()
realSegmentsAfterFullCompaction.get(0).getInterval()
);
Assert.assertEquals(
Intervals.of("2014-01-01T01:00:00.000Z/2014-01-01T01:01:00.000Z"),
segmentsAfterFullCompaction.get(1).getInterval()
realSegmentsAfterFullCompaction.get(1).getInterval()
);
Assert.assertEquals(
Intervals.of("2014-01-01T02:00:00.000Z/2014-01-01T03:00:00.000Z"),
segmentsAfterFullCompaction.get(2).getInterval()
realSegmentsAfterFullCompaction.get(2).getInterval()
);
}
@Test

View File

@ -732,6 +732,44 @@ public class IndexTaskTest extends IngestionTestBase
Assert.assertEquals(0, segments.get(2).getShardSpec().getPartitionNum());
}
@Test
public void testIntervalNotSpecifiedWithReplace() throws Exception
{
File tmpDir = temporaryFolder.newFolder();
File tmpFile = File.createTempFile("druid", "index", tmpDir);
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
writer.write("2014-01-01T00:00:10Z,a,1\n");
writer.write("2014-01-01T01:00:20Z,b,1\n");
writer.write("2014-01-01T02:00:30Z,c,1\n");
}
// Expect exception if reingest with dropExisting and null intervals is attempted
expectedException.expect(IAE.class);
expectedException.expectMessage(
"GranularitySpec's intervals cannot be empty when setting dropExisting to true."
);
IndexTask indexTask = new IndexTask(
null,
null,
createDefaultIngestionSpec(
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.MINUTE,
null
),
null,
createTuningConfigWithMaxRowsPerSegment(2, true),
false,
true
),
null
);
}
@Test
public void testCSVFileWithHeader() throws Exception
{
@ -2214,7 +2252,7 @@ public class IndexTaskTest extends IngestionTestBase
}
@Test
public void testOldSegmentNotDropWhenDropFlagFalse() throws Exception
public void testOldSegmentNotReplacedWhenDropFlagFalse() throws Exception
{
File tmpDir = temporaryFolder.newFolder();
@ -2298,16 +2336,16 @@ public class IndexTaskTest extends IngestionTestBase
}
@Test
public void testOldSegmentNotDropWhenDropFlagTrueSinceIngestionIntervalDoesNotContainsOldSegment() throws Exception
public void testOldSegmentNotCoveredByTombstonesWhenDropFlagTrueSinceIngestionIntervalDoesNotContainsOldSegment() throws Exception
{
File tmpDir = temporaryFolder.newFolder();
File tmpFile = File.createTempFile("druid", "index", tmpDir);
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
writer.write("2014-01-01T00:00:10Z,a,1\n");
writer.write("2014-01-01T01:00:20Z,b,1\n");
writer.write("2014-01-01T02:00:30Z,c,1\n");
writer.write("2014-01-01T01:00:10Z,a,1\n");
writer.write("2014-01-01T01:10:20Z,b,1\n");
writer.write("2014-01-01T01:20:30Z,c,1\n");
}
IndexTask indexTask = new IndexTask(
@ -2317,9 +2355,9 @@ public class IndexTaskTest extends IngestionTestBase
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.YEAR,
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
Collections.singletonList(Intervals.of("2014-01-01T01:00:00Z/2014-01-01T02:00:00Z"))
),
null,
createTuningConfigWithMaxRowsPerSegment(10, true),
@ -2329,14 +2367,14 @@ public class IndexTaskTest extends IngestionTestBase
null
);
// Ingest data with YEAR segment granularity
// Ingest data with DAY segment granularity
List<DataSegment> segments = runTask(indexTask).rhs;
Assert.assertEquals(1, segments.size());
Set<DataSegment> usedSegmentsBeforeOverwrite = Sets.newHashSet(getSegmentsMetadataManager().iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DATASOURCE, Intervals.ETERNITY, true).get());
Assert.assertEquals(1, usedSegmentsBeforeOverwrite.size());
for (DataSegment segment : usedSegmentsBeforeOverwrite) {
Assert.assertTrue(Granularities.YEAR.isAligned(segment.getInterval()));
Assert.assertTrue(Granularities.DAY.isAligned(segment.getInterval()));
}
indexTask = new IndexTask(
@ -2346,9 +2384,9 @@ public class IndexTaskTest extends IngestionTestBase
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.MINUTE,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
Collections.singletonList(Intervals.of("2014-01-01T01:10:00Z/2014-01-01T02:00:00Z"))
),
null,
createTuningConfigWithMaxRowsPerSegment(10, true),
@ -2358,31 +2396,39 @@ public class IndexTaskTest extends IngestionTestBase
null
);
// Ingest data with overwrite and MINUTE segment granularity
// Ingest data with overwrite and HOUR segment granularity
segments = runTask(indexTask).rhs;
Assert.assertEquals(3, segments.size());
Assert.assertEquals(1, segments.size());
Set<DataSegment> usedSegmentsBeforeAfterOverwrite = Sets.newHashSet(getSegmentsMetadataManager().iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DATASOURCE, Intervals.ETERNITY, true).get());
Assert.assertEquals(4, usedSegmentsBeforeAfterOverwrite.size());
int yearSegmentFound = 0;
int minuteSegmentFound = 0;
Assert.assertEquals(2, usedSegmentsBeforeAfterOverwrite.size());
int segmentFound = 0;
int tombstonesFound = 0;
int hourSegmentFound = 0;
int daySegmentFound = 0;
for (DataSegment segment : usedSegmentsBeforeAfterOverwrite) {
// Used segments after overwrite will contain 1 old segment with YEAR segmentGranularity (from first ingestion)
// and 3 new segments with MINUTE segmentGranularity (from second ingestion)
if (usedSegmentsBeforeOverwrite.contains(segment)) {
Assert.assertTrue(Granularities.YEAR.isAligned(segment.getInterval()));
yearSegmentFound++;
if (segment.isTombstone()) {
tombstonesFound++;
} else {
Assert.assertTrue(Granularities.MINUTE.isAligned(segment.getInterval()));
minuteSegmentFound++;
segmentFound++;
}
if (usedSegmentsBeforeOverwrite.contains(segment)) {
Assert.assertTrue(Granularities.DAY.isAligned(segment.getInterval()));
daySegmentFound++;
} else {
Assert.assertTrue(Granularities.HOUR.isAligned(segment.getInterval()));
hourSegmentFound++;
}
}
Assert.assertEquals(1, yearSegmentFound);
Assert.assertEquals(3, minuteSegmentFound);
Assert.assertEquals(1, daySegmentFound);
Assert.assertEquals(1, hourSegmentFound);
Assert.assertEquals(2, segmentFound);
Assert.assertEquals(0, tombstonesFound);
}
@Test
public void testOldSegmentDropWhenDropFlagTrueAndIngestionIntervalContainsOldSegment() throws Exception
public void testOldSegmentCoveredByTombstonesWhenDropFlagTrueSinceIngestionIntervalContainsOldSegment() throws Exception
{
File tmpDir = temporaryFolder.newFolder();
@ -2401,9 +2447,9 @@ public class IndexTaskTest extends IngestionTestBase
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.YEAR,
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
Collections.singletonList(Intervals.of("2014-01-01T01:00:00Z/2014-01-01T02:00:00Z"))
),
null,
createTuningConfigWithMaxRowsPerSegment(10, true),
@ -2413,14 +2459,14 @@ public class IndexTaskTest extends IngestionTestBase
null
);
// Ingest data with YEAR segment granularity
// Ingest data with DAY segment granularity
List<DataSegment> segments = runTask(indexTask).rhs;
Assert.assertEquals(1, segments.size());
Set<DataSegment> usedSegmentsBeforeOverwrite = Sets.newHashSet(getSegmentsMetadataManager().iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DATASOURCE, Intervals.ETERNITY, true).get());
Assert.assertEquals(1, usedSegmentsBeforeOverwrite.size());
for (DataSegment segment : usedSegmentsBeforeOverwrite) {
Assert.assertTrue(Granularities.YEAR.isAligned(segment.getInterval()));
Assert.assertTrue(Granularities.DAY.isAligned(segment.getInterval()));
}
indexTask = new IndexTask(
@ -2430,9 +2476,9 @@ public class IndexTaskTest extends IngestionTestBase
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.MINUTE,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014-01-01/2015-01-01"))
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
),
null,
createTuningConfigWithMaxRowsPerSegment(10, true),
@ -2442,34 +2488,34 @@ public class IndexTaskTest extends IngestionTestBase
null
);
// Ingest data with overwrite and MINUTE segment granularity
// Ingest data with overwrite and HOUR segment granularity
segments = runTask(indexTask).rhs;
Assert.assertEquals(3, segments.size());
Assert.assertEquals(24, segments.size());
Set<DataSegment> usedSegmentsBeforeAfterOverwrite = Sets.newHashSet(getSegmentsMetadataManager().iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DATASOURCE, Intervals.ETERNITY, true).get());
Assert.assertEquals(3, usedSegmentsBeforeAfterOverwrite.size());
Assert.assertEquals(24, usedSegmentsBeforeAfterOverwrite.size());
for (DataSegment segment : usedSegmentsBeforeAfterOverwrite) {
// Used segments after overwrite and drop will contain only the
// 3 new segments with MINUTE segmentGranularity (from second ingestion)
// 24 new segments with HOUR segmentGranularity (from second ingestion)
if (usedSegmentsBeforeOverwrite.contains(segment)) {
Assert.fail();
} else {
Assert.assertTrue(Granularities.MINUTE.isAligned(segment.getInterval()));
Assert.assertTrue(Granularities.HOUR.isAligned(segment.getInterval()));
}
}
}
@Test
public void testOldSegmentNotDropWhenDropFlagTrueAndIngestionIntervalEmpty() throws Exception
public void verifyPublishingOnlyTombstones() throws Exception
{
File tmpDir = temporaryFolder.newFolder();
File tmpFile = File.createTempFile("druid", "index", tmpDir);
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
writer.write("2014-01-01T00:00:10Z,a,1\n");
writer.write("2014-01-01T01:00:20Z,b,1\n");
writer.write("2014-01-01T02:00:30Z,c,1\n");
writer.write("2014-03-01T00:00:10Z,a,1\n");
writer.write("2014-03-01T01:00:20Z,b,1\n");
writer.write("2014-03-01T02:00:30Z,c,1\n");
}
IndexTask indexTask = new IndexTask(
@ -2479,9 +2525,9 @@ public class IndexTaskTest extends IngestionTestBase
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.YEAR,
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
Collections.singletonList(Intervals.of("2014-01-03/2014-04-01"))
),
null,
createTuningConfigWithMaxRowsPerSegment(10, true),
@ -2491,14 +2537,24 @@ public class IndexTaskTest extends IngestionTestBase
null
);
// Ingest data with YEAR segment granularity
// Ingest data with DAY segment granularity
List<DataSegment> segments = runTask(indexTask).rhs;
Assert.assertEquals(1, segments.size());
Set<DataSegment> usedSegmentsBeforeOverwrite = Sets.newHashSet(getSegmentsMetadataManager().iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DATASOURCE, Intervals.ETERNITY, true).get());
Assert.assertEquals(1, usedSegmentsBeforeOverwrite.size());
for (DataSegment segment : usedSegmentsBeforeOverwrite) {
Assert.assertTrue(Granularities.YEAR.isAligned(segment.getInterval()));
Assert.assertTrue(Granularities.DAY.isAligned(segment.getInterval()));
}
// create new data but with an ingestion interval appropriate to filter it all out so that only tombstones
// are created:
tmpDir = temporaryFolder.newFolder();
tmpFile = File.createTempFile("druid", "index", tmpDir);
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
writer.write("2014-01-01T00:00:10Z,a,1\n");
writer.write("2014-01-01T01:00:20Z,b,1\n");
writer.write("2014-12-01T02:00:30Z,c,1\n");
}
indexTask = new IndexTask(
@ -2508,9 +2564,9 @@ public class IndexTaskTest extends IngestionTestBase
jsonMapper,
tmpDir,
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Granularities.MINUTE,
null
Collections.singletonList(Intervals.of("2014-03-01/2014-04-01")) // filter out all data
),
null,
createTuningConfigWithMaxRowsPerSegment(10, true),
@ -2520,29 +2576,14 @@ public class IndexTaskTest extends IngestionTestBase
null
);
// Ingest data with overwrite and MINUTE segment granularity
// Ingest data with overwrite and same segment granularity
segments = runTask(indexTask).rhs;
Assert.assertEquals(3, segments.size());
Set<DataSegment> usedSegmentsBeforeAfterOverwrite = Sets.newHashSet(getSegmentsMetadataManager().iterateAllUsedNonOvershadowedSegmentsForDatasourceInterval(DATASOURCE, Intervals.ETERNITY, true).get());
Assert.assertEquals(4, usedSegmentsBeforeAfterOverwrite.size());
int yearSegmentFound = 0;
int minuteSegmentFound = 0;
for (DataSegment segment : usedSegmentsBeforeAfterOverwrite) {
// Used segments after overwrite will contain 1 old segment with YEAR segmentGranularity (from first ingestion)
// and 3 new segments with MINUTE segmentGranularity (from second ingestion)
if (usedSegmentsBeforeOverwrite.contains(segment)) {
Assert.assertTrue(Granularities.YEAR.isAligned(segment.getInterval()));
yearSegmentFound++;
} else {
Assert.assertTrue(Granularities.MINUTE.isAligned(segment.getInterval()));
minuteSegmentFound++;
}
}
Assert.assertEquals(1, yearSegmentFound);
Assert.assertEquals(3, minuteSegmentFound);
Assert.assertEquals(1, segments.size()); // one tombstone
Assert.assertTrue(segments.get(0).isTombstone());
}
@Test
public void testErrorWhenDropFlagTrueAndOverwriteFalse() throws Exception
{

View File

@ -59,6 +59,7 @@ import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.TombstoneLoadSpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
@ -99,7 +100,11 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
super(transientTaskFailureRate, transientApiCallFailureRate);
this.lockGranularity = lockGranularity;
this.useInputFormatApi = useInputFormatApi;
getObjectMapper().registerSubtypes(ParallelIndexTuningConfig.class, DruidInputSource.class);
getObjectMapper().registerSubtypes(
ParallelIndexTuningConfig.class,
DruidInputSource.class,
TombstoneLoadSpec.class
);
}
boolean isUseInputFormatApi()
@ -131,6 +136,7 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
partitionsSpec,
maxNumConcurrentSubTasks,
expectedTaskStatus,
false,
false
);
}
@ -141,12 +147,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
@Nullable InputFormat inputFormat,
@Nullable ParseSpec parseSpec,
Interval interval,
File inputDir,
File inputDirectory,
String filter,
PartitionsSpec partitionsSpec,
int maxNumConcurrentSubTasks,
TaskState expectedTaskStatus,
boolean appendToExisting
boolean appendToExisting,
boolean dropExisting
)
{
final ParallelIndexSupervisorTask task = createTask(
@ -155,11 +162,12 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
inputFormat,
parseSpec,
interval,
inputDir,
inputDirectory,
filter,
partitionsSpec,
maxNumConcurrentSubTasks,
appendToExisting
appendToExisting,
dropExisting
);
return runTask(task, expectedTaskStatus);
@ -190,11 +198,12 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
@Nullable InputFormat inputFormat,
@Nullable ParseSpec parseSpec,
Interval interval,
File inputDir,
File inputDirectory,
String filter,
PartitionsSpec partitionsSpec,
int maxNumConcurrentSubTasks,
boolean appendToExisting
boolean appendToExisting,
boolean dropExisting
)
{
GranularitySpec granularitySpec = new UniformGranularitySpec(
@ -215,10 +224,10 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
Preconditions.checkArgument(parseSpec == null);
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
null,
new LocalInputSource(inputDir, filter),
new LocalInputSource(inputDirectory, filter),
inputFormat,
appendToExisting,
null
dropExisting
);
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
@ -235,8 +244,9 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
} else {
Preconditions.checkArgument(inputFormat == null);
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, filter, null),
appendToExisting
new LocalFirehoseFactory(inputDirectory, filter, null),
appendToExisting,
dropExisting
);
//noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec(

View File

@ -40,6 +40,7 @@ import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.HashPartitionFunction;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
@ -157,9 +158,15 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
inputIntervals.sort(Comparators.intervalsByStartThenEnd());
}
// The next test also verifies replace functionality. Now, they are together to save on test execution time
// due to Travis CI 10 minute default running time (with no output) -- having it separate made it
// last longer. At some point we should really simplify this file, so it runs faster (splitting, etc.)
@Test
public void testRun() throws Exception
{
// verify dropExisting false:
final Integer maxRowsPerSegment = numShards == null ? 10 : null;
final Set<DataSegment> publishedSegments = runTask(createTask(
new HashedPartitionsSpec(
@ -167,6 +174,8 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
numShards,
ImmutableList.of("dim1", "dim2")
),
inputDir,
false,
false
), TaskState.SUCCESS);
@ -175,6 +184,49 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
numShards
);
assertHashedPartition(publishedSegments, expectedIntervalToNumSegments);
// verify dropExisting true:
if (intervalToIndex == null) {
// replace only works when intervals are provided
return;
}
final Set<DataSegment> publishedSegmentsAfterReplace = runTask(createTask(
new HashedPartitionsSpec(
maxRowsPerSegment,
numShards,
ImmutableList.of("dim1", "dim2")
),
newInputDirForReplace(),
false,
true
), TaskState.SUCCESS);
final Map<Interval, Integer> expectedIntervalToNumSegmentsAfterReplace = computeExpectedIntervalToNumSegments(
maxRowsPerSegment,
numShards
);
// Regardless of whether numShards is set or not the replace will put data in six intervals.
// When numShards are set (2) it will generate 12 segments. When not, the hash ingestion code will estimate
// one shard perinterval thus siz segments:
// adjust expected wrt to tombstones:
int tombstones = 0;
for (DataSegment ds : publishedSegmentsAfterReplace) {
if (ds.isTombstone()) {
expectedIntervalToNumSegmentsAfterReplace.put(ds.getInterval(), 1);
tombstones++;
} else if (numShards == null) {
expectedIntervalToNumSegmentsAfterReplace.put(ds.getInterval(), 1);
}
}
Assert.assertEquals(5, tombstones); // five tombstones
int expectedSegments = 12;
if (numShards == null) {
expectedSegments = 6;
}
Assert.assertEquals(expectedSegments, publishedSegmentsAfterReplace.size() - tombstones); // six segments
assertHashedPartition(publishedSegmentsAfterReplace, expectedIntervalToNumSegmentsAfterReplace);
}
@Test
@ -188,7 +240,8 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
ImmutableList.of("dim1", "dim2"),
HashPartitionFunction.MURMUR3_32_ABS
),
false), TaskState.SUCCESS);
inputDir, false, false
), TaskState.SUCCESS);
final Map<Interval, Integer> expectedIntervalToNumSegments = computeExpectedIntervalToNumSegments(
maxRowsPerSegment,
numShards
@ -224,7 +277,8 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
runTask(
createTask(
new HashedPartitionsSpec(null, numShards, ImmutableList.of("dim1", "dim2")),
false),
inputDir, false, false
),
TaskState.SUCCESS)
);
// Append
@ -232,14 +286,16 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
runTask(
createTask(
new DynamicPartitionsSpec(5, null),
true),
inputDir, true, false
),
TaskState.SUCCESS));
// And append again
publishedSegments.addAll(
runTask(
createTask(
new DynamicPartitionsSpec(10, null),
true),
inputDir, true, false
),
TaskState.SUCCESS)
);
@ -273,7 +329,9 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
private ParallelIndexSupervisorTask createTask(
PartitionsSpec partitionsSpec,
boolean appendToExisting
File inputDirectory,
boolean appendToExisting,
boolean dropExisting
)
{
if (isUseInputFormatApi()) {
@ -283,11 +341,12 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
INPUT_FORMAT,
null,
intervalToIndex,
inputDir,
inputDirectory,
"test_*",
partitionsSpec,
maxNumConcurrentSubTasks,
appendToExisting
appendToExisting,
dropExisting
);
} else {
return createTask(
@ -296,11 +355,12 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
null,
PARSE_SPEC,
intervalToIndex,
inputDir,
inputDirectory,
"test_*",
partitionsSpec,
maxNumConcurrentSubTasks,
appendToExisting
appendToExisting,
dropExisting
);
}
}
@ -321,30 +381,61 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
List<DataSegment> segmentsInInterval = entry.getValue();
Assert.assertEquals(expectedIntervalToNumSegments.get(interval).intValue(), segmentsInInterval.size());
for (DataSegment segment : segmentsInInterval) {
Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass());
final HashBasedNumberedShardSpec shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec();
Assert.assertEquals(HashPartitionFunction.MURMUR3_32_ABS, shardSpec.getPartitionFunction());
HashBasedNumberedShardSpec shardSpec = null;
if (segment.isTombstone()) {
Assert.assertSame(TombstoneShardSpec.class, segment.getShardSpec().getClass());
} else {
Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass());
shardSpec = (HashBasedNumberedShardSpec) segment.getShardSpec();
Assert.assertEquals(HashPartitionFunction.MURMUR3_32_ABS, shardSpec.getPartitionFunction());
}
List<ScanResultValue> results = querySegment(segment, ImmutableList.of("dim1", "dim2"), tempSegmentDir);
final int hash = shardSpec.getPartitionFunction().hash(
HashBasedNumberedShardSpec.serializeGroupKey(
getObjectMapper(),
(List<Object>) results.get(0).getEvents()
),
shardSpec.getNumBuckets()
);
for (ScanResultValue value : results) {
Assert.assertEquals(
hash,
shardSpec.getPartitionFunction().hash(
HashBasedNumberedShardSpec.serializeGroupKey(
getObjectMapper(),
(List<Object>) value.getEvents()
),
shardSpec.getNumBuckets()
)
if (segment.isTombstone()) {
Assert.assertTrue(results.isEmpty());
} else {
final int hash = shardSpec.getPartitionFunction().hash(
HashBasedNumberedShardSpec.serializeGroupKey(
getObjectMapper(),
(List<Object>) results.get(0).getEvents()
),
shardSpec.getNumBuckets()
);
for (ScanResultValue value : results) {
Assert.assertEquals(
hash,
shardSpec.getPartitionFunction().hash(
HashBasedNumberedShardSpec.serializeGroupKey(
getObjectMapper(),
(List<Object>) value.getEvents()
),
shardSpec.getNumBuckets()
)
);
}
}
}
}
}
private File newInputDirForReplace() throws IOException
{
File inputDirectory = temporaryFolder.newFolder("dataReplace");
// set up data
Set<Integer> fileIds = new HashSet<>();
fileIds.add(3);
fileIds.add(7);
fileIds.add(9);
for (Integer i : fileIds) {
try (final Writer writer =
Files.newBufferedWriter(new File(inputDirectory, "test_" + i).toPath(), StandardCharsets.UTF_8)) {
for (int j = 0; j < 10; j++) {
writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", i + 1, j + 10, i));
writer.write(StringUtils.format("2017-12-%d,%d,%d th test file\n", i + 2, j + 11, i));
}
}
}
return inputDirectory;
}
}

View File

@ -126,6 +126,7 @@ public class MultiPhaseParallelIndexingRowStatsTest extends AbstractMultiPhasePa
"test_*",
new HashedPartitionsSpec(null, numShards, ImmutableList.of("dim1", "dim2"), null),
maxNumConcurrentSubTasks,
false,
false
);
@ -163,6 +164,7 @@ public class MultiPhaseParallelIndexingRowStatsTest extends AbstractMultiPhasePa
//new DimensionRangePartitionsSpec(targetRowsPerSegment, null, DIMS, false),
new SingleDimensionPartitionsSpec(targetRowsPerSegment, null, DIM1, false),
10,
false,
false
);
Map<String, Object> expectedReports = buildExpectedTaskReportParallel(

View File

@ -219,7 +219,8 @@ public class PartialCompactionTest extends AbstractMultiPhaseParallelIndexingTes
partitionsSpec,
2,
expectedTaskState,
appendToExisting
appendToExisting,
false
);
}

View File

@ -190,6 +190,40 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
return intervalToDims;
}
private static SetMultimap<Interval, List<Object>> createInputFilesForReplace(File inputDir, boolean useMultivalueDim)
throws IOException
{
SetMultimap<Interval, List<Object>> intervalToDims = HashMultimap.create();
Set<Integer> fileIds = new HashSet<>();
fileIds.add(1);
fileIds.add(7);
fileIds.add(9);
for (Integer fileIndex : fileIds) {
Path path = new File(inputDir, TEST_FILE_NAME_PREFIX + fileIndex).toPath();
try (final Writer writer = Files.newBufferedWriter(path, StandardCharsets.UTF_8)) {
for (int i = 11; i < 2 * (NUM_ROW / DIM_FILE_CARDINALITY); i++) {
for (int d = 0; d < DIM_FILE_CARDINALITY; d++) {
int rowIndex = i * DIM_FILE_CARDINALITY + d;
String dim1Value = createDim1Value(rowIndex, fileIndex, useMultivalueDim);
// This is the original row
writeRow(writer, i + d, dim1Value, fileIndex, intervalToDims);
// This row should get rolled up with original row
writeRow(writer, i + d, dim1Value, fileIndex, intervalToDims);
// This row should not get rolled up with original row
writeRow(writer, i + d, dim1Value, fileIndex + NUM_FILE, intervalToDims);
}
}
}
}
return intervalToDims;
}
@Nullable
private static String createDim1Value(int rowIndex, int fileIndex, boolean useMultivalueDim)
{
@ -221,10 +255,15 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
intervalToDims.put(interval, Arrays.asList(dim1Value, dim2Value));
}
// The next test also verifies replace functionality. Now, they are together to save on test execution time
// due to Travis CI 10 minute default running time (with no output) -- having it separate made it
// last longer. At some point we should really simplify this file, so it runs faster (splitting, etc.)
@Test
public void createsCorrectRangePartitions() throws Exception
{
int targetRowsPerSegment = NUM_ROW * 2 / DIM_FILE_CARDINALITY / NUM_PARTITION;
// verify dropExisting false
final Set<DataSegment> publishedSegments = runTask(runTestTask(
new DimensionRangePartitionsSpec(
targetRowsPerSegment,
@ -232,12 +271,47 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
Collections.singletonList(DIM1),
false
),
inputDir,
false,
false
), useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS);
if (!useMultivalueDim) {
assertRangePartitions(publishedSegments);
}
// verify dropExisting true
if (intervalToIndex == null) {
// dropExisting requires intervals
return;
}
File inputDirectory = temporaryFolder.newFolder("dataReplace");
createInputFilesForReplace(inputDirectory, useMultivalueDim);
final Set<DataSegment> publishedSegmentsAfterReplace = runTask(runTestTask(
new DimensionRangePartitionsSpec(
targetRowsPerSegment,
null,
Collections.singletonList(DIM1),
false
),
inputDirectory,
false,
true
), useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS);
int tombstones = 0;
for (DataSegment ds : publishedSegmentsAfterReplace) {
if (ds.isTombstone()) {
tombstones++;
}
}
if (!useMultivalueDim) {
Assert.assertEquals(11, tombstones);
Assert.assertEquals(10, publishedSegmentsAfterReplace.size() - tombstones);
}
}
@Test
@ -256,6 +330,8 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
DIM1,
false
),
inputDir,
false,
false
), TaskState.SUCCESS)
);
@ -263,14 +339,18 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
publishedSegments.addAll(
runTask(runTestTask(
new DynamicPartitionsSpec(5, null),
true
inputDir,
true,
false
), TaskState.SUCCESS)
);
// And append again
publishedSegments.addAll(
runTask(runTestTask(
new DynamicPartitionsSpec(10, null),
true
inputDir,
true,
false
), TaskState.SUCCESS)
);
@ -304,7 +384,9 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
private ParallelIndexSupervisorTask runTestTask(
PartitionsSpec partitionsSpec,
boolean appendToExisting
File inputDirectory,
boolean appendToExisting,
boolean dropExisting
)
{
if (isUseInputFormatApi()) {
@ -314,11 +396,12 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
INPUT_FORMAT,
null,
intervalToIndex,
inputDir,
inputDirectory,
TEST_FILE_NAME_PREFIX + "*",
partitionsSpec,
maxNumConcurrentSubTasks,
appendToExisting
appendToExisting,
dropExisting
);
} else {
return createTask(
@ -327,11 +410,12 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
null,
PARSE_SPEC,
intervalToIndex,
inputDir,
inputDirectory,
TEST_FILE_NAME_PREFIX + "*",
partitionsSpec,
maxNumConcurrentSubTasks,
appendToExisting
appendToExisting,
dropExisting
);
}
}

View File

@ -779,6 +779,17 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
boolean appendToExisting,
boolean splittableInputSource
)
{
return newTask(interval, segmentGranularity, appendToExisting, splittableInputSource, false);
}
private ParallelIndexSupervisorTask newTask(
@Nullable Interval interval,
Granularity segmentGranularity,
boolean appendToExisting,
boolean splittableInputSource,
boolean isReplace
)
{
return newTask(
interval,

View File

@ -0,0 +1,123 @@
/*
* 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.common.task.batch.parallel;
import org.apache.druid.indexing.common.actions.TaskAction;
import org.apache.druid.indexing.common.actions.TaskActionClient;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.mockito.ArgumentMatchers.any;
public class TombstoneHelperTest
{
private final TaskActionClient taskActionClient = Mockito.mock(TaskActionClient.class);
@Test
public void noTombstonesWhenNoDataInInputIntervalAndNoExistingSegments() throws Exception
{
Interval interval = Intervals.of("2020-04-01/2020-04-04");
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
Collections.singletonList(interval)
);
DataSchema dataSchema =
new DataSchema("test", null, null, null, granularitySpec, null);
// no segments will be pushed when all rows are thrown away, assume that:
List<DataSegment> pushedSegments = Collections.emptyList();
// Assume no used segments :
Mockito.when(taskActionClient.submit(any(TaskAction.class))).thenReturn(Collections.emptyList());
TombstoneHelper tombstoneHelper = new TombstoneHelper(
pushedSegments,
dataSchema,
taskActionClient
);
List<Interval> tombstoneIntervals = tombstoneHelper.computeTombstoneIntervals();
Assert.assertTrue(tombstoneIntervals.isEmpty());
Map<Interval, SegmentIdWithShardSpec> intervalToLockVersion = Collections.emptyMap();
Set<DataSegment> tombstones = tombstoneHelper.computeTombstones(intervalToLockVersion);
Assert.assertEquals(0, tombstones.size());
}
@Test
public void tombstonesCreatedWhenNoDataInInputIntervalAndExistingSegments() throws Exception
{
Interval interval = Intervals.of("2020-04-01/2020-04-04");
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
Collections.singletonList(interval)
);
DataSchema dataSchema =
new DataSchema("test", null, null, null, granularitySpec, null);
// no segments will be pushed when all rows are thrown away, assume that:
List<DataSegment> pushedSegments = Collections.emptyList();
// Assume used segments exist:
DataSegment existingUsedSegment =
DataSegment.builder()
.dataSource("test")
.interval(interval)
.version("oldVersion")
.size(100)
.build();
Assert.assertFalse(existingUsedSegment.isTombstone());
Mockito.when(taskActionClient.submit(any(TaskAction.class)))
.thenReturn(Collections.singletonList(existingUsedSegment));
TombstoneHelper tombstoneHelper = new TombstoneHelper(
pushedSegments,
dataSchema,
taskActionClient
);
List<Interval> tombstoneIntervals = tombstoneHelper.computeTombstoneIntervals();
Assert.assertEquals(3, tombstoneIntervals.size());
Map<Interval, SegmentIdWithShardSpec> intervalToVersion = new HashMap<>();
for (Interval ti : tombstoneIntervals) {
intervalToVersion.put(
ti,
new SegmentIdWithShardSpec("test", ti, "newVersion", new TombstoneShardSpec())
);
}
Set<DataSegment> tombstones = tombstoneHelper.computeTombstones(intervalToVersion);
Assert.assertEquals(3, tombstones.size());
tombstones.forEach(ts -> Assert.assertTrue(ts.isTombstone()));
}
}

View File

@ -327,17 +327,22 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
LOG.info("Auto compaction test with DAY segment granularity");
// Since dropExisting is set to true...
// The earlier segment with YEAR granularity will be dropped post-compaction
// Hence, we will only have 2013-08-31 to 2013-09-01 and 2013-09-01 to 2013-09-02.
// The earlier segment with YEAR granularity will be completely covered, overshadowed, by the
// new DAY segments for data and tombstones for days with no data
// Hence, we will only have 2013-08-31 to 2013-09-01 and 2013-09-01 to 2013-09-02
// plus 363 tombstones
final List<String> intervalsAfterYEARCompactionButBeforeDAYCompaction =
coordinator.getSegmentIntervals(fullDatasourceName);
expectedIntervalAfterCompaction = new ArrayList<>();
for (String interval : intervalsBeforeCompaction) {
for (String interval : intervalsAfterYEARCompactionButBeforeDAYCompaction) {
for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) {
expectedIntervalAfterCompaction.add(newinterval.toString());
}
}
forceTriggerAutoCompaction(2);
forceTriggerAutoCompaction(365);
verifyQuery(INDEX_QUERIES_RESOURCE);
verifySegmentsCompacted(2, 1000);
verifyTombstones(363);
verifySegmentsCompacted(365, 1000);
checkCompactionIntervals(expectedIntervalAfterCompaction);
}
}
@ -513,7 +518,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true);
List<String> expectedIntervalAfterCompaction = new ArrayList<>();
// We wil have one segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR)
// We will still have one visible segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR)
// and four overshadowed segments
for (String interval : intervalsBeforeCompaction) {
for (Interval newinterval : newGranularity.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) {
expectedIntervalAfterCompaction.add(newinterval.toString());
@ -535,22 +541,27 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
checkCompactionIntervals(expectedIntervalAfterCompaction);
newGranularity = Granularities.MONTH;
// Set dropExisting to true
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true);
final List<String> intervalsAfterYEARButBeforeMONTHCompaction =
coordinator.getSegmentIntervals(fullDatasourceName);
// Since dropExisting is set to true...
// This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity
submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true);
// verify:
expectedIntervalAfterCompaction = new ArrayList<>();
// The previous segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) will be dropped
// We will only have one segments with interval of 2013-09-01/2013-10-01 (compacted with MONTH)
// and one segments with interval of 2013-10-01/2013-11-01 (compacted with MONTH)
for (String interval : intervalsBeforeCompaction) {
// The previous segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) will be
// completely overshadowed by a combination of tombstones and segments with data.
// We will only have one segment with interval of 2013-08-01/2013-09-01 (compacted with MONTH)
// and one segment with interval of 2013-09-01/2013-10-01 (compacted with MONTH)
// plus ten tombstones for the remaining months, thus expecting 12 intervals...
for (String interval : intervalsAfterYEARButBeforeMONTHCompaction) {
for (Interval newinterval : Granularities.MONTH.getIterable(new Interval(interval, ISOChronology.getInstanceUTC()))) {
expectedIntervalAfterCompaction.add(newinterval.toString());
}
}
forceTriggerAutoCompaction(2);
forceTriggerAutoCompaction(12);
verifyQuery(INDEX_QUERIES_RESOURCE);
verifySegmentsCompacted(2, MAX_ROWS_PER_SEGMENT_COMPACTED);
verifyTombstones(10);
verifySegmentsCompacted(12, MAX_ROWS_PER_SEGMENT_COMPACTED);
checkCompactionIntervals(expectedIntervalAfterCompaction);
}
}
@ -1125,6 +1136,18 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
);
}
private void verifyTombstones(int expectedCompactedTombstoneCount)
{
List<DataSegment> segments = coordinator.getFullSegmentsMetadata(fullDatasourceName);
int actualTombstoneCount = 0;
for (DataSegment segment : segments) {
if (segment.isTombstone()) {
actualTombstoneCount++;
}
}
Assert.assertEquals(actualTombstoneCount, expectedCompactedTombstoneCount);
}
private void verifySegmentsCompacted(PartitionsSpec partitionsSpec, int expectedCompactedSegmentCount)
{
List<DataSegment> segments = coordinator.getFullSegmentsMetadata(fullDatasourceName);

View File

@ -409,16 +409,47 @@ public abstract class AbstractITBatchIndexTest extends AbstractIndexerTest
);
ITRetryUtil.retryUntilTrue(
() -> {
int segmentCount = coordinator.getAvailableSegments(
List<DataSegment> segments = coordinator.getAvailableSegments(
dataSource + config.getExtraDatasourceNameSuffix()
).size();
);
int segmentCount = segments.size();
LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments);
return segmentCount == numExpectedSegments;
},
"Segment count check"
);
}
void verifySegmentsCountAndLoaded(String dataSource, int numExpectedSegments, int numExpectedTombstones)
{
ITRetryUtil.retryUntilTrue(
() -> coordinator.areSegmentsLoaded(dataSource + config.getExtraDatasourceNameSuffix()),
"Segment load check"
);
ITRetryUtil.retryUntilTrue(
() -> {
List<DataSegment> segments = coordinator.getAvailableSegments(
dataSource + config.getExtraDatasourceNameSuffix()
);
int segmentCount = segments.size();
LOG.info("Current segment count: %d, expected: %d", segmentCount, numExpectedSegments);
int tombstoneCount = 0;
for (DataSegment segment : segments) {
if (segment.isTombstone()) {
tombstoneCount++;
}
}
LOG.info("Current tombstone count: %d, expected: %d", tombstoneCount, numExpectedTombstones);
return segmentCount == numExpectedSegments && tombstoneCount == numExpectedTombstones;
},
"Segment count check"
);
}
void compactData(String dataSource, String compactionTask) throws Exception
{
final String fullDatasourceName = dataSource + config.getExtraDatasourceNameSuffix();

View File

@ -49,20 +49,26 @@ public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest
{
final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
try (
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix())
) {
// Submit initial ingestion task
// The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity
// dropExisting true or false does not matter as there is no existing segments
// Even though there are only 10 rows in the three input files and that maxRowsPerSegment is the default
// there will be four segments generated due to the distribution of rows in the input files and the
// fact that there are three sub-tasks (two of the tasks generate one segment each and the other
// task generates two segments)
submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, true);
verifySegmentsCountAndLoaded(indexDatasource, 4);
verifySegmentsCountAndLoaded(indexDatasource, 4, 0);
doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE);
// Submit overwrite ingestion task with drop existing
// The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"),
// however, the input data only contains one day of data, 2013-09-02 (instead of 2 days of data)
// with dropExisting flag set to true, after the second ingestion, we should expect to only have data of 2013-09-02
// however, the input data only contains one file with one day of data, 2013-09-02
// with dropExisting flag set to true, after the second ingestion, we should expect to
// have two segments one for the sub-task that ingested the single data file containing one day
// and one segment for the tombstone to cover the other segment previously generated by the first ingestion
submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, true);
verifySegmentsCountAndLoaded(indexDatasource, 1);
verifySegmentsCountAndLoaded(indexDatasource, 2, 1);
doTestQuery(indexDatasource, INDEX_QUERIES_WITH_DROP_INGESTION_RESOURCE);
}
}
@ -72,13 +78,13 @@ public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest
{
final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
try (
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix())
) {
// Submit initial ingestion task
// The data interval is 2013-08-31 to 2013-09-02 with DAY segmentGranularity
// dropExisting true or false does not matter as there is no existing segments
submitIngestionTaskAndVerify(indexDatasource, ALL_DATA, false);
verifySegmentsCountAndLoaded(indexDatasource, 4);
verifySegmentsCountAndLoaded(indexDatasource, 4, 0);
doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE);
// Submit overwrite ingestion task without drop existing
// The ingestion task interval is the same as the first ingestion ("2013-08-31/2013-09-02"),
@ -87,7 +93,7 @@ public class ITOverwriteBatchIndexTest extends AbstractITBatchIndexTest
// data from 2013-08-31/2013-09-01 remains unchanged and data for 2013-09-01/2013-09-02 from
// the second overwrite ingestion task
submitIngestionTaskAndVerify(indexDatasource, ONE_DAY_DATA, false);
verifySegmentsCountAndLoaded(indexDatasource, 3);
verifySegmentsCountAndLoaded(indexDatasource, 3, 0);
doTestQuery(indexDatasource, INDEX_QUERIES_ALL_INGESTION_RESOURCE);
}
}

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import org.apache.druid.segment.loading.BroadcastJoinableMMappedQueryableSegmentizerFactory;
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
import org.apache.druid.segment.loading.TombstoneSegmentizerFactory;
public class SegmentizerModule extends SimpleModule
{
@ -30,6 +31,9 @@ public class SegmentizerModule extends SimpleModule
{
super("SegmentizerModule");
registerSubtypes(new NamedType(MMappedQueryableSegmentizerFactory.class, "mMapSegmentFactory"));
registerSubtypes(
new NamedType(TombstoneSegmentizerFactory.class, "tombstoneSegmentFactory")
);
registerSubtypes(
new NamedType(BroadcastJoinableMMappedQueryableSegmentizerFactory.class, "broadcastJoinableMMapSegmentFactory")
);

View File

@ -64,6 +64,10 @@ public class ScanQueryEngine
final ResponseContext responseContext
)
{
if (segment.asQueryableIndex() != null && segment.asQueryableIndex().isFromTombstone()) {
return Sequences.empty();
}
// "legacy" should be non-null due to toolChest.mergeResults
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");

View File

@ -51,4 +51,12 @@ public interface QueryableIndex extends ColumnSelector, Closeable
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
@Override
void close();
/**
* @return true if this index was created from a tombstone or false otherwise
*/
default boolean isFromTombstone()
{
return false;
}
}

View File

@ -0,0 +1,169 @@
/*
* 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.segment.loading;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.segment.DimensionHandler;
import org.apache.druid.segment.Metadata;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.Segment;
import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.util.List;
import java.util.Map;
public class TombstoneSegmentizerFactory implements SegmentizerFactory
{
@Override
public Segment factorize(
DataSegment segment,
File parentDir,
boolean lazy,
SegmentLazyLoadFailCallback loadFailed
)
{
return segmentForTombstone(segment);
}
@VisibleForTesting
public static Segment segmentForTombstone(DataSegment tombstone)
{
Preconditions.checkArgument(tombstone.isTombstone());
// Create a no-op queryable index that indicates that it was created from a tombstone...then the
// server manager will use the information to short-circuit and create a no-op query runner for
// it since it has no data:
final QueryableIndex queryableIndex =
new QueryableIndex()
{
@Override
public Interval getDataInterval()
{
return tombstone.getInterval();
}
@Override
public int getNumRows()
{
throw new UnsupportedOperationException();
}
@Override
public Indexed<String> getAvailableDimensions()
{
throw new UnsupportedOperationException();
}
@Override
public BitmapFactory getBitmapFactoryForDimensions()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public Metadata getMetadata()
{
throw new UnsupportedOperationException();
}
@Override
public Map<String, DimensionHandler> getDimensionHandlers()
{
throw new UnsupportedOperationException();
}
@Override
public void close()
{
}
@Override
public List<String> getColumnNames()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public ColumnHolder getColumnHolder(String columnName)
{
throw new UnsupportedOperationException();
}
// mark this index to indicate that it comes from a tombstone:
@Override
public boolean isFromTombstone()
{
return true;
}
};
final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex);
Segment segmentObject = new Segment()
{
@Override
public SegmentId getId()
{
return tombstone.getId();
}
@Override
public Interval getDataInterval()
{
return asQueryableIndex().getDataInterval();
}
@Nullable
@Override
public QueryableIndex asQueryableIndex()
{
return queryableIndex;
}
@Override
public StorageAdapter asStorageAdapter()
{
return storageAdapter;
}
@Override
public void close()
{
}
};
return segmentObject;
}
}

View File

@ -0,0 +1,101 @@
/*
* 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.segment;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Map;
public class SimpleQueryableIndexTest
{
@Test
public void testTombstoneDefaultInterface()
{
QueryableIndex qi = new QueryableIndex()
{
@Override
public Interval getDataInterval()
{
return null;
}
@Override
public int getNumRows()
{
return 0;
}
@Override
public Indexed<String> getAvailableDimensions()
{
return null;
}
@Override
public BitmapFactory getBitmapFactoryForDimensions()
{
return null;
}
@Nullable
@Override
public Metadata getMetadata()
{
return null;
}
@Override
public Map<String, DimensionHandler> getDimensionHandlers()
{
return null;
}
@Override
public void close()
{
}
@Override
public List<String> getColumnNames()
{
return null;
}
@Nullable
@Override
public ColumnHolder getColumnHolder(String columnName)
{
return null;
}
};
Assert.assertFalse(qi.isFromTombstone());
}
}

View File

@ -0,0 +1,68 @@
/*
* 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.segment.loading;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.QueryableIndex;
import org.apache.druid.segment.Segment;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
import static org.junit.Assert.assertThrows;
public class TombstoneSegmentizerFactoryTest
{
@Test
public void testSegmentCreation()
{
Interval expectedInterval = Intervals.of("2021/2022");
TombstoneSegmentizerFactory factory = new TombstoneSegmentizerFactory();
DataSegment tombstone = DataSegment.builder()
.dataSource("foo")
.interval(expectedInterval)
.version("1")
.shardSpec(TombstoneShardSpec.INSTANCE)
.loadSpec(ImmutableMap.of("type", DataSegment.TOMBSTONE_LOADSPEC_TYPE))
.size(1)
.build();
Segment segment = factory.factorize(tombstone, null, true, null);
Assert.assertNotNull(segment.asStorageAdapter());
Assert.assertEquals("foo_2021-01-01T00:00:00.000Z_2022-01-01T00:00:00.000Z_1", segment.getId().toString());
Assert.assertEquals(expectedInterval, segment.getDataInterval());
QueryableIndex queryableIndex = segment.asQueryableIndex();
Assert.assertNotNull(queryableIndex);
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getNumRows());
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getAvailableDimensions());
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getBitmapFactoryForDimensions());
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getMetadata());
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getDimensionHandlers());
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnNames());
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder(null));
Assert.assertTrue(queryableIndex.isFromTombstone());
}
}

View File

@ -274,7 +274,8 @@ public class BrokerServerView implements TimelineServerView
VersionedIntervalTimeline<String, ServerSelector> timeline = timelines.get(segment.getDataSource());
if (timeline == null) {
timeline = new VersionedIntervalTimeline<>(Ordering.natural());
// broker needs to skip tombstones
timeline = new VersionedIntervalTimeline<>(Ordering.natural(), true);
timelines.put(segment.getDataSource(), timeline);
}

View File

@ -425,9 +425,12 @@ public class CachingClusteredClient implements QuerySegmentWalker
{
final java.util.function.Function<Interval, List<TimelineObjectHolder<String, ServerSelector>>> lookupFn
= specificSegments ? timeline::lookupWithIncompletePartitions : timeline::lookup;
List<TimelineObjectHolder<String, ServerSelector>> timelineObjectHolders =
intervals.stream().flatMap(i -> lookupFn.apply(i).stream()).collect(Collectors.toList());
final List<TimelineObjectHolder<String, ServerSelector>> serversLookup = toolChest.filterSegments(
query,
intervals.stream().flatMap(i -> lookupFn.apply(i).stream()).collect(Collectors.toList())
timelineObjectHolders
);
final Set<SegmentServerSelector> segments = new LinkedHashSet<>();
@ -859,7 +862,7 @@ public class CachingClusteredClient implements QuerySegmentWalker
public TimelineLookup<String, ServerSelector> apply(TimelineLookup<String, ServerSelector> timeline)
{
final VersionedIntervalTimeline<String, ServerSelector> timeline2 =
new VersionedIntervalTimeline<>(Ordering.natural());
new VersionedIntervalTimeline<>(Ordering.natural(), true);
Iterator<PartitionChunkEntry<String, ServerSelector>> unfilteredIterator =
Iterators.transform(specs.iterator(), spec -> toChunkEntry(timeline, spec));
Iterator<PartitionChunkEntry<String, ServerSelector>> iterator = Iterators.filter(

View File

@ -209,4 +209,11 @@ public class ServerSelector implements Overshadowable<ServerSelector>
{
return segment.get().getAtomicUpdateGroupSize();
}
@Override
public boolean hasData()
{
return segment.get().hasData();
}
}

View File

@ -420,6 +420,7 @@ public class Initialization
new IndexingServiceDiscoveryModule(),
new CoordinatorDiscoveryModule(),
new LocalDataStorageDruidModule(),
new TombstoneDataStorageModule(),
new FirehoseModule(),
new JavaScriptModule(),
new AuthenticatorModule(),

View File

@ -24,6 +24,7 @@ import com.google.inject.Inject;
import org.apache.druid.java.util.common.MapUtils;
import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.util.Map;
/**
@ -43,11 +44,18 @@ public class OmniDataSegmentKiller implements DataSegmentKiller
@Override
public void kill(DataSegment segment) throws SegmentLoadingException
{
getKiller(segment).kill(segment);
DataSegmentKiller dataSegmentKiller = getKiller(segment);
if (dataSegmentKiller != null) {
dataSegmentKiller.kill(segment);
}
}
@Nullable
private DataSegmentKiller getKiller(DataSegment segment) throws SegmentLoadingException
{
if (segment.isTombstone()) {
return null;
}
String type = MapUtils.getString(segment.getLoadSpec(), "type");
DataSegmentKiller loader = killers.get(type);

View File

@ -21,6 +21,7 @@ package org.apache.druid.segment.loading;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.Segment;
@ -28,12 +29,13 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.timeline.DataSegment;
import javax.inject.Inject;
import java.io.File;
import java.io.IOException;
public class SegmentLocalCacheLoader implements SegmentLoader
{
private static final EmittingLogger log = new EmittingLogger(SegmentLocalCacheLoader.class);
private final SegmentCacheManager cacheManager;
private final IndexIO indexIO;
private final ObjectMapper jsonMapper;
@ -47,7 +49,8 @@ public class SegmentLocalCacheLoader implements SegmentLoader
}
@Override
public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed) throws SegmentLoadingException
public ReferenceCountingSegment getSegment(DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback loadFailed)
throws SegmentLoadingException
{
final File segmentFiles = cacheManager.getSegmentFiles(segment);
File factoryJson = new File(segmentFiles, "factory.json");
@ -65,6 +68,7 @@ public class SegmentLocalCacheLoader implements SegmentLoader
}
Segment segmentObject = factory.factorize(segment, segmentFiles, lazy, loadFailed);
return ReferenceCountingSegment.wrapSegment(segmentObject, segment.getShardSpec());
}
@ -73,4 +77,6 @@ public class SegmentLocalCacheLoader implements SegmentLoader
{
cacheManager.cleanup(segment);
}
}

View File

@ -53,6 +53,7 @@ import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
@ -560,12 +561,18 @@ public abstract class BaseAppenderatorDriver implements Closeable
ListenableFuture<SegmentsAndCommitMetadata> publishInBackground(
@Nullable Set<DataSegment> segmentsToBeOverwritten,
@Nullable Set<DataSegment> segmentsToBeDropped,
@Nullable Set<DataSegment> tombstones,
SegmentsAndCommitMetadata segmentsAndCommitMetadata,
TransactionalSegmentPublisher publisher,
java.util.function.Function<Set<DataSegment>, Set<DataSegment>> outputSegmentsAnnotateFunction
)
{
if (segmentsAndCommitMetadata.getSegments().isEmpty()) {
final Set<DataSegment> pushedAndTombstones = new HashSet<>(segmentsAndCommitMetadata.getSegments());
if (tombstones != null) {
pushedAndTombstones.addAll(tombstones);
}
if (pushedAndTombstones.isEmpty()) {
// no tombstones and no pushed segments, so nothing to publish...
if (!publisher.supportsEmptyPublish()) {
log.info("Nothing to publish, skipping publish step.");
final SettableFuture<SegmentsAndCommitMetadata> retVal = SettableFuture.create();
@ -588,11 +595,10 @@ public abstract class BaseAppenderatorDriver implements Closeable
final Object callerMetadata = metadata == null
? null
: ((AppenderatorDriverMetadata) metadata).getCallerMetadata();
return executor.submit(
() -> {
try {
final ImmutableSet<DataSegment> ourSegments = ImmutableSet.copyOf(segmentsAndCommitMetadata.getSegments());
final ImmutableSet<DataSegment> ourSegments = ImmutableSet.copyOf(pushedAndTombstones);
final SegmentPublishResult publishResult = publisher.publishSegments(
segmentsToBeOverwritten,
segmentsToBeDropped,

View File

@ -34,6 +34,7 @@ import org.apache.druid.timeline.DataSegment;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.Collection;
import java.util.Collections;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
@ -199,6 +200,7 @@ public class BatchAppenderatorDriver extends BaseAppenderatorDriver
public ListenableFuture<SegmentsAndCommitMetadata> publishAll(
@Nullable final Set<DataSegment> segmentsToBeOverwritten,
@Nullable final Set<DataSegment> segmentsToBeDropped,
@Nullable final Set<DataSegment> tombstones,
final TransactionalSegmentPublisher publisher,
final Function<Set<DataSegment>, Set<DataSegment>> outputSegmentsAnnotateFunction
)
@ -211,6 +213,7 @@ public class BatchAppenderatorDriver extends BaseAppenderatorDriver
return publishInBackground(
segmentsToBeOverwritten,
segmentsToBeDropped,
tombstones == null ? Collections.emptySet() : tombstones,
new SegmentsAndCommitMetadata(
snapshot
.values()

View File

@ -280,6 +280,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
// version of a segment with the same identifier containing different data; see DataSegmentPusher.push() docs
pushInBackground(wrapCommitter(committer), theSegments, true),
(AsyncFunction<SegmentsAndCommitMetadata, SegmentsAndCommitMetadata>) sam -> publishInBackground(
null,
null,
null,
sam,

View File

@ -165,7 +165,6 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer
SegmentZNode availableZNode = iter.next();
if (availableZNode.getBytes().length + newBytesLen < config.getMaxBytesPerNode()) {
availableZNode.addSegment(toAnnounce);
log.info(
"Announcing segment[%s] at existing path[%s]",
toAnnounce.getId(),
@ -194,7 +193,11 @@ public class BatchDataSegmentAnnouncer implements DataSegmentAnnouncer
SegmentZNode availableZNode = new SegmentZNode(makeServedSegmentPath());
availableZNode.addSegment(toAnnounce);
log.info("Announcing segment[%s] at new path[%s]", toAnnounce.getId(), availableZNode.getPath());
log.info("Announcing %s[%s] at new path[%s]",
toAnnounce.isTombstone() ? DataSegment.TOMBSTONE_LOADSPEC_TYPE : "segment",
toAnnounce.getId(),
availableZNode.getPath()
);
announcer.announce(availableZNode.getPath(), availableZNode.getBytes());
segmentLookup.put(toAnnounce, availableZNode);
availableZNodes.add(availableZNode);

View File

@ -279,6 +279,13 @@ public class ServerManager implements QuerySegmentWalker
final AtomicLong cpuTimeAccumulator
)
{
// Short-circuit when the index comes from a tombstone (it has no data by definition),
// check for null also since no all segments (higher level ones) will have QueryableIndex...
if (segment.asQueryableIndex() != null && segment.asQueryableIndex().isFromTombstone()) {
return new NoopQueryRunner<>();
}
final SpecificSegmentSpec segmentSpec = new SpecificSegmentSpec(segmentDescriptor);
final SegmentId segmentId = segment.getId();
final Interval segmentInterval = segment.getDataInterval();

View File

@ -130,7 +130,8 @@ public class SimpleServerView implements TimelineServerView
k -> new ServerSelector(segment, tierSelectorStrategy)
);
selector.addServerAndUpdateSegment(servers.get(server), segment);
timelines.computeIfAbsent(segment.getDataSource(), k -> new VersionedIntervalTimeline<>(Ordering.natural()))
// broker needs to skip tombstones in its timelines
timelines.computeIfAbsent(segment.getDataSource(), k -> new VersionedIntervalTimeline<>(Ordering.natural(), true))
.add(segment.getInterval(), segment.getVersion(), segment.getShardSpec().createChunk(selector));
}

View File

@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.server.coordination.ServerType;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
@ -100,4 +101,66 @@ public class ServerSelectorTest
Assert.assertEquals(ImmutableList.of("a", "b", "c"), selector.getSegment().getDimensions());
}
@Test(expected = NullPointerException.class)
public void testSegmentCannotBeNull()
{
final ServerSelector selector = new ServerSelector(
null,
new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())
);
}
@Test
public void testSegmentWithNoData()
{
final ServerSelector selector = new ServerSelector(
DataSegment.builder()
.dataSource("test_broker_server_view")
.interval(Intervals.of("2012/2013"))
.loadSpec(
ImmutableMap.of(
"type",
"tombstone"
)
)
.version("v1")
.dimensions(ImmutableList.of())
.metrics(ImmutableList.of())
.shardSpec(new TombstoneShardSpec())
.binaryVersion(9)
.size(0)
.build(),
new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())
);
Assert.assertFalse(selector.hasData());
}
@Test
public void testSegmentWithData()
{
final ServerSelector selector = new ServerSelector(
DataSegment.builder()
.dataSource("another segment") // fool the interner inside the selector
.interval(Intervals.of("2012/2013"))
.loadSpec(
ImmutableMap.of(
"type",
"local",
"path",
"somewhere"
)
)
.version("v1")
.dimensions(ImmutableList.of())
.metrics(ImmutableList.of())
.shardSpec(NoneShardSpec.instance())
.binaryVersion(9)
.size(0)
.build(),
new HighestPriorityTierSelectorStrategy(new RandomServerSelectorStrategy())
);
Assert.assertTrue(selector.hasData());
}
}

View File

@ -25,7 +25,9 @@ import com.google.inject.Injector;
import com.google.inject.multibindings.MapBinder;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.GuiceInjectors;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.junit.Assert;
import org.junit.Test;
import org.mockito.Mockito;
@ -39,6 +41,7 @@ public class OmniDataSegmentKillerTest
{
final DataSegmentKiller killer = Mockito.mock(DataSegmentKiller.class);
final DataSegment segment = Mockito.mock(DataSegment.class);
Mockito.when(segment.isTombstone()).thenReturn(false);
Mockito.when(segment.getLoadSpec()).thenReturn(ImmutableMap.of("type", "sane"));
final Injector injector = createInjector(killer);
@ -75,4 +78,26 @@ public class OmniDataSegmentKillerTest
)
);
}
@Test
public void testKillTombstone() throws Exception
{
// tombstone
DataSegment tombstone =
DataSegment.builder()
.dataSource("test")
.interval(Intervals.of("2021-01-01/P1D"))
.version("version")
.size(1)
.loadSpec(ImmutableMap.of("type", "tombstone", "path", "null"))
.shardSpec(new TombstoneShardSpec())
.build();
final Injector injector = createInjector(null);
final OmniDataSegmentKiller segmentKiller = injector.getInstance(OmniDataSegmentKiller.class);
segmentKiller.kill(tombstone);
}
}

View File

@ -0,0 +1,110 @@
/*
* 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.segment.loading;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.segment.ReferenceCountingSegment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.server.coordination.TestStorageLocation;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import java.io.IOException;
import java.util.Collections;
public class SegmentLocalCacheLoaderTest
{
private static final long MAX_SIZE = 1000L;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private TestStorageLocation storageLoc;
private ObjectMapper objectMapper;
private SegmentLocalCacheLoader segmentLocalCacheLoader;
@Before
public void setUp() throws IOException
{
storageLoc = new TestStorageLocation(temporaryFolder);
SegmentLoaderConfig config = new SegmentLoaderConfig()
.withLocations(Collections.singletonList(storageLoc.toStorageLocationConfig(MAX_SIZE, null)))
.withInfoDir(storageLoc.getInfoDir());
objectMapper = TestHelper.makeJsonMapper();
objectMapper.registerSubtypes(TombstoneLoadSpec.class);
objectMapper.registerSubtypes(TombstoneSegmentizerFactory.class);
SegmentCacheManager cacheManager = new SegmentLocalCacheManager(config, objectMapper);
segmentLocalCacheLoader = new SegmentLocalCacheLoader(cacheManager, null, objectMapper);
TombstoneLoadSpec.writeFactoryFile(storageLoc.getCacheDir());
}
@Test
public void testGetSegmentWithTombstones() throws SegmentLoadingException
{
Interval interval = Intervals.of("2014-01-01/2014-01-02");
DataSegment tombstone = new DataSegment("foo", interval, "version",
ImmutableMap.of("type", "tombstone"),
null, null, new TombstoneShardSpec(),
null, 0
);
ReferenceCountingSegment segment = segmentLocalCacheLoader.getSegment(tombstone, false, null);
Assert.assertNotNull(segment.getId());
Assert.assertEquals(interval, segment.getDataInterval());
Assert.assertNotNull(segment.asStorageAdapter());
Assert.assertTrue(segment.asQueryableIndex().isFromTombstone());
Assert.assertEquals(interval, segment.asQueryableIndex().getDataInterval());
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getMetadata());
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getNumRows());
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getAvailableDimensions());
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getBitmapFactoryForDimensions()
);
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getDimensionHandlers()
);
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getColumnHolder(null)
);
Assert.assertThrows(
UnsupportedOperationException.class,
() -> segment.asQueryableIndex().getColumnHolder(null)
);
}
}

View File

@ -31,6 +31,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.apache.druid.timeline.partition.TombstoneShardSpec;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -41,6 +42,7 @@ import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.List;
public class SegmentLocalCacheManagerTest
@ -56,7 +58,8 @@ public class SegmentLocalCacheManagerTest
public SegmentLocalCacheManagerTest()
{
jsonMapper = new DefaultObjectMapper();
jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"));
jsonMapper.registerSubtypes(new NamedType(LocalLoadSpec.class, "local"),
new NamedType(TombstoneLoadSpec.class, "tombstone"));
jsonMapper.setInjectableValues(
new InjectableValues.Std().addValue(
LocalDataSegmentPuller.class,
@ -97,6 +100,33 @@ public class SegmentLocalCacheManagerTest
Assert.assertFalse("Expect cache miss", manager.isSegmentCached(uncachedSegment));
}
@Test
public void testIfTombstoneIsLoaded() throws IOException, SegmentLoadingException
{
final DataSegment tombstone = DataSegment.builder()
.dataSource("foo")
.interval(Intervals.of("2014-10-20T00:00:00Z/P1D"))
.version("version")
.loadSpec(Collections.singletonMap(
"type",
DataSegment.TOMBSTONE_LOADSPEC_TYPE
))
.shardSpec(TombstoneShardSpec.INSTANCE)
.size(1)
.build();
final File cachedSegmentFile = new File(
localSegmentCacheFolder,
"test_segment_loader/2014-10-20T00:00:00.000Z_2014-10-21T00:00:00.000Z/2015-05-27T03:38:35.683Z/0"
);
FileUtils.mkdirp(cachedSegmentFile);
manager.getSegmentFiles(tombstone);
Assert.assertTrue("Expect cache hit after downloading segment", manager.isSegmentCached(tombstone));
}
@Test
public void testGetAndCleanSegmentFiles() throws Exception
{

View File

@ -43,6 +43,7 @@ import org.junit.Before;
import org.junit.Test;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -126,7 +127,8 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp
checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED);
final SegmentsAndCommitMetadata published =
driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
driver.publishAll(null, null, Collections.emptySet(), makeOkPublisher(), Function.identity())
.get(TIMEOUT, TimeUnit.MILLISECONDS);
Assert.assertEquals(
ImmutableSet.of(
@ -160,7 +162,8 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp
}
final SegmentsAndCommitMetadata published =
driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
driver.publishAll(null, null, Collections.emptySet(), makeOkPublisher(), Function.identity())
.get(TIMEOUT, TimeUnit.MILLISECONDS);
Assert.assertEquals(
ImmutableSet.of(

View File

@ -125,7 +125,7 @@ public class OpenAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSu
checkSegmentStates(2, SegmentState.PUSHED_AND_DROPPED);
final SegmentsAndCommitMetadata published =
driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
driver.publishAll(null, null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
Assert.assertEquals(
ImmutableSet.of(
@ -159,7 +159,7 @@ public class OpenAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSu
}
final SegmentsAndCommitMetadata published =
driver.publishAll(null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
driver.publishAll(null, null, null, makeOkPublisher(), Function.identity()).get(TIMEOUT, TimeUnit.MILLISECONDS);
Assert.assertEquals(
ImmutableSet.of(

View File

@ -28,6 +28,7 @@ import org.apache.druid.client.cache.CacheConfig;
import org.apache.druid.client.cache.CachePopulatorStats;
import org.apache.druid.client.cache.ForegroundCachePopulator;
import org.apache.druid.client.cache.LocalCacheProvider;
import org.apache.druid.collections.bitmap.BitmapFactory;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
@ -71,6 +72,7 @@ import org.apache.druid.query.search.SearchResultValue;
import org.apache.druid.query.spec.MultipleSpecificSegmentSpec;
import org.apache.druid.query.spec.QuerySegmentSpec;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.DimensionHandler;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.Metadata;
import org.apache.druid.segment.QueryableIndex;
@ -80,10 +82,12 @@ import org.apache.druid.segment.SegmentLazyLoadFailCallback;
import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.join.NoopJoinableFactory;
import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.TombstoneSegmentizerFactory;
import org.apache.druid.server.SegmentManager;
import org.apache.druid.server.initialization.ServerConfig;
import org.apache.druid.server.metrics.NoopServiceEmitter;
@ -148,10 +152,15 @@ public class ServerManagerTest
@Override
public ReferenceCountingSegment getSegment(final DataSegment segment, boolean lazy, SegmentLazyLoadFailCallback SegmentLazyLoadFailCallback)
{
return ReferenceCountingSegment.wrapSegment(new SegmentForTesting(
MapUtils.getString(segment.getLoadSpec(), "version"),
(Interval) segment.getLoadSpec().get("interval")
), segment.getShardSpec());
if (segment.isTombstone()) {
return ReferenceCountingSegment
.wrapSegment(TombstoneSegmentizerFactory.segmentForTombstone(segment), segment.getShardSpec());
} else {
return ReferenceCountingSegment.wrapSegment(new SegmentForTesting(
MapUtils.getString(segment.getLoadSpec(), "version"),
(Interval) segment.getLoadSpec().get("interval")
), segment.getShardSpec());
}
}
@Override
@ -198,6 +207,7 @@ public class ServerManagerTest
loadQueryable("test", "2", Intervals.of("PT1h/2011-04-04T06"));
loadQueryable("test2", "1", Intervals.of("P1d/2011-04-01"));
loadQueryable("test2", "1", Intervals.of("P1d/2011-04-02"));
loadQueryable("testTombstone", "1", Intervals.of("P1d/2011-04-02"));
}
@Test
@ -225,6 +235,19 @@ public class ServerManagerTest
waitForTestVerificationAndCleanup(future);
}
@Test
public void testSimpleGetTombstone()
{
Future future = assertQueryable(
Granularities.DAY,
"testTombstone",
Intervals.of("P1d/2011-04-01"),
Collections.emptyList() // tombstone returns no data
);
waitForTestVerificationAndCleanup(future);
}
@Test
public void testDelete1()
{
@ -651,21 +674,43 @@ public class ServerManagerTest
public void loadQueryable(String dataSource, String version, Interval interval)
{
try {
segmentManager.loadSegment(
new DataSegment(
dataSource,
interval,
version,
ImmutableMap.of("version", version, "interval", interval),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
123L
),
false,
SegmentLazyLoadFailCallback.NOOP
);
if ("testTombstone".equals(dataSource)) {
segmentManager.loadSegment(
new DataSegment(
dataSource,
interval,
version,
ImmutableMap.of("version", version,
"interval", interval,
"type",
DataSegment.TOMBSTONE_LOADSPEC_TYPE
),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
123L
),
false,
SegmentLazyLoadFailCallback.NOOP
);
} else {
segmentManager.loadSegment(
new DataSegment(
dataSource,
interval,
version,
ImmutableMap.of("version", version, "interval", interval),
Arrays.asList("dim1", "dim2", "dim3"),
Arrays.asList("metric1", "metric2"),
NoneShardSpec.instance(),
IndexIO.CURRENT_VERSION_ID,
123L
),
false,
SegmentLazyLoadFailCallback.NOOP
);
}
}
catch (SegmentLoadingException e) {
throw new RuntimeException(e);
@ -789,6 +834,64 @@ public class ServerManagerTest
private final Interval interval;
private final Object lock = new Object();
private volatile boolean closed = false;
private QueryableIndex index = new QueryableIndex()
{
@Override
public Interval getDataInterval()
{
throw new UnsupportedOperationException();
}
@Override
public int getNumRows()
{
throw new UnsupportedOperationException();
}
@Override
public Indexed<String> getAvailableDimensions()
{
throw new UnsupportedOperationException();
}
@Override
public BitmapFactory getBitmapFactoryForDimensions()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public Metadata getMetadata()
{
throw new UnsupportedOperationException();
}
@Override
public Map<String, DimensionHandler> getDimensionHandlers()
{
throw new UnsupportedOperationException();
}
@Override
public void close()
{
}
@Override
public List<String> getColumnNames()
{
throw new UnsupportedOperationException();
}
@Nullable
@Override
public ColumnHolder getColumnHolder(String columnName)
{
throw new UnsupportedOperationException();
}
};
SegmentForTesting(
String version,
@ -829,7 +932,7 @@ public class ServerManagerTest
@Override
public QueryableIndex asQueryableIndex()
{
throw new UnsupportedOperationException();
return index;
}
@Override

View File

@ -52,6 +52,7 @@ import org.junit.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.Iterator;
import java.util.List;
@ -231,6 +232,43 @@ public class BatchDataSegmentAnnouncerTest
Assert.assertEquals(4, snapshot.getCounter().getCounter());
}
@Test
public void testSingleTombstoneAnnounce() throws Exception
{
DataSegment firstSegment = makeSegment(0, true);
segmentAnnouncer.announceSegment(firstSegment);
List<String> zNodes = cf.getChildren().forPath(TEST_SEGMENTS_PATH);
for (String zNode : zNodes) {
Set<DataSegment> segments = segmentReader.read(JOINER.join(TEST_SEGMENTS_PATH, zNode));
Assert.assertEquals(segments.iterator().next(), firstSegment);
}
ChangeRequestsSnapshot<DataSegmentChangeRequest> snapshot = segmentAnnouncer.getSegmentChangesSince(
new ChangeRequestHistory.Counter(-1, -1)
).get();
Assert.assertEquals(1, snapshot.getRequests().size());
Assert.assertEquals(1, snapshot.getCounter().getCounter());
segmentAnnouncer.unannounceSegment(firstSegment);
Assert.assertTrue(cf.getChildren().forPath(TEST_SEGMENTS_PATH).isEmpty());
snapshot = segmentAnnouncer.getSegmentChangesSince(
snapshot.getCounter()
).get();
Assert.assertEquals(1, snapshot.getRequests().size());
Assert.assertEquals(2, snapshot.getCounter().getCounter());
snapshot = segmentAnnouncer.getSegmentChangesSince(
new ChangeRequestHistory.Counter(-1, -1)
).get();
Assert.assertEquals(0, snapshot.getRequests().size());
Assert.assertEquals(2, snapshot.getCounter().getCounter());
}
@Test
public void testSkipDimensions() throws Exception
{
@ -430,22 +468,31 @@ public class BatchDataSegmentAnnouncerTest
}
}
private DataSegment makeSegment(int offset, boolean isTombstone)
{
DataSegment.Builder builder = DataSegment.builder();
builder.dataSource("foo")
.interval(
new Interval(
DateTimes.of("2013-01-01").plusDays(offset),
DateTimes.of("2013-01-02").plusDays(offset)
)
)
.version(DateTimes.nowUtc().toString())
.dimensions(ImmutableList.of("dim1", "dim2"))
.metrics(ImmutableList.of("met1", "met2"))
.loadSpec(ImmutableMap.of("type", "local"))
.size(0);
if (isTombstone) {
builder.loadSpec(Collections.singletonMap("type", DataSegment.TOMBSTONE_LOADSPEC_TYPE));
}
return builder.build();
}
private DataSegment makeSegment(int offset)
{
return DataSegment.builder()
.dataSource("foo")
.interval(
new Interval(
DateTimes.of("2013-01-01").plusDays(offset),
DateTimes.of("2013-01-02").plusDays(offset)
)
)
.version(DateTimes.nowUtc().toString())
.dimensions(ImmutableList.of("dim1", "dim2"))
.metrics(ImmutableList.of("met1", "met2"))
.loadSpec(ImmutableMap.of("type", "local"))
.size(0)
.build();
return makeSegment(offset, false);
}
private static class SegmentReader