mirror of
https://github.com/apache/druid.git
synced 2025-02-17 07:25:02 +00:00
Refactor: Rename UsedSegmentChecker and cleanup task actions (#16644)
Changes: - Rename `UsedSegmentChecker` to `PublishedSegmentsRetriever` - Remove deprecated single `Interval` argument from `RetrieveUsedSegmentsAction` as it is now unused and has been deprecated since #1988 - Return `Set` of segments instead of a `Collection` from `IndexerMetadataStorageCoordinator.retrieveUsedSegments()`
This commit is contained in:
parent
52c9929019
commit
d9bd02256a
@ -27,7 +27,7 @@ import org.apache.druid.indexing.common.actions.TaskActionClient;
|
|||||||
import org.apache.druid.indexing.overlord.Segments;
|
import org.apache.druid.indexing.overlord.Segments;
|
||||||
import org.apache.druid.java.util.common.JodaUtils;
|
import org.apache.druid.java.util.common.JodaUtils;
|
||||||
import org.apache.druid.java.util.common.logger.Logger;
|
import org.apache.druid.java.util.common.logger.Logger;
|
||||||
import org.apache.druid.segment.realtime.appenderator.UsedSegmentChecker;
|
import org.apache.druid.segment.realtime.appenderator.PublishedSegmentRetriever;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.SegmentId;
|
import org.apache.druid.timeline.SegmentId;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
@ -40,13 +40,13 @@ import java.util.List;
|
|||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
|
public class ActionBasedPublishedSegmentRetriever implements PublishedSegmentRetriever
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(ActionBasedUsedSegmentChecker.class);
|
private static final Logger log = new Logger(ActionBasedPublishedSegmentRetriever.class);
|
||||||
|
|
||||||
private final TaskActionClient taskActionClient;
|
private final TaskActionClient taskActionClient;
|
||||||
|
|
||||||
public ActionBasedUsedSegmentChecker(TaskActionClient taskActionClient)
|
public ActionBasedPublishedSegmentRetriever(TaskActionClient taskActionClient)
|
||||||
{
|
{
|
||||||
this.taskActionClient = taskActionClient;
|
this.taskActionClient = taskActionClient;
|
||||||
}
|
}
|
||||||
@ -92,7 +92,7 @@ public class ActionBasedUsedSegmentChecker implements UsedSegmentChecker
|
|||||||
Iterables.transform(segmentIds, SegmentId::getInterval)
|
Iterables.transform(segmentIds, SegmentId::getInterval)
|
||||||
);
|
);
|
||||||
final Collection<DataSegment> foundUsedSegments = taskActionClient.submit(
|
final Collection<DataSegment> foundUsedSegments = taskActionClient.submit(
|
||||||
new RetrieveUsedSegmentsAction(dataSource, null, usedSearchIntervals, Segments.INCLUDING_OVERSHADOWED)
|
new RetrieveUsedSegmentsAction(dataSource, usedSearchIntervals, Segments.INCLUDING_OVERSHADOWED)
|
||||||
);
|
);
|
||||||
for (DataSegment segment : foundUsedSegments) {
|
for (DataSegment segment : foundUsedSegments) {
|
||||||
if (segmentIds.contains(segment.getId())) {
|
if (segmentIds.contains(segment.getId())) {
|
@ -22,8 +22,8 @@ package org.apache.druid.indexing.common.actions;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.google.common.base.Preconditions;
|
import org.apache.druid.common.config.Configs;
|
||||||
import com.google.common.collect.ImmutableList;
|
import org.apache.druid.error.InvalidInput;
|
||||||
import org.apache.druid.indexing.common.task.Task;
|
import org.apache.druid.indexing.common.task.Task;
|
||||||
import org.apache.druid.indexing.common.task.batch.parallel.AbstractBatchSubtask;
|
import org.apache.druid.indexing.common.task.batch.parallel.AbstractBatchSubtask;
|
||||||
import org.apache.druid.indexing.overlord.Segments;
|
import org.apache.druid.indexing.overlord.Segments;
|
||||||
@ -35,6 +35,7 @@ import org.apache.druid.metadata.ReplaceTaskLock;
|
|||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.Partitions;
|
import org.apache.druid.timeline.Partitions;
|
||||||
import org.apache.druid.timeline.SegmentTimeline;
|
import org.apache.druid.timeline.SegmentTimeline;
|
||||||
|
import org.apache.druid.utils.CollectionUtils;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
@ -48,19 +49,13 @@ import java.util.Set;
|
|||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This TaskAction returns a collection of segments which have data within the specified intervals and are marked as
|
* Task action to retrieve a collection of segments which have data within the
|
||||||
* used.
|
* specified intervals and are marked as used.
|
||||||
|
* <p>
|
||||||
* If the task holds REPLACE locks and is writing back to the same datasource,
|
* If the task holds REPLACE locks and is writing back to the same datasource,
|
||||||
* only segments that were created before the REPLACE lock was acquired are returned for an interval.
|
* only segments that were created before the REPLACE lock was acquired are
|
||||||
* This ensures that the input set of segments for this replace task remains consistent
|
* returned for an interval. This ensures that the input set of segments for this
|
||||||
* even when new data is appended by other concurrent tasks.
|
* replace task remains consistent even when new data is appended by other concurrent tasks.
|
||||||
*
|
|
||||||
* The order of segments within the returned collection is unspecified, but each segment is guaranteed to appear in
|
|
||||||
* the collection only once.
|
|
||||||
*
|
|
||||||
* @implNote This action doesn't produce a {@link Set} because it's implemented via {@link
|
|
||||||
* org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator#retrieveUsedSegmentsForIntervals} which returns
|
|
||||||
* a collection. Producing a {@link Set} would require an unnecessary copy of segments collection.
|
|
||||||
*/
|
*/
|
||||||
public class RetrieveUsedSegmentsAction implements TaskAction<Collection<DataSegment>>
|
public class RetrieveUsedSegmentsAction implements TaskAction<Collection<DataSegment>>
|
||||||
{
|
{
|
||||||
@ -73,35 +68,22 @@ public class RetrieveUsedSegmentsAction implements TaskAction<Collection<DataSeg
|
|||||||
@JsonCreator
|
@JsonCreator
|
||||||
public RetrieveUsedSegmentsAction(
|
public RetrieveUsedSegmentsAction(
|
||||||
@JsonProperty("dataSource") String dataSource,
|
@JsonProperty("dataSource") String dataSource,
|
||||||
@Deprecated @JsonProperty("interval") Interval interval,
|
|
||||||
@JsonProperty("intervals") Collection<Interval> intervals,
|
@JsonProperty("intervals") Collection<Interval> intervals,
|
||||||
// When JSON object is deserialized, this parameter is optional for backward compatibility.
|
|
||||||
// Otherwise, it shouldn't be considered optional.
|
|
||||||
@JsonProperty("visibility") @Nullable Segments visibility
|
@JsonProperty("visibility") @Nullable Segments visibility
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dataSource = dataSource;
|
if (CollectionUtils.isNullOrEmpty(intervals)) {
|
||||||
|
throw InvalidInput.exception("No interval specified for retrieving used segments");
|
||||||
Preconditions.checkArgument(
|
|
||||||
interval == null || intervals == null,
|
|
||||||
"please specify intervals only"
|
|
||||||
);
|
|
||||||
|
|
||||||
List<Interval> theIntervals = null;
|
|
||||||
if (interval != null) {
|
|
||||||
theIntervals = ImmutableList.of(interval);
|
|
||||||
} else if (intervals != null && intervals.size() > 0) {
|
|
||||||
theIntervals = JodaUtils.condenseIntervals(intervals);
|
|
||||||
}
|
}
|
||||||
this.intervals = Preconditions.checkNotNull(theIntervals, "no intervals found");
|
|
||||||
|
|
||||||
// Defaulting to the former behaviour when visibility wasn't explicitly specified for backward compatibility
|
this.dataSource = dataSource;
|
||||||
this.visibility = visibility != null ? visibility : Segments.ONLY_VISIBLE;
|
this.intervals = JodaUtils.condenseIntervals(intervals);
|
||||||
|
this.visibility = Configs.valueOrDefault(visibility, Segments.ONLY_VISIBLE);
|
||||||
}
|
}
|
||||||
|
|
||||||
public RetrieveUsedSegmentsAction(String dataSource, Collection<Interval> intervals)
|
public RetrieveUsedSegmentsAction(String dataSource, Collection<Interval> intervals)
|
||||||
{
|
{
|
||||||
this(dataSource, null, intervals, Segments.ONLY_VISIBLE);
|
this(dataSource, intervals, Segments.ONLY_VISIBLE);
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@ -198,7 +180,7 @@ public class RetrieveUsedSegmentsAction implements TaskAction<Collection<DataSeg
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Collection<DataSegment> retrieveUsedSegments(TaskActionToolbox toolbox)
|
private Set<DataSegment> retrieveUsedSegments(TaskActionToolbox toolbox)
|
||||||
{
|
{
|
||||||
return toolbox.getIndexerMetadataStorageCoordinator()
|
return toolbox.getIndexerMetadataStorageCoordinator()
|
||||||
.retrieveUsedSegmentsForIntervals(dataSource, intervals, visibility);
|
.retrieveUsedSegmentsForIntervals(dataSource, intervals, visibility);
|
||||||
|
@ -38,7 +38,7 @@ import java.util.concurrent.Future;
|
|||||||
@JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class),
|
@JsonSubTypes.Type(name = "segmentTransactionalInsert", value = SegmentTransactionalInsertAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class),
|
@JsonSubTypes.Type(name = "segmentTransactionalAppend", value = SegmentTransactionalAppendAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class),
|
@JsonSubTypes.Type(name = "segmentTransactionalReplace", value = SegmentTransactionalReplaceAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentListById", value = RetrieveSegmentsByIdAction.class),
|
@JsonSubTypes.Type(name = "retrieveSegmentsById", value = RetrieveSegmentsByIdAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class),
|
@JsonSubTypes.Type(name = "segmentListUsed", value = RetrieveUsedSegmentsAction.class),
|
||||||
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
|
@JsonSubTypes.Type(name = "segmentListUnused", value = RetrieveUnusedSegmentsAction.class),
|
||||||
@JsonSubTypes.Type(name = "markSegmentsAsUnused", value = MarkSegmentsAsUnusedAction.class),
|
@JsonSubTypes.Type(name = "markSegmentsAsUnused", value = MarkSegmentsAsUnusedAction.class),
|
||||||
|
@ -19,7 +19,7 @@
|
|||||||
|
|
||||||
package org.apache.druid.indexing.common.task;
|
package org.apache.druid.indexing.common.task;
|
||||||
|
|
||||||
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
|
import org.apache.druid.indexing.appenderator.ActionBasedPublishedSegmentRetriever;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
import org.apache.druid.indexing.common.config.TaskConfig;
|
import org.apache.druid.indexing.common.config.TaskConfig;
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
@ -134,7 +134,7 @@ public final class BatchAppenderators
|
|||||||
return new BatchAppenderatorDriver(
|
return new BatchAppenderatorDriver(
|
||||||
appenderator,
|
appenderator,
|
||||||
segmentAllocator,
|
segmentAllocator,
|
||||||
new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
|
new ActionBasedPublishedSegmentRetriever(toolbox.getTaskActionClient()),
|
||||||
toolbox.getDataSegmentKiller()
|
toolbox.getDataSegmentKiller()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -198,7 +198,6 @@ public class KillUnusedSegmentsTask extends AbstractFixedIntervalTask
|
|||||||
|
|
||||||
RetrieveUsedSegmentsAction retrieveUsedSegmentsAction = new RetrieveUsedSegmentsAction(
|
RetrieveUsedSegmentsAction retrieveUsedSegmentsAction = new RetrieveUsedSegmentsAction(
|
||||||
getDataSource(),
|
getDataSource(),
|
||||||
null,
|
|
||||||
ImmutableList.of(getInterval()),
|
ImmutableList.of(getInterval()),
|
||||||
Segments.INCLUDING_OVERSHADOWED
|
Segments.INCLUDING_OVERSHADOWED
|
||||||
);
|
);
|
||||||
|
@ -53,6 +53,6 @@ public class OverlordActionBasedUsedSegmentsRetriever implements UsedSegmentsRet
|
|||||||
{
|
{
|
||||||
return toolbox
|
return toolbox
|
||||||
.getTaskActionClient()
|
.getTaskActionClient()
|
||||||
.submit(new RetrieveUsedSegmentsAction(dataSource, null, intervals, visibility));
|
.submit(new RetrieveUsedSegmentsAction(dataSource, intervals, visibility));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -27,8 +27,8 @@ import com.google.common.base.Suppliers;
|
|||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
import org.apache.druid.data.input.impl.ByteEntity;
|
import org.apache.druid.data.input.impl.ByteEntity;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
|
import org.apache.druid.indexing.appenderator.ActionBasedPublishedSegmentRetriever;
|
||||||
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
|
import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator;
|
||||||
import org.apache.druid.indexing.appenderator.ActionBasedUsedSegmentChecker;
|
|
||||||
import org.apache.druid.indexing.common.LockGranularity;
|
import org.apache.druid.indexing.common.LockGranularity;
|
||||||
import org.apache.druid.indexing.common.TaskLockType;
|
import org.apache.druid.indexing.common.TaskLockType;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
@ -237,7 +237,7 @@ public abstract class SeekableStreamIndexTask<PartitionIdType, SequenceOffsetTyp
|
|||||||
)
|
)
|
||||||
),
|
),
|
||||||
toolbox.getSegmentHandoffNotifierFactory(),
|
toolbox.getSegmentHandoffNotifierFactory(),
|
||||||
new ActionBasedUsedSegmentChecker(toolbox.getTaskActionClient()),
|
new ActionBasedPublishedSegmentRetriever(toolbox.getTaskActionClient()),
|
||||||
toolbox.getDataSegmentKiller(),
|
toolbox.getDataSegmentKiller(),
|
||||||
toolbox.getJsonMapper(),
|
toolbox.getJsonMapper(),
|
||||||
metrics
|
metrics
|
||||||
|
@ -976,7 +976,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||||||
sequenceMetadata.getCommitterSupplier(this, stream, lastPersistedOffsets).get(),
|
sequenceMetadata.getCommitterSupplier(this, stream, lastPersistedOffsets).get(),
|
||||||
Collections.singletonList(sequenceMetadata.getSequenceName())
|
Collections.singletonList(sequenceMetadata.getSequenceName())
|
||||||
),
|
),
|
||||||
(Function<SegmentsAndCommitMetadata, SegmentsAndCommitMetadata>) publishedSegmentsAndMetadata -> {
|
publishedSegmentsAndMetadata -> {
|
||||||
if (publishedSegmentsAndMetadata == null) {
|
if (publishedSegmentsAndMetadata == null) {
|
||||||
throw new ISE(
|
throw new ISE(
|
||||||
"Transaction failure publishing segments for sequence [%s]",
|
"Transaction failure publishing segments for sequence [%s]",
|
||||||
|
@ -393,15 +393,13 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||||||
// if we created no segments and didn't change any offsets, just do nothing and return.
|
// if we created no segments and didn't change any offsets, just do nothing and return.
|
||||||
log.info(
|
log.info(
|
||||||
"With empty segment set, start offsets [%s] and end offsets [%s] are the same, skipping metadata commit.",
|
"With empty segment set, start offsets [%s] and end offsets [%s] are the same, skipping metadata commit.",
|
||||||
startPartitions,
|
startPartitions, finalPartitions
|
||||||
finalPartitions
|
|
||||||
);
|
);
|
||||||
return SegmentPublishResult.ok(segmentsToPush);
|
return SegmentPublishResult.ok(segmentsToPush);
|
||||||
} else {
|
} else {
|
||||||
log.info(
|
log.info(
|
||||||
"With empty segment set, start offsets [%s] and end offsets [%s] changed, committing new metadata.",
|
"With empty segment set, start offsets [%s] and end offsets [%s] changed, committing new metadata.",
|
||||||
startPartitions,
|
startPartitions, finalPartitions
|
||||||
finalPartitions
|
|
||||||
);
|
);
|
||||||
action = SegmentTransactionalInsertAction.commitMetadataOnlyAction(
|
action = SegmentTransactionalInsertAction.commitMetadataOnlyAction(
|
||||||
runner.getAppenderator().getDataSource(),
|
runner.getAppenderator().getDataSource(),
|
||||||
@ -419,12 +417,10 @@ public class SequenceMetadata<PartitionIdType, SequenceOffsetType>
|
|||||||
);
|
);
|
||||||
final DataSourceMetadata endMetadata = runner.createDataSourceMetadata(finalPartitions);
|
final DataSourceMetadata endMetadata = runner.createDataSourceMetadata(finalPartitions);
|
||||||
action = taskLockType == TaskLockType.APPEND
|
action = taskLockType == TaskLockType.APPEND
|
||||||
? SegmentTransactionalAppendAction.forSegmentsAndMetadata(segmentsToPush, startMetadata, endMetadata,
|
? SegmentTransactionalAppendAction
|
||||||
segmentSchemaMapping
|
.forSegmentsAndMetadata(segmentsToPush, startMetadata, endMetadata, segmentSchemaMapping)
|
||||||
)
|
: SegmentTransactionalInsertAction
|
||||||
: SegmentTransactionalInsertAction.appendAction(segmentsToPush, startMetadata, endMetadata,
|
.appendAction(segmentsToPush, startMetadata, endMetadata, segmentSchemaMapping);
|
||||||
segmentSchemaMapping
|
|
||||||
);
|
|
||||||
} else {
|
} else {
|
||||||
action = taskLockType == TaskLockType.APPEND
|
action = taskLockType == TaskLockType.APPEND
|
||||||
? SegmentTransactionalAppendAction.forSegments(segmentsToPush, segmentSchemaMapping)
|
? SegmentTransactionalAppendAction.forSegments(segmentsToPush, segmentSchemaMapping)
|
||||||
|
@ -43,16 +43,16 @@ import java.util.List;
|
|||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
|
||||||
public class ActionBasedUsedSegmentCheckerTest
|
public class ActionBasedPublishedSegmentRetrieverTest
|
||||||
{
|
{
|
||||||
private TaskActionClient taskActionClient;
|
private TaskActionClient taskActionClient;
|
||||||
private ActionBasedUsedSegmentChecker segmentRetriever;
|
private ActionBasedPublishedSegmentRetriever segmentRetriever;
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setup()
|
public void setup()
|
||||||
{
|
{
|
||||||
taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
taskActionClient = EasyMock.createMock(TaskActionClient.class);
|
||||||
segmentRetriever = new ActionBasedUsedSegmentChecker(taskActionClient);
|
segmentRetriever = new ActionBasedPublishedSegmentRetriever(taskActionClient);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
@ -103,7 +103,6 @@ public class ActionBasedUsedSegmentCheckerTest
|
|||||||
taskActionClient.submit(
|
taskActionClient.submit(
|
||||||
new RetrieveUsedSegmentsAction(
|
new RetrieveUsedSegmentsAction(
|
||||||
"wiki",
|
"wiki",
|
||||||
null,
|
|
||||||
Collections.singletonList(Intervals.of("2013-01-01/P3D")),
|
Collections.singletonList(Intervals.of("2013-01-01/P3D")),
|
||||||
Segments.INCLUDING_OVERSHADOWED
|
Segments.INCLUDING_OVERSHADOWED
|
||||||
)
|
)
|
@ -28,6 +28,7 @@ import org.joda.time.Interval;
|
|||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -42,7 +43,7 @@ public class RetrieveUsedSegmentsActionSerdeTest
|
|||||||
Interval interval = Intervals.of("2014/2015");
|
Interval interval = Intervals.of("2014/2015");
|
||||||
|
|
||||||
RetrieveUsedSegmentsAction expected =
|
RetrieveUsedSegmentsAction expected =
|
||||||
new RetrieveUsedSegmentsAction("dataSource", interval, null, Segments.ONLY_VISIBLE);
|
new RetrieveUsedSegmentsAction("dataSource", Collections.singletonList(interval), Segments.ONLY_VISIBLE);
|
||||||
|
|
||||||
RetrieveUsedSegmentsAction actual =
|
RetrieveUsedSegmentsAction actual =
|
||||||
MAPPER.readValue(MAPPER.writeValueAsString(expected), RetrieveUsedSegmentsAction.class);
|
MAPPER.readValue(MAPPER.writeValueAsString(expected), RetrieveUsedSegmentsAction.class);
|
||||||
@ -68,11 +69,15 @@ public class RetrieveUsedSegmentsActionSerdeTest
|
|||||||
@Test
|
@Test
|
||||||
public void testOldJsonDeserialization() throws Exception
|
public void testOldJsonDeserialization() throws Exception
|
||||||
{
|
{
|
||||||
String jsonStr = "{\"type\": \"segmentListUsed\", \"dataSource\": \"test\", \"interval\": \"2014/2015\"}";
|
String jsonStr = "{\"type\": \"segmentListUsed\", \"dataSource\": \"test\", \"intervals\": [\"2014/2015\"]}";
|
||||||
RetrieveUsedSegmentsAction actual = (RetrieveUsedSegmentsAction) MAPPER.readValue(jsonStr, TaskAction.class);
|
RetrieveUsedSegmentsAction actual = (RetrieveUsedSegmentsAction) MAPPER.readValue(jsonStr, TaskAction.class);
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
new RetrieveUsedSegmentsAction("test", Intervals.of("2014/2015"), null, Segments.ONLY_VISIBLE),
|
new RetrieveUsedSegmentsAction(
|
||||||
|
"test",
|
||||||
|
Collections.singletonList(Intervals.of("2014/2015")),
|
||||||
|
Segments.ONLY_VISIBLE
|
||||||
|
),
|
||||||
actual
|
actual
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -35,18 +35,13 @@ import org.apache.druid.java.util.common.Intervals;
|
|||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.LinearShardSpec;
|
import org.apache.druid.timeline.partition.LinearShardSpec;
|
||||||
import org.assertj.core.api.Assertions;
|
import org.assertj.core.api.Assertions;
|
||||||
import org.hamcrest.CoreMatchers;
|
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Rule;
|
import org.junit.Rule;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.rules.ExpectedException;
|
|
||||||
|
|
||||||
public class SegmentTransactionalInsertActionTest
|
public class SegmentTransactionalInsertActionTest
|
||||||
{
|
{
|
||||||
@Rule
|
|
||||||
public ExpectedException thrown = ExpectedException.none();
|
|
||||||
|
|
||||||
@Rule
|
@Rule
|
||||||
public TaskActionTestKit actionTestKit = new TaskActionTestKit();
|
public TaskActionTestKit actionTestKit = new TaskActionTestKit();
|
||||||
|
|
||||||
@ -157,8 +152,8 @@ public class SegmentTransactionalInsertActionTest
|
|||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
SegmentPublishResult.fail(
|
SegmentPublishResult.fail(
|
||||||
InvalidInput.exception(
|
InvalidInput.exception(
|
||||||
"The new start metadata state[ObjectMetadata{theObject=[1]}] is ahead of the last commited end"
|
"The new start metadata state[ObjectMetadata{theObject=[1]}] is"
|
||||||
+ " state[null]. Try resetting the supervisor."
|
+ " ahead of the last committed end state[null]. Try resetting the supervisor."
|
||||||
).toString()
|
).toString()
|
||||||
),
|
),
|
||||||
result
|
result
|
||||||
@ -169,17 +164,15 @@ public class SegmentTransactionalInsertActionTest
|
|||||||
public void testFailBadVersion() throws Exception
|
public void testFailBadVersion() throws Exception
|
||||||
{
|
{
|
||||||
final Task task = NoopTask.create();
|
final Task task = NoopTask.create();
|
||||||
final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction.overwriteAction(
|
final SegmentTransactionalInsertAction action = SegmentTransactionalInsertAction
|
||||||
null,
|
.overwriteAction(null, ImmutableSet.of(SEGMENT3), null);
|
||||||
ImmutableSet.of(SEGMENT3),
|
|
||||||
null
|
|
||||||
);
|
|
||||||
actionTestKit.getTaskLockbox().add(task);
|
actionTestKit.getTaskLockbox().add(task);
|
||||||
acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
|
acquireTimeChunkLock(TaskLockType.EXCLUSIVE, task, INTERVAL, 5000);
|
||||||
|
|
||||||
thrown.expect(IllegalStateException.class);
|
IllegalStateException exception = Assert.assertThrows(
|
||||||
thrown.expectMessage(CoreMatchers.containsString("are not covered by locks"));
|
IllegalStateException.class,
|
||||||
SegmentPublishResult result = action.perform(task, actionTestKit.getTaskActionToolbox());
|
() -> action.perform(task, actionTestKit.getTaskActionToolbox())
|
||||||
Assert.assertEquals(SegmentPublishResult.ok(ImmutableSet.of(SEGMENT3)), result);
|
);
|
||||||
|
Assert.assertTrue(exception.getMessage().contains("are not covered by locks"));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -984,7 +984,6 @@ public class ConcurrentReplaceAndAppendTest extends IngestionTestBase
|
|||||||
Collection<DataSegment> allUsedSegments = dummyTaskActionClient.submit(
|
Collection<DataSegment> allUsedSegments = dummyTaskActionClient.submit(
|
||||||
new RetrieveUsedSegmentsAction(
|
new RetrieveUsedSegmentsAction(
|
||||||
WIKI,
|
WIKI,
|
||||||
null,
|
|
||||||
ImmutableList.of(interval),
|
ImmutableList.of(interval),
|
||||||
visibility
|
visibility
|
||||||
)
|
)
|
||||||
|
@ -709,7 +709,6 @@ public class ConcurrentReplaceAndStreamingAppendTest extends IngestionTestBase
|
|||||||
Collection<DataSegment> allUsedSegments = dummyTaskActionClient.submit(
|
Collection<DataSegment> allUsedSegments = dummyTaskActionClient.submit(
|
||||||
new RetrieveUsedSegmentsAction(
|
new RetrieveUsedSegmentsAction(
|
||||||
WIKI,
|
WIKI,
|
||||||
null,
|
|
||||||
ImmutableList.of(interval),
|
ImmutableList.of(interval),
|
||||||
visibility
|
visibility
|
||||||
)
|
)
|
||||||
@ -829,7 +828,6 @@ public class ConcurrentReplaceAndStreamingAppendTest extends IngestionTestBase
|
|||||||
return dummyTaskActionClient.submit(
|
return dummyTaskActionClient.submit(
|
||||||
new RetrieveUsedSegmentsAction(
|
new RetrieveUsedSegmentsAction(
|
||||||
WIKI,
|
WIKI,
|
||||||
null,
|
|
||||||
ImmutableList.of(Intervals.ETERNITY),
|
ImmutableList.of(Intervals.ETERNITY),
|
||||||
Segments.INCLUDING_OVERSHADOWED
|
Segments.INCLUDING_OVERSHADOWED
|
||||||
)
|
)
|
||||||
|
@ -85,9 +85,9 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility)
|
public Set<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility)
|
||||||
{
|
{
|
||||||
return ImmutableList.of();
|
return ImmutableSet.of();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -97,13 +97,13 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public List<DataSegment> retrieveUsedSegmentsForIntervals(
|
public Set<DataSegment> retrieveUsedSegmentsForIntervals(
|
||||||
String dataSource,
|
String dataSource,
|
||||||
List<Interval> intervals,
|
List<Interval> intervals,
|
||||||
Segments visibility
|
Segments visibility
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return ImmutableList.of();
|
return ImmutableSet.of();
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -61,7 +61,11 @@ public class ConfigManager
|
|||||||
private volatile PollingCallable poller;
|
private volatile PollingCallable poller;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public ConfigManager(MetadataStorageConnector dbConnector, Supplier<MetadataStorageTablesConfig> dbTables, Supplier<ConfigManagerConfig> config)
|
public ConfigManager(
|
||||||
|
MetadataStorageConnector dbConnector,
|
||||||
|
Supplier<MetadataStorageTablesConfig> dbTables,
|
||||||
|
Supplier<ConfigManagerConfig> config
|
||||||
|
)
|
||||||
{
|
{
|
||||||
this.dbConnector = dbConnector;
|
this.dbConnector = dbConnector;
|
||||||
this.config = config;
|
this.config = config;
|
||||||
|
@ -39,32 +39,15 @@ import java.util.Map;
|
|||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
* Handles metadata transactions performed by the Overlord.
|
||||||
*/
|
*/
|
||||||
public interface IndexerMetadataStorageCoordinator
|
public interface IndexerMetadataStorageCoordinator
|
||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* Retrieve all published segments which may include any data in the interval and are marked as used from the
|
* Retrieves all published segments that have partial or complete overlap with
|
||||||
* metadata store.
|
* the given interval and are marked as used.
|
||||||
*
|
|
||||||
* The order of segments within the returned collection is unspecified, but each segment is guaranteed to appear in
|
|
||||||
* the collection only once.
|
|
||||||
*
|
|
||||||
* @param dataSource The data source to query
|
|
||||||
* @param interval The interval for which all applicable and used segmented are requested.
|
|
||||||
* @param visibility Whether only visible or visible as well as overshadowed segments should be returned. The
|
|
||||||
* visibility is considered within the specified interval: that is, a segment which is visible
|
|
||||||
* outside of the specified interval, but overshadowed within the specified interval will not be
|
|
||||||
* returned if {@link Segments#ONLY_VISIBLE} is passed. See more precise description in the doc for
|
|
||||||
* {@link Segments}.
|
|
||||||
* @return The DataSegments which include data in the requested interval. These segments may contain data outside the
|
|
||||||
* requested interval.
|
|
||||||
*
|
|
||||||
* @implNote This method doesn't return a {@link Set} because there may be an expectation that {@code Set.contains()}
|
|
||||||
* is O(1) operation, while it's not the case for the returned collection unless it copies all segments into a new
|
|
||||||
* {@link java.util.HashSet} or {@link com.google.common.collect.ImmutableSet} which may in turn be unnecessary in
|
|
||||||
* other use cases. So clients should perform such copy themselves if they need {@link Set} semantics.
|
|
||||||
*/
|
*/
|
||||||
default Collection<DataSegment> retrieveUsedSegmentsForInterval(
|
default Set<DataSegment> retrieveUsedSegmentsForInterval(
|
||||||
String dataSource,
|
String dataSource,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
Segments visibility
|
Segments visibility
|
||||||
@ -74,21 +57,16 @@ public interface IndexerMetadataStorageCoordinator
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve all published used segments in the data source from the metadata store.
|
* Retrieves all published used segments for the given data source.
|
||||||
*
|
*
|
||||||
* @param dataSource The data source to query
|
* @see #retrieveUsedSegmentsForInterval(String, Interval, Segments)
|
||||||
*
|
|
||||||
* @return all segments belonging to the given data source
|
|
||||||
* @see #retrieveUsedSegmentsForInterval(String, Interval, Segments) similar to this method but also accepts data
|
|
||||||
* interval.
|
|
||||||
*/
|
*/
|
||||||
Collection<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility);
|
Set<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
|
||||||
* Retrieve all published segments which are marked as used and the created_date of these segments belonging to the
|
* Retrieve all published segments which are marked as used and the created_date of these segments belonging to the
|
||||||
* given data source and list of intervals from the metadata store.
|
* given data source and list of intervals from the metadata store.
|
||||||
*
|
* <p>
|
||||||
* Unlike other similar methods in this interface, this method doesn't accept a {@link Segments} "visibility"
|
* Unlike other similar methods in this interface, this method doesn't accept a {@link Segments} "visibility"
|
||||||
* parameter. The returned collection may include overshadowed segments and their created_dates, as if {@link
|
* parameter. The returned collection may include overshadowed segments and their created_dates, as if {@link
|
||||||
* Segments#INCLUDING_OVERSHADOWED} was passed. It's the responsibility of the caller to filter out overshadowed ones
|
* Segments#INCLUDING_OVERSHADOWED} was passed. It's the responsibility of the caller to filter out overshadowed ones
|
||||||
@ -99,32 +77,16 @@ public interface IndexerMetadataStorageCoordinator
|
|||||||
*
|
*
|
||||||
* @return The DataSegments and the related created_date of segments
|
* @return The DataSegments and the related created_date of segments
|
||||||
*/
|
*/
|
||||||
Collection<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(String dataSource, List<Interval> intervals);
|
Collection<Pair<DataSegment, String>> retrieveUsedSegmentsAndCreatedDates(
|
||||||
|
String dataSource,
|
||||||
|
List<Interval> intervals
|
||||||
|
);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Retrieve all published segments which may include any data in the given intervals and are marked as used from the
|
* Retrieves all published segments that have partial or complete overlap with
|
||||||
* metadata store.
|
* the given intervals and are marked as used.
|
||||||
* <p>
|
|
||||||
* The order of segments within the returned collection is unspecified, but each segment is guaranteed to appear in
|
|
||||||
* the collection only once.
|
|
||||||
* </p>
|
|
||||||
*
|
|
||||||
* @param dataSource The data source to query
|
|
||||||
* @param intervals The intervals for which all applicable and used segments are requested.
|
|
||||||
* @param visibility Whether only visible or visible as well as overshadowed segments should be returned. The
|
|
||||||
* visibility is considered within the specified intervals: that is, a segment which is visible
|
|
||||||
* outside of the specified intervals, but overshadowed on the specified intervals will not be
|
|
||||||
* returned if {@link Segments#ONLY_VISIBLE} is passed. See more precise description in the doc for
|
|
||||||
* {@link Segments}.
|
|
||||||
* @return The DataSegments which include data in the requested intervals. These segments may contain data outside the
|
|
||||||
* requested intervals.
|
|
||||||
*
|
|
||||||
* @implNote This method doesn't return a {@link Set} because there may be an expectation that {@code Set.contains()}
|
|
||||||
* is O(1) operation, while it's not the case for the returned collection unless it copies all segments into a new
|
|
||||||
* {@link java.util.HashSet} or {@link com.google.common.collect.ImmutableSet} which may in turn be unnecessary in
|
|
||||||
* other use cases. So clients should perform such copy themselves if they need {@link Set} semantics.
|
|
||||||
*/
|
*/
|
||||||
Collection<DataSegment> retrieveUsedSegmentsForIntervals(
|
Set<DataSegment> retrieveUsedSegmentsForIntervals(
|
||||||
String dataSource,
|
String dataSource,
|
||||||
List<Interval> intervals,
|
List<Interval> intervals,
|
||||||
Segments visibility
|
Segments visibility
|
||||||
|
@ -152,7 +152,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<DataSegment> retrieveUsedSegmentsForIntervals(
|
public Set<DataSegment> retrieveUsedSegmentsForIntervals(
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final List<Interval> intervals,
|
final List<Interval> intervals,
|
||||||
final Segments visibility
|
final Segments visibility
|
||||||
@ -165,7 +165,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Collection<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility)
|
public Set<DataSegment> retrieveAllUsedSegments(String dataSource, Segments visibility)
|
||||||
{
|
{
|
||||||
return doRetrieveUsedSegments(dataSource, Collections.emptyList(), visibility);
|
return doRetrieveUsedSegments(dataSource, Collections.emptyList(), visibility);
|
||||||
}
|
}
|
||||||
@ -173,7 +173,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
/**
|
/**
|
||||||
* @param intervals empty list means unrestricted interval.
|
* @param intervals empty list means unrestricted interval.
|
||||||
*/
|
*/
|
||||||
private Collection<DataSegment> doRetrieveUsedSegments(
|
private Set<DataSegment> doRetrieveUsedSegments(
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final List<Interval> intervals,
|
final List<Interval> intervals,
|
||||||
final Segments visibility
|
final Segments visibility
|
||||||
@ -431,7 +431,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private Collection<DataSegment> retrieveAllUsedSegmentsForIntervalsWithHandle(
|
private Set<DataSegment> retrieveAllUsedSegmentsForIntervalsWithHandle(
|
||||||
final Handle handle,
|
final Handle handle,
|
||||||
final String dataSource,
|
final String dataSource,
|
||||||
final List<Interval> intervals
|
final List<Interval> intervals
|
||||||
@ -440,7 +440,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
try (final CloseableIterator<DataSegment> iterator =
|
try (final CloseableIterator<DataSegment> iterator =
|
||||||
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
|
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
|
||||||
.retrieveUsedSegments(dataSource, intervals)) {
|
.retrieveUsedSegments(dataSource, intervals)) {
|
||||||
final List<DataSegment> retVal = new ArrayList<>();
|
final Set<DataSegment> retVal = new HashSet<>();
|
||||||
iterator.forEachRemaining(retVal::add);
|
iterator.forEachRemaining(retVal::add);
|
||||||
return retVal;
|
return retVal;
|
||||||
}
|
}
|
||||||
@ -2564,8 +2564,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
* oldCommitMetadata when this function is called (based on T.equals). This method is idempotent in that if
|
* oldCommitMetadata when this function is called (based on T.equals). This method is idempotent in that if
|
||||||
* the metadata already equals newCommitMetadata, it will return true.
|
* the metadata already equals newCommitMetadata, it will return true.
|
||||||
*
|
*
|
||||||
* @param handle database handle
|
|
||||||
* @param dataSource druid dataSource
|
|
||||||
* @param startMetadata dataSource metadata pre-insert must match this startMetadata according to
|
* @param startMetadata dataSource metadata pre-insert must match this startMetadata according to
|
||||||
* {@link DataSourceMetadata#matches(DataSourceMetadata)}
|
* {@link DataSourceMetadata#matches(DataSourceMetadata)}
|
||||||
* @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with
|
* @param endMetadata dataSource metadata post-insert will have this endMetadata merged in with
|
||||||
@ -2627,15 +2625,16 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
|
|
||||||
if (startMetadataGreaterThanExisting && !startMetadataMatchesExisting) {
|
if (startMetadataGreaterThanExisting && !startMetadataMatchesExisting) {
|
||||||
// Offsets stored in startMetadata is greater than the last commited metadata.
|
// Offsets stored in startMetadata is greater than the last commited metadata.
|
||||||
return new DataStoreMetadataUpdateResult(true, false,
|
return DataStoreMetadataUpdateResult.failure(
|
||||||
"The new start metadata state[%s] is ahead of the last commited"
|
"The new start metadata state[%s] is ahead of the last committed"
|
||||||
+ " end state[%s]. Try resetting the supervisor.", startMetadata, oldCommitMetadataFromDb
|
+ " end state[%s]. Try resetting the supervisor.",
|
||||||
|
startMetadata, oldCommitMetadataFromDb
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!startMetadataMatchesExisting) {
|
if (!startMetadataMatchesExisting) {
|
||||||
// Not in the desired start state.
|
// Not in the desired start state.
|
||||||
return new DataStoreMetadataUpdateResult(true, false,
|
return DataStoreMetadataUpdateResult.failure(
|
||||||
"Inconsistency between stored metadata state[%s] and target state[%s]. Try resetting the supervisor.",
|
"Inconsistency between stored metadata state[%s] and target state[%s]. Try resetting the supervisor.",
|
||||||
oldCommitMetadataFromDb, startMetadata
|
oldCommitMetadataFromDb, startMetadata
|
||||||
);
|
);
|
||||||
@ -2668,11 +2667,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
|
|
||||||
retVal = numRows == 1
|
retVal = numRows == 1
|
||||||
? DataStoreMetadataUpdateResult.SUCCESS
|
? DataStoreMetadataUpdateResult.SUCCESS
|
||||||
: new DataStoreMetadataUpdateResult(
|
: DataStoreMetadataUpdateResult.retryableFailure("Failed to insert metadata for datasource[%s]", dataSource);
|
||||||
true,
|
|
||||||
true,
|
|
||||||
"Failed to insert metadata for datasource [%s]",
|
|
||||||
dataSource);
|
|
||||||
} else {
|
} else {
|
||||||
// Expecting a particular old metadata; use the SHA1 in a compare-and-swap UPDATE
|
// Expecting a particular old metadata; use the SHA1 in a compare-and-swap UPDATE
|
||||||
final int numRows = handle.createStatement(
|
final int numRows = handle.createStatement(
|
||||||
@ -2692,11 +2687,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
|
|
||||||
retVal = numRows == 1
|
retVal = numRows == 1
|
||||||
? DataStoreMetadataUpdateResult.SUCCESS
|
? DataStoreMetadataUpdateResult.SUCCESS
|
||||||
: new DataStoreMetadataUpdateResult(
|
: DataStoreMetadataUpdateResult.retryableFailure("Failed to update metadata for datasource[%s]", dataSource);
|
||||||
true,
|
|
||||||
true,
|
|
||||||
"Failed to update metadata for datasource [%s]",
|
|
||||||
dataSource);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
if (retVal.isSuccess()) {
|
if (retVal.isSuccess()) {
|
||||||
@ -2712,19 +2703,12 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
public boolean deleteDataSourceMetadata(final String dataSource)
|
public boolean deleteDataSourceMetadata(final String dataSource)
|
||||||
{
|
{
|
||||||
return connector.retryWithHandle(
|
return connector.retryWithHandle(
|
||||||
new HandleCallback<Boolean>()
|
handle -> {
|
||||||
{
|
int rows = handle.createStatement(
|
||||||
@Override
|
StringUtils.format("DELETE from %s WHERE dataSource = :dataSource", dbTables.getDataSourceTable())
|
||||||
public Boolean withHandle(Handle handle)
|
).bind("dataSource", dataSource).execute();
|
||||||
{
|
|
||||||
int rows = handle.createStatement(
|
|
||||||
StringUtils.format("DELETE from %s WHERE dataSource = :dataSource", dbTables.getDataSourceTable())
|
|
||||||
)
|
|
||||||
.bind("dataSource", dataSource)
|
|
||||||
.execute();
|
|
||||||
|
|
||||||
return rows > 0;
|
return rows > 0;
|
||||||
}
|
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -2767,17 +2751,12 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
public void updateSegmentMetadata(final Set<DataSegment> segments)
|
public void updateSegmentMetadata(final Set<DataSegment> segments)
|
||||||
{
|
{
|
||||||
connector.getDBI().inTransaction(
|
connector.getDBI().inTransaction(
|
||||||
new TransactionCallback<Void>()
|
(handle, transactionStatus) -> {
|
||||||
{
|
for (final DataSegment segment : segments) {
|
||||||
@Override
|
updatePayload(handle, segment);
|
||||||
public Void inTransaction(Handle handle, TransactionStatus transactionStatus) throws Exception
|
|
||||||
{
|
|
||||||
for (final DataSegment segment : segments) {
|
|
||||||
updatePayload(handle, segment);
|
|
||||||
}
|
|
||||||
|
|
||||||
return null;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
return 0;
|
||||||
}
|
}
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
@ -2990,10 +2969,21 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
{
|
{
|
||||||
private final boolean failed;
|
private final boolean failed;
|
||||||
private final boolean canRetry;
|
private final boolean canRetry;
|
||||||
@Nullable private final String errorMsg;
|
@Nullable
|
||||||
|
private final String errorMsg;
|
||||||
|
|
||||||
public static final DataStoreMetadataUpdateResult SUCCESS = new DataStoreMetadataUpdateResult(false, false, null);
|
public static final DataStoreMetadataUpdateResult SUCCESS = new DataStoreMetadataUpdateResult(false, false, null);
|
||||||
|
|
||||||
|
public static DataStoreMetadataUpdateResult failure(String errorMsgFormat, Object... messageArgs)
|
||||||
|
{
|
||||||
|
return new DataStoreMetadataUpdateResult(true, false, errorMsgFormat, messageArgs);
|
||||||
|
}
|
||||||
|
|
||||||
|
public static DataStoreMetadataUpdateResult retryableFailure(String errorMsgFormat, Object... messageArgs)
|
||||||
|
{
|
||||||
|
return new DataStoreMetadataUpdateResult(true, true, errorMsgFormat, messageArgs);
|
||||||
|
}
|
||||||
|
|
||||||
DataStoreMetadataUpdateResult(boolean failed, boolean canRetry, @Nullable String errorMsg, Object... errorFormatArgs)
|
DataStoreMetadataUpdateResult(boolean failed, boolean canRetry, @Nullable String errorMsg, Object... errorFormatArgs)
|
||||||
{
|
{
|
||||||
this.failed = failed;
|
this.failed = failed;
|
||||||
@ -3022,34 +3012,5 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||||||
{
|
{
|
||||||
return errorMsg;
|
return errorMsg;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public boolean equals(Object o)
|
|
||||||
{
|
|
||||||
if (this == o) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
if (o == null || getClass() != o.getClass()) {
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
DataStoreMetadataUpdateResult that = (DataStoreMetadataUpdateResult) o;
|
|
||||||
return failed == that.failed && canRetry == that.canRetry && Objects.equals(errorMsg, that.errorMsg);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public int hashCode()
|
|
||||||
{
|
|
||||||
return Objects.hash(failed, canRetry, errorMsg);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public String toString()
|
|
||||||
{
|
|
||||||
return "DataStoreMetadataUpdateResult{" +
|
|
||||||
"failed=" + failed +
|
|
||||||
", canRetry=" + canRetry +
|
|
||||||
", errorMsg='" + errorMsg + '\'' +
|
|
||||||
'}';
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -318,7 +318,7 @@ public abstract class SQLMetadataConnector implements MetadataStorageConnector
|
|||||||
|
|
||||||
public void createSegmentTable(final String tableName)
|
public void createSegmentTable(final String tableName)
|
||||||
{
|
{
|
||||||
List<String> columns = new ArrayList<>();
|
final List<String> columns = new ArrayList<>();
|
||||||
columns.add("id VARCHAR(255) NOT NULL");
|
columns.add("id VARCHAR(255) NOT NULL");
|
||||||
columns.add("dataSource VARCHAR(255) %4$s NOT NULL");
|
columns.add("dataSource VARCHAR(255) %4$s NOT NULL");
|
||||||
columns.add("created_date VARCHAR(255) NOT NULL");
|
columns.add("created_date VARCHAR(255) NOT NULL");
|
||||||
|
@ -254,7 +254,7 @@ public abstract class BaseAppenderatorDriver implements Closeable
|
|||||||
private static final Logger log = new Logger(BaseAppenderatorDriver.class);
|
private static final Logger log = new Logger(BaseAppenderatorDriver.class);
|
||||||
|
|
||||||
private final SegmentAllocator segmentAllocator;
|
private final SegmentAllocator segmentAllocator;
|
||||||
private final UsedSegmentChecker usedSegmentChecker;
|
private final PublishedSegmentRetriever publishedSegmentRetriever;
|
||||||
private final DataSegmentKiller dataSegmentKiller;
|
private final DataSegmentKiller dataSegmentKiller;
|
||||||
|
|
||||||
protected final Appenderator appenderator;
|
protected final Appenderator appenderator;
|
||||||
@ -269,13 +269,13 @@ public abstract class BaseAppenderatorDriver implements Closeable
|
|||||||
BaseAppenderatorDriver(
|
BaseAppenderatorDriver(
|
||||||
Appenderator appenderator,
|
Appenderator appenderator,
|
||||||
SegmentAllocator segmentAllocator,
|
SegmentAllocator segmentAllocator,
|
||||||
UsedSegmentChecker usedSegmentChecker,
|
PublishedSegmentRetriever segmentRetriever,
|
||||||
DataSegmentKiller dataSegmentKiller
|
DataSegmentKiller dataSegmentKiller
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.appenderator = Preconditions.checkNotNull(appenderator, "appenderator");
|
this.appenderator = Preconditions.checkNotNull(appenderator, "appenderator");
|
||||||
this.segmentAllocator = Preconditions.checkNotNull(segmentAllocator, "segmentAllocator");
|
this.segmentAllocator = Preconditions.checkNotNull(segmentAllocator, "segmentAllocator");
|
||||||
this.usedSegmentChecker = Preconditions.checkNotNull(usedSegmentChecker, "segmentRetriever");
|
this.publishedSegmentRetriever = Preconditions.checkNotNull(segmentRetriever, "segmentRetriever");
|
||||||
this.dataSegmentKiller = Preconditions.checkNotNull(dataSegmentKiller, "dataSegmentKiller");
|
this.dataSegmentKiller = Preconditions.checkNotNull(dataSegmentKiller, "dataSegmentKiller");
|
||||||
this.executor = MoreExecutors.listeningDecorator(
|
this.executor = MoreExecutors.listeningDecorator(
|
||||||
Execs.singleThreaded("[" + StringUtils.encodeForFormat(appenderator.getId()) + "]-publish")
|
Execs.singleThreaded("[" + StringUtils.encodeForFormat(appenderator.getId()) + "]-publish")
|
||||||
@ -665,7 +665,7 @@ public abstract class BaseAppenderatorDriver implements Closeable
|
|||||||
.map(DataSegment::getId)
|
.map(DataSegment::getId)
|
||||||
.collect(Collectors.toSet());
|
.collect(Collectors.toSet());
|
||||||
|
|
||||||
final Set<DataSegment> publishedSegments = usedSegmentChecker.findPublishedSegments(segmentIds);
|
final Set<DataSegment> publishedSegments = publishedSegmentRetriever.findPublishedSegments(segmentIds);
|
||||||
if (publishedSegments.equals(ourSegments)) {
|
if (publishedSegments.equals(ourSegments)) {
|
||||||
log.info(
|
log.info(
|
||||||
"Could not publish [%d] segments, but they have already been published by another task.",
|
"Could not publish [%d] segments, but they have already been published by another task.",
|
||||||
|
@ -61,20 +61,16 @@ import java.util.stream.Collectors;
|
|||||||
public class BatchAppenderatorDriver extends BaseAppenderatorDriver
|
public class BatchAppenderatorDriver extends BaseAppenderatorDriver
|
||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* Create a driver.
|
* Creates a driver for batch ingestion.
|
||||||
*
|
|
||||||
* @param appenderator appenderator
|
|
||||||
* @param segmentAllocator segment allocator
|
|
||||||
* @param usedSegmentChecker used segment checker
|
|
||||||
*/
|
*/
|
||||||
public BatchAppenderatorDriver(
|
public BatchAppenderatorDriver(
|
||||||
Appenderator appenderator,
|
Appenderator appenderator,
|
||||||
SegmentAllocator segmentAllocator,
|
SegmentAllocator segmentAllocator,
|
||||||
UsedSegmentChecker usedSegmentChecker,
|
PublishedSegmentRetriever segmentRetriever,
|
||||||
DataSegmentKiller dataSegmentKiller
|
DataSegmentKiller dataSegmentKiller
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(appenderator, segmentAllocator, usedSegmentChecker, dataSegmentKiller);
|
super(appenderator, segmentAllocator, segmentRetriever, dataSegmentKiller);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
|
@ -25,7 +25,7 @@ import org.apache.druid.timeline.SegmentId;
|
|||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
public interface UsedSegmentChecker
|
public interface PublishedSegmentRetriever
|
||||||
{
|
{
|
||||||
/**
|
/**
|
||||||
* For any identifiers that exist and are actually used, returns the corresponding DataSegment objects.
|
* For any identifiers that exist and are actually used, returns the corresponding DataSegment objects.
|
@ -91,7 +91,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
|
|||||||
* @param appenderator appenderator
|
* @param appenderator appenderator
|
||||||
* @param segmentAllocator segment allocator
|
* @param segmentAllocator segment allocator
|
||||||
* @param handoffNotifierFactory handoff notifier factory
|
* @param handoffNotifierFactory handoff notifier factory
|
||||||
* @param usedSegmentChecker used segment checker
|
* @param segmentRetriever used segment checker
|
||||||
* @param objectMapper object mapper, used for serde of commit metadata
|
* @param objectMapper object mapper, used for serde of commit metadata
|
||||||
* @param metrics Firedepartment metrics
|
* @param metrics Firedepartment metrics
|
||||||
*/
|
*/
|
||||||
@ -99,13 +99,13 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
|
|||||||
Appenderator appenderator,
|
Appenderator appenderator,
|
||||||
SegmentAllocator segmentAllocator,
|
SegmentAllocator segmentAllocator,
|
||||||
SegmentHandoffNotifierFactory handoffNotifierFactory,
|
SegmentHandoffNotifierFactory handoffNotifierFactory,
|
||||||
UsedSegmentChecker usedSegmentChecker,
|
PublishedSegmentRetriever segmentRetriever,
|
||||||
DataSegmentKiller dataSegmentKiller,
|
DataSegmentKiller dataSegmentKiller,
|
||||||
ObjectMapper objectMapper,
|
ObjectMapper objectMapper,
|
||||||
SegmentGenerationMetrics metrics
|
SegmentGenerationMetrics metrics
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
super(appenderator, segmentAllocator, usedSegmentChecker, dataSegmentKiller);
|
super(appenderator, segmentAllocator, segmentRetriever, dataSegmentKiller);
|
||||||
|
|
||||||
this.handoffNotifier = Preconditions.checkNotNull(handoffNotifierFactory, "handoffNotifierFactory")
|
this.handoffNotifier = Preconditions.checkNotNull(handoffNotifierFactory, "handoffNotifierFactory")
|
||||||
.createSegmentHandoffNotifier(appenderator.getDataSource());
|
.createSegmentHandoffNotifier(appenderator.getDataSource());
|
||||||
@ -416,7 +416,7 @@ public class StreamAppenderatorDriver extends BaseAppenderatorDriver
|
|||||||
{
|
{
|
||||||
return Futures.transformAsync(
|
return Futures.transformAsync(
|
||||||
publish(publisher, committer, sequenceNames),
|
publish(publisher, committer, sequenceNames),
|
||||||
(AsyncFunction<SegmentsAndCommitMetadata, SegmentsAndCommitMetadata>) this::registerHandoff,
|
this::registerHandoff,
|
||||||
MoreExecutors.directExecutor()
|
MoreExecutors.directExecutor()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -523,7 +523,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
|
|||||||
{
|
{
|
||||||
metadataUpdateCounter.getAndIncrement();
|
metadataUpdateCounter.getAndIncrement();
|
||||||
if (attemptCounter.getAndIncrement() == 0) {
|
if (attemptCounter.getAndIncrement() == 0) {
|
||||||
return new DataStoreMetadataUpdateResult(true, true, null);
|
return DataStoreMetadataUpdateResult.retryableFailure(null);
|
||||||
} else {
|
} else {
|
||||||
return super.updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata);
|
return super.updateDataSourceMetadataWithHandle(handle, dataSource, startMetadata, endMetadata);
|
||||||
}
|
}
|
||||||
@ -593,7 +593,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
|
|||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
SegmentPublishResult.fail(
|
SegmentPublishResult.fail(
|
||||||
InvalidInput.exception(
|
InvalidInput.exception(
|
||||||
"The new start metadata state[ObjectMetadata{theObject={foo=bar}}] is ahead of the last commited"
|
"The new start metadata state[ObjectMetadata{theObject={foo=bar}}] is ahead of the last committed"
|
||||||
+ " end state[null]. Try resetting the supervisor."
|
+ " end state[null]. Try resetting the supervisor."
|
||||||
).toString()),
|
).toString()),
|
||||||
result1
|
result1
|
||||||
|
@ -95,7 +95,7 @@ public class ClosedSegmentsSinksBatchAppenderatorDriverTest extends EasyMockSupp
|
|||||||
driver = new BatchAppenderatorDriver(
|
driver = new BatchAppenderatorDriver(
|
||||||
appenderatorTester.getAppenderator(),
|
appenderatorTester.getAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
new TestUsedSegmentChecker(appenderatorTester.getPushedSegments()),
|
new TestPublishedSegmentRetriever(appenderatorTester.getPushedSegments()),
|
||||||
dataSegmentKiller
|
dataSegmentKiller
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -93,7 +93,7 @@ public class OpenAndClosedSegmentsBatchAppenderatorDriverTest extends EasyMockSu
|
|||||||
driver = new BatchAppenderatorDriver(
|
driver = new BatchAppenderatorDriver(
|
||||||
openAndClosedSegmentsAppenderatorTester.getAppenderator(),
|
openAndClosedSegmentsAppenderatorTester.getAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
new TestUsedSegmentChecker(openAndClosedSegmentsAppenderatorTester.getPushedSegments()),
|
new TestPublishedSegmentRetriever(openAndClosedSegmentsAppenderatorTester.getPushedSegments()),
|
||||||
dataSegmentKiller
|
dataSegmentKiller
|
||||||
);
|
);
|
||||||
|
|
||||||
|
@ -133,7 +133,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport
|
|||||||
createPersistFailAppenderator(),
|
createPersistFailAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
segmentHandoffNotifierFactory,
|
segmentHandoffNotifierFactory,
|
||||||
new NoopUsedSegmentChecker(),
|
new NoopPublishedSegmentRetriever(),
|
||||||
dataSegmentKiller,
|
dataSegmentKiller,
|
||||||
OBJECT_MAPPER,
|
OBJECT_MAPPER,
|
||||||
new SegmentGenerationMetrics()
|
new SegmentGenerationMetrics()
|
||||||
@ -171,7 +171,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport
|
|||||||
createPushFailAppenderator(),
|
createPushFailAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
segmentHandoffNotifierFactory,
|
segmentHandoffNotifierFactory,
|
||||||
new NoopUsedSegmentChecker(),
|
new NoopPublishedSegmentRetriever(),
|
||||||
dataSegmentKiller,
|
dataSegmentKiller,
|
||||||
OBJECT_MAPPER,
|
OBJECT_MAPPER,
|
||||||
new SegmentGenerationMetrics()
|
new SegmentGenerationMetrics()
|
||||||
@ -209,7 +209,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport
|
|||||||
createDropFailAppenderator(),
|
createDropFailAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
segmentHandoffNotifierFactory,
|
segmentHandoffNotifierFactory,
|
||||||
new NoopUsedSegmentChecker(),
|
new NoopPublishedSegmentRetriever(),
|
||||||
dataSegmentKiller,
|
dataSegmentKiller,
|
||||||
OBJECT_MAPPER,
|
OBJECT_MAPPER,
|
||||||
new SegmentGenerationMetrics()
|
new SegmentGenerationMetrics()
|
||||||
@ -260,7 +260,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport
|
|||||||
new FailableAppenderator(),
|
new FailableAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
segmentHandoffNotifierFactory,
|
segmentHandoffNotifierFactory,
|
||||||
new NoopUsedSegmentChecker(),
|
new NoopPublishedSegmentRetriever(),
|
||||||
dataSegmentKiller,
|
dataSegmentKiller,
|
||||||
OBJECT_MAPPER,
|
OBJECT_MAPPER,
|
||||||
new SegmentGenerationMetrics()
|
new SegmentGenerationMetrics()
|
||||||
@ -324,7 +324,7 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class NoopUsedSegmentChecker implements UsedSegmentChecker
|
private static class NoopPublishedSegmentRetriever implements PublishedSegmentRetriever
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Set<DataSegment> findPublishedSegments(Set<SegmentId> identifiers)
|
public Set<DataSegment> findPublishedSegments(Set<SegmentId> identifiers)
|
||||||
@ -338,11 +338,6 @@ public class StreamAppenderatorDriverFailTest extends EasyMockSupport
|
|||||||
return new FailableAppenderator().disablePush();
|
return new FailableAppenderator().disablePush();
|
||||||
}
|
}
|
||||||
|
|
||||||
static Appenderator createPushInterruptAppenderator()
|
|
||||||
{
|
|
||||||
return new FailableAppenderator().interruptPush();
|
|
||||||
}
|
|
||||||
|
|
||||||
static Appenderator createPersistFailAppenderator()
|
static Appenderator createPersistFailAppenderator()
|
||||||
{
|
{
|
||||||
return new FailableAppenderator().disablePersist();
|
return new FailableAppenderator().disablePersist();
|
||||||
|
@ -127,7 +127,7 @@ public class StreamAppenderatorDriverTest extends EasyMockSupport
|
|||||||
streamAppenderatorTester.getAppenderator(),
|
streamAppenderatorTester.getAppenderator(),
|
||||||
allocator,
|
allocator,
|
||||||
segmentHandoffNotifierFactory,
|
segmentHandoffNotifierFactory,
|
||||||
new TestUsedSegmentChecker(streamAppenderatorTester.getPushedSegments()),
|
new TestPublishedSegmentRetriever(streamAppenderatorTester.getPushedSegments()),
|
||||||
dataSegmentKiller,
|
dataSegmentKiller,
|
||||||
OBJECT_MAPPER,
|
OBJECT_MAPPER,
|
||||||
new SegmentGenerationMetrics()
|
new SegmentGenerationMetrics()
|
||||||
|
@ -29,11 +29,11 @@ import java.util.HashSet;
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
|
||||||
public class TestUsedSegmentChecker implements UsedSegmentChecker
|
public class TestPublishedSegmentRetriever implements PublishedSegmentRetriever
|
||||||
{
|
{
|
||||||
private final List<DataSegment> pushedSegments;
|
private final List<DataSegment> pushedSegments;
|
||||||
|
|
||||||
public TestUsedSegmentChecker(List<DataSegment> pushedSegments)
|
public TestPublishedSegmentRetriever(List<DataSegment> pushedSegments)
|
||||||
{
|
{
|
||||||
this.pushedSegments = pushedSegments;
|
this.pushedSegments = pushedSegments;
|
||||||
}
|
}
|
Loading…
x
Reference in New Issue
Block a user