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:
Kashif Faraz 2024-06-25 22:18:59 -07:00 committed by GitHub
parent 52c9929019
commit d9bd02256a
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
29 changed files with 137 additions and 248 deletions

View File

@ -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())) {

View File

@ -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);

View File

@ -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),

View File

@ -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()
); );
} }

View File

@ -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
); );

View File

@ -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));
} }
} }

View File

@ -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

View File

@ -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]",

View File

@ -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)

View File

@ -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
) )

View File

@ -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
); );
} }

View File

@ -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"));
} }
} }

View File

@ -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
) )

View File

@ -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
) )

View File

@ -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

View File

@ -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;

View File

@ -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

View File

@ -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 + '\'' +
'}';
}
} }
} }

View File

@ -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");

View File

@ -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.",

View File

@ -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

View File

@ -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.

View File

@ -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()
); );
} }

View File

@ -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

View File

@ -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
); );

View File

@ -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
); );

View File

@ -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();

View File

@ -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()

View File

@ -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;
} }