mirror of https://github.com/apache/druid.git
Filter pending segments upgraded with transactional replace (#15169)
* Filter pending segments upgraded with transactional replace * Push sequence name filter to metadata query
This commit is contained in:
parent
2e31cb2901
commit
65b69cded4
|
@ -296,6 +296,12 @@ public class MaterializedViewSupervisor implements Supervisor
|
|||
throw new UnsupportedOperationException("Compute Lag Stats not supported in MaterializedViewSupervisor");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getActiveRealtimeSequencePrefixes()
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getActiveTaskGroupsCount()
|
||||
{
|
||||
|
|
|
@ -203,19 +203,11 @@ public class MaterializedViewSupervisorSpecTest
|
|||
SupervisorTaskAutoScaler autoscaler = spec.createAutoscaler(supervisor);
|
||||
Assert.assertNull(autoscaler);
|
||||
|
||||
try {
|
||||
supervisor.computeLagStats();
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(e instanceof UnsupportedOperationException);
|
||||
}
|
||||
Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.computeLagStats());
|
||||
|
||||
try {
|
||||
int count = supervisor.getActiveTaskGroupsCount();
|
||||
}
|
||||
catch (Exception e) {
|
||||
Assert.assertTrue(e instanceof UnsupportedOperationException);
|
||||
}
|
||||
Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.getActiveTaskGroupsCount());
|
||||
|
||||
Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.getActiveRealtimeSequencePrefixes());
|
||||
|
||||
Callable<Integer> noop = new Callable<Integer>() {
|
||||
@Override
|
||||
|
|
|
@ -145,8 +145,11 @@ public class SegmentTransactionalReplaceAction implements TaskAction<SegmentPubl
|
|||
return;
|
||||
}
|
||||
|
||||
final Set<String> activeRealtimeSequencePrefixes
|
||||
= supervisorManager.getActiveRealtimeSequencePrefixes(activeSupervisorIdWithAppendLock.get());
|
||||
Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> upgradedPendingSegments =
|
||||
toolbox.getIndexerMetadataStorageCoordinator().upgradePendingSegmentsOverlappingWith(segments);
|
||||
toolbox.getIndexerMetadataStorageCoordinator()
|
||||
.upgradePendingSegmentsOverlappingWith(segments, activeRealtimeSequencePrefixes);
|
||||
log.info(
|
||||
"Upgraded [%d] pending segments for REPLACE task[%s]: [%s]",
|
||||
upgradedPendingSegments.size(), task.getId(), upgradedPendingSegments
|
||||
|
|
|
@ -39,6 +39,7 @@ import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
|
|||
|
||||
import javax.annotation.Nullable;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -110,6 +111,15 @@ public class SupervisorManager
|
|||
return Optional.absent();
|
||||
}
|
||||
|
||||
public Set<String> getActiveRealtimeSequencePrefixes(String activeSupervisorId)
|
||||
{
|
||||
if (supervisors.containsKey(activeSupervisorId)) {
|
||||
return supervisors.get(activeSupervisorId).lhs.getActiveRealtimeSequencePrefixes();
|
||||
} else {
|
||||
return Collections.emptySet();
|
||||
}
|
||||
}
|
||||
|
||||
public Optional<SupervisorSpec> getSupervisorSpec(String id)
|
||||
{
|
||||
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(id);
|
||||
|
|
|
@ -1093,6 +1093,28 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
|
|||
addNotice(new ResetOffsetsNotice(resetDataSourceMetadata));
|
||||
}
|
||||
|
||||
/**
|
||||
* The base sequence name of a seekable stream task group is used as a prefix of the sequence names
|
||||
* of pending segments published by it.
|
||||
* This method can be used to identify the active pending segments for a datasource
|
||||
* by checking if the sequence name begins with any of the active realtime sequence prefix returned by this method
|
||||
* @return the set of base sequence names of both active and pending completion task gruops.
|
||||
*/
|
||||
@Override
|
||||
public Set<String> getActiveRealtimeSequencePrefixes()
|
||||
{
|
||||
final Set<String> activeBaseSequences = new HashSet<>();
|
||||
for (TaskGroup taskGroup : activelyReadingTaskGroups.values()) {
|
||||
activeBaseSequences.add(taskGroup.baseSequenceName);
|
||||
}
|
||||
for (List<TaskGroup> taskGroupList : pendingCompletionTaskGroups.values()) {
|
||||
for (TaskGroup taskGroup : taskGroupList) {
|
||||
activeBaseSequences.add(taskGroup.baseSequenceName);
|
||||
}
|
||||
}
|
||||
return activeBaseSequences;
|
||||
}
|
||||
|
||||
public void registerNewVersionOfPendingSegment(
|
||||
SegmentIdWithShardSpec basePendingSegment,
|
||||
SegmentIdWithShardSpec newSegmentVersion
|
||||
|
|
|
@ -1313,6 +1313,48 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
|||
validateSupervisorStateAfterResetOffsets(supervisor, resetOffsets, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetActiveRealtimeSequencePrefixes()
|
||||
{
|
||||
EasyMock.expect(spec.isSuspended()).andReturn(false);
|
||||
|
||||
replayAll();
|
||||
|
||||
final TestSeekableStreamSupervisor supervisor = new TestSeekableStreamSupervisor();
|
||||
|
||||
// Spin off two active tasks with each task serving one partition.
|
||||
supervisor.getIoConfig().setTaskCount(3);
|
||||
supervisor.start();
|
||||
supervisor.addTaskGroupToActivelyReadingTaskGroup(
|
||||
supervisor.getTaskGroupIdForPartition("0"),
|
||||
ImmutableMap.of("0", "5"),
|
||||
Optional.absent(),
|
||||
Optional.absent(),
|
||||
ImmutableSet.of("task1"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
supervisor.addTaskGroupToActivelyReadingTaskGroup(
|
||||
supervisor.getTaskGroupIdForPartition("1"),
|
||||
ImmutableMap.of("1", "6"),
|
||||
Optional.absent(),
|
||||
Optional.absent(),
|
||||
ImmutableSet.of("task2"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
supervisor.addTaskGroupToPendingCompletionTaskGroup(
|
||||
supervisor.getTaskGroupIdForPartition("2"),
|
||||
ImmutableMap.of("2", "100"),
|
||||
Optional.absent(),
|
||||
Optional.absent(),
|
||||
ImmutableSet.of("task3"),
|
||||
ImmutableSet.of()
|
||||
);
|
||||
|
||||
Assert.assertEquals(3, supervisor.getActiveRealtimeSequencePrefixes().size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSupervisorResetSpecificOffsetsTasksWithCheckpoints() throws InterruptedException, IOException
|
||||
{
|
||||
|
|
|
@ -238,7 +238,10 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
|
|||
}
|
||||
|
||||
@Override
|
||||
public Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> upgradePendingSegmentsOverlappingWith(Set<DataSegment> replaceSegments)
|
||||
public Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> upgradePendingSegmentsOverlappingWith(
|
||||
Set<DataSegment> replaceSegments,
|
||||
Set<String> activeBaseSequenceNames
|
||||
)
|
||||
{
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
|
|
@ -347,10 +347,13 @@ public interface IndexerMetadataStorageCoordinator
|
|||
* </ul>
|
||||
*
|
||||
* @param replaceSegments Segments being committed by a REPLACE task
|
||||
* @param activeRealtimeSequencePrefixes Set of sequence prefixes of active and pending completion task groups
|
||||
* of the supervisor (if any) for this datasource
|
||||
* @return Map from originally allocated pending segment to its new upgraded ID.
|
||||
*/
|
||||
Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> upgradePendingSegmentsOverlappingWith(
|
||||
Set<DataSegment> replaceSegments
|
||||
Set<DataSegment> replaceSegments,
|
||||
Set<String> activeRealtimeSequencePrefixes
|
||||
);
|
||||
|
||||
/**
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.druid.server.security.ResourceAction;
|
|||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Set;
|
||||
|
@ -185,6 +186,12 @@ public class NoopSupervisorSpec implements SupervisorSpec
|
|||
{
|
||||
return -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Set<String> getActiveRealtimeSequencePrefixes()
|
||||
{
|
||||
return Collections.emptySet();
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
|
|
|
@ -29,6 +29,7 @@ import org.apache.druid.segment.incremental.ParseExceptionReport;
|
|||
import javax.annotation.Nullable;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
public interface Supervisor
|
||||
{
|
||||
|
@ -93,4 +94,9 @@ public interface Supervisor
|
|||
LagStats computeLagStats();
|
||||
|
||||
int getActiveTaskGroupsCount();
|
||||
|
||||
/**
|
||||
* @return active sequence prefixes for reading and pending completion task groups of a seekable stream supervisor
|
||||
*/
|
||||
Set<String> getActiveRealtimeSequencePrefixes();
|
||||
}
|
||||
|
|
|
@ -231,7 +231,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
(handle, status) -> {
|
||||
try (final CloseableIterator<DataSegment> iterator =
|
||||
SqlSegmentsMetadataQuery.forHandle(handle, connector, dbTables, jsonMapper)
|
||||
.retrieveUnusedSegments(dataSource, Collections.singletonList(interval), limit)) {
|
||||
.retrieveUnusedSegments(dataSource, Collections.singletonList(interval), limit)) {
|
||||
return ImmutableList.copyOf(iterator);
|
||||
}
|
||||
}
|
||||
|
@ -258,9 +258,62 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
|
||||
/**
|
||||
* Fetches all the pending segments, whose interval overlaps with the given
|
||||
* search interval from the metadata store. Returns a Map from the
|
||||
* pending segment ID to the sequence name.
|
||||
* search interval and has a sequence_name that begins with one of the prefixes in sequenceNamePrefixFilter
|
||||
* from the metadata store. Returns a Map from the pending segment ID to the sequence name.
|
||||
*/
|
||||
@VisibleForTesting
|
||||
Map<SegmentIdWithShardSpec, String> getPendingSegmentsForIntervalWithHandle(
|
||||
final Handle handle,
|
||||
final String dataSource,
|
||||
final Interval interval,
|
||||
final Set<String> sequenceNamePrefixFilter
|
||||
) throws IOException
|
||||
{
|
||||
if (sequenceNamePrefixFilter.isEmpty()) {
|
||||
return Collections.emptyMap();
|
||||
}
|
||||
|
||||
final List<String> sequenceNamePrefixes = new ArrayList<>(sequenceNamePrefixFilter);
|
||||
final List<String> sequenceNamePrefixConditions = new ArrayList<>();
|
||||
for (int i = 0; i < sequenceNamePrefixes.size(); i++) {
|
||||
sequenceNamePrefixConditions.add(StringUtils.format("(sequence_name LIKE :prefix%d)", i));
|
||||
}
|
||||
|
||||
String sql = "SELECT sequence_name, payload"
|
||||
+ " FROM " + dbTables.getPendingSegmentsTable()
|
||||
+ " WHERE dataSource = :dataSource"
|
||||
+ " AND start < :end"
|
||||
+ StringUtils.format(" AND %1$send%1$s > :start", connector.getQuoteString())
|
||||
+ " AND ( " + String.join(" OR ", sequenceNamePrefixConditions) + " )";
|
||||
|
||||
Query<Map<String, Object>> query = handle.createQuery(sql)
|
||||
.bind("dataSource", dataSource)
|
||||
.bind("start", interval.getStart().toString())
|
||||
.bind("end", interval.getEnd().toString());
|
||||
|
||||
for (int i = 0; i < sequenceNamePrefixes.size(); i++) {
|
||||
query.bind(StringUtils.format("prefix%d", i), sequenceNamePrefixes.get(i) + "%");
|
||||
}
|
||||
|
||||
final ResultIterator<PendingSegmentsRecord> dbSegments =
|
||||
query.map((index, r, ctx) -> PendingSegmentsRecord.fromResultSet(r))
|
||||
.iterator();
|
||||
|
||||
final Map<SegmentIdWithShardSpec, String> pendingSegmentToSequenceName = new HashMap<>();
|
||||
while (dbSegments.hasNext()) {
|
||||
PendingSegmentsRecord record = dbSegments.next();
|
||||
final SegmentIdWithShardSpec identifier = jsonMapper.readValue(record.payload, SegmentIdWithShardSpec.class);
|
||||
|
||||
if (interval.overlaps(identifier.getInterval())) {
|
||||
pendingSegmentToSequenceName.put(identifier, record.sequenceName);
|
||||
}
|
||||
}
|
||||
|
||||
dbSegments.close();
|
||||
|
||||
return pendingSegmentToSequenceName;
|
||||
}
|
||||
|
||||
private Map<SegmentIdWithShardSpec, String> getPendingSegmentsForIntervalWithHandle(
|
||||
final Handle handle,
|
||||
final String dataSource,
|
||||
|
@ -620,7 +673,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
|
||||
@Override
|
||||
public Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> upgradePendingSegmentsOverlappingWith(
|
||||
Set<DataSegment> replaceSegments
|
||||
Set<DataSegment> replaceSegments,
|
||||
Set<String> activeRealtimeSequencePrefixes
|
||||
)
|
||||
{
|
||||
if (replaceSegments.isEmpty()) {
|
||||
|
@ -639,7 +693,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
|
||||
final String datasource = replaceSegments.iterator().next().getDataSource();
|
||||
return connector.retryWithHandle(
|
||||
handle -> upgradePendingSegments(handle, datasource, replaceIntervalToMaxId)
|
||||
handle -> upgradePendingSegments(handle, datasource, replaceIntervalToMaxId, activeRealtimeSequencePrefixes)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -658,7 +712,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
private Map<SegmentIdWithShardSpec, SegmentIdWithShardSpec> upgradePendingSegments(
|
||||
Handle handle,
|
||||
String datasource,
|
||||
Map<Interval, DataSegment> replaceIntervalToMaxId
|
||||
Map<Interval, DataSegment> replaceIntervalToMaxId,
|
||||
Set<String> activeRealtimeSequencePrefixes
|
||||
) throws IOException
|
||||
{
|
||||
final Map<SegmentCreateRequest, SegmentIdWithShardSpec> newPendingSegmentVersions = new HashMap<>();
|
||||
|
@ -673,12 +728,13 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
int currentPartitionNumber = maxSegmentId.getShardSpec().getPartitionNum();
|
||||
|
||||
final Map<SegmentIdWithShardSpec, String> overlappingPendingSegments
|
||||
= getPendingSegmentsForIntervalWithHandle(handle, datasource, replaceInterval);
|
||||
= getPendingSegmentsForIntervalWithHandle(handle, datasource, replaceInterval, activeRealtimeSequencePrefixes);
|
||||
|
||||
for (Map.Entry<SegmentIdWithShardSpec, String> overlappingPendingSegment
|
||||
: overlappingPendingSegments.entrySet()) {
|
||||
final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getKey();
|
||||
final String pendingSegmentSequence = overlappingPendingSegment.getValue();
|
||||
|
||||
if (shouldUpgradePendingSegment(pendingSegmentId, pendingSegmentSequence, replaceInterval, replaceVersion)) {
|
||||
// Ensure unique sequence_name_prev_id_sha1 by setting
|
||||
// sequence_prev_id -> pendingSegmentId
|
||||
|
@ -1281,9 +1337,9 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
final Map<Interval, Set<DataSegment>> overlappingIntervalToSegments = new HashMap<>();
|
||||
for (DataSegment segment : overlappingSegments) {
|
||||
overlappingVersionToIntervals.computeIfAbsent(segment.getVersion(), v -> new HashSet<>())
|
||||
.add(segment.getInterval());
|
||||
.add(segment.getInterval());
|
||||
overlappingIntervalToSegments.computeIfAbsent(segment.getInterval(), i -> new HashSet<>())
|
||||
.add(segment);
|
||||
.add(segment);
|
||||
}
|
||||
|
||||
final Set<DataSegment> upgradedSegments = new HashSet<>();
|
||||
|
@ -2275,7 +2331,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
|
|||
// Not in the desired start state.
|
||||
return new DataStoreMetadataUpdateResult(true, false, StringUtils.format(
|
||||
"Inconsistent metadata state. This can happen if you update input topic in a spec without changing " +
|
||||
"the supervisor name. Stored state: [%s], Target state: [%s].",
|
||||
"the supervisor name. Stored state: [%s], Target state: [%s].",
|
||||
oldCommitMetadataFromDb,
|
||||
startMetadata
|
||||
));
|
||||
|
|
|
@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableList;
|
|||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.hash.Hashing;
|
||||
import com.google.common.io.BaseEncoding;
|
||||
import org.apache.druid.data.input.StringTuple;
|
||||
import org.apache.druid.indexing.overlord.DataSourceMetadata;
|
||||
import org.apache.druid.indexing.overlord.ObjectMetadata;
|
||||
|
@ -464,6 +466,44 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
|||
);
|
||||
}
|
||||
|
||||
private Boolean insertPendingSegmentAndSequenceName(Pair<SegmentIdWithShardSpec, String> pendingSegmentSequenceName)
|
||||
{
|
||||
final SegmentIdWithShardSpec pendingSegment = pendingSegmentSequenceName.lhs;
|
||||
final String sequenceName = pendingSegmentSequenceName.rhs;
|
||||
final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getPendingSegmentsTable();
|
||||
return derbyConnector.retryWithHandle(
|
||||
handle -> {
|
||||
handle.createStatement(
|
||||
StringUtils.format(
|
||||
"INSERT INTO %1$s (id, dataSource, created_date, start, %2$send%2$s, sequence_name, sequence_prev_id, "
|
||||
+ "sequence_name_prev_id_sha1, payload) "
|
||||
+ "VALUES (:id, :dataSource, :created_date, :start, :end, :sequence_name, :sequence_prev_id, "
|
||||
+ ":sequence_name_prev_id_sha1, :payload)",
|
||||
table,
|
||||
derbyConnector.getQuoteString()
|
||||
)
|
||||
)
|
||||
.bind("id", pendingSegment.toString())
|
||||
.bind("dataSource", pendingSegment.getDataSource())
|
||||
.bind("created_date", DateTimes.nowUtc().toString())
|
||||
.bind("start", pendingSegment.getInterval().getStart().toString())
|
||||
.bind("end", pendingSegment.getInterval().getEnd().toString())
|
||||
.bind("sequence_name", sequenceName)
|
||||
.bind("sequence_prev_id", pendingSegment.toString())
|
||||
.bind("sequence_name_prev_id_sha1", BaseEncoding.base16().encode(
|
||||
Hashing.sha1()
|
||||
.newHasher()
|
||||
.putLong((long) pendingSegment.hashCode() * sequenceName.hashCode())
|
||||
.hash()
|
||||
.asBytes()
|
||||
))
|
||||
.bind("payload", mapper.writeValueAsBytes(pendingSegment))
|
||||
.execute();
|
||||
return true;
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
private Map<String, String> getSegmentsCommittedDuringReplaceTask(String taskId)
|
||||
{
|
||||
final String table = derbyConnectorRule.metadataTablesConfigSupplier().get().getUpgradeSegmentsTable();
|
||||
|
@ -2554,6 +2594,64 @@ public class IndexerSQLMetadataStorageCoordinatorTest
|
|||
);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetPendingSegmentsForIntervalWithSequencePrefixes()
|
||||
{
|
||||
Pair<SegmentIdWithShardSpec, String> validIntervalValidSequence = Pair.of(
|
||||
SegmentIdWithShardSpec.fromDataSegment(defaultSegment),
|
||||
"validLOL"
|
||||
);
|
||||
insertPendingSegmentAndSequenceName(validIntervalValidSequence);
|
||||
|
||||
Pair<SegmentIdWithShardSpec, String> validIntervalInvalidSequence = Pair.of(
|
||||
SegmentIdWithShardSpec.fromDataSegment(defaultSegment2),
|
||||
"invalidRandom"
|
||||
);
|
||||
insertPendingSegmentAndSequenceName(validIntervalInvalidSequence);
|
||||
|
||||
Pair<SegmentIdWithShardSpec, String> invalidIntervalvalidSequence = Pair.of(
|
||||
SegmentIdWithShardSpec.fromDataSegment(existingSegment1),
|
||||
"validStuff"
|
||||
);
|
||||
insertPendingSegmentAndSequenceName(invalidIntervalvalidSequence);
|
||||
|
||||
Pair<SegmentIdWithShardSpec, String> twentyFifteenWithAnotherValidSequence = Pair.of(
|
||||
new SegmentIdWithShardSpec(
|
||||
existingSegment1.getDataSource(),
|
||||
Intervals.of("2015/2016"),
|
||||
"1970-01-01",
|
||||
new NumberedShardSpec(1, 0)
|
||||
),
|
||||
"alsoValidAgain"
|
||||
);
|
||||
insertPendingSegmentAndSequenceName(twentyFifteenWithAnotherValidSequence);
|
||||
|
||||
Pair<SegmentIdWithShardSpec, String> twentyFifteenWithInvalidSequence = Pair.of(
|
||||
new SegmentIdWithShardSpec(
|
||||
existingSegment1.getDataSource(),
|
||||
Intervals.of("2015/2016"),
|
||||
"1970-01-01",
|
||||
new NumberedShardSpec(2, 0)
|
||||
),
|
||||
"definitelyInvalid"
|
||||
);
|
||||
insertPendingSegmentAndSequenceName(twentyFifteenWithInvalidSequence);
|
||||
|
||||
|
||||
final Map<SegmentIdWithShardSpec, String> expected = new HashMap<>();
|
||||
expected.put(validIntervalValidSequence.lhs, validIntervalValidSequence.rhs);
|
||||
expected.put(twentyFifteenWithAnotherValidSequence.lhs, twentyFifteenWithAnotherValidSequence.rhs);
|
||||
|
||||
final Map<SegmentIdWithShardSpec, String> actual =
|
||||
derbyConnector.retryWithHandle(handle -> coordinator.getPendingSegmentsForIntervalWithHandle(
|
||||
handle,
|
||||
defaultSegment.getDataSource(),
|
||||
defaultSegment.getInterval(),
|
||||
ImmutableSet.of("valid", "alsoValid")
|
||||
));
|
||||
Assert.assertEquals(expected, actual);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRetrieveUsedSegmentsAndCreatedDates()
|
||||
{
|
||||
|
|
Loading…
Reference in New Issue