Refactor: Minor cleanup of segment allocation flow (#17524)

Changes
--------
- Simplify the arguments of IndexerMetadataStorageCoordinator.allocatePendingSegment
- Remove field SegmentCreateRequest.upgradedFromSegmentId as it was always null
- Miscellaneous cleanup
This commit is contained in:
Kashif Faraz 2024-12-12 18:16:57 -08:00 committed by GitHub
parent b86ea4d5c4
commit 24e5d8a9e8
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
10 changed files with 213 additions and 271 deletions

View File

@ -183,7 +183,7 @@ public class TaskLockbox
? savedTaskLock.withPriority(task.getPriority()) ? savedTaskLock.withPriority(task.getPriority())
: savedTaskLock; : savedTaskLock;
final TaskLockPosse taskLockPosse = verifyAndCreateOrFindLockPosse( final TaskLockPosse taskLockPosse = reacquireLockOnStartup(
task, task,
savedTaskLockWithPriority savedTaskLockWithPriority
); );
@ -192,15 +192,11 @@ public class TaskLockbox
if (savedTaskLockWithPriority.getVersion().equals(taskLock.getVersion())) { if (savedTaskLockWithPriority.getVersion().equals(taskLock.getVersion())) {
taskLockCount++; taskLockCount++;
log.info( log.info("Reacquired lock[%s] for task[%s].", taskLock, task.getId());
"Reacquired lock[%s] for task: %s",
taskLock,
task.getId()
);
} else { } else {
taskLockCount++; taskLockCount++;
log.info( log.info(
"Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task: %s", "Could not reacquire lock on interval[%s] version[%s] (got version[%s] instead) for task[%s].",
savedTaskLockWithPriority.getInterval(), savedTaskLockWithPriority.getInterval(),
savedTaskLockWithPriority.getVersion(), savedTaskLockWithPriority.getVersion(),
taskLock.getVersion(), taskLock.getVersion(),
@ -210,7 +206,7 @@ public class TaskLockbox
} else { } else {
failedToReacquireLockTaskGroups.add(task.getGroupId()); failedToReacquireLockTaskGroups.add(task.getGroupId());
log.error( log.error(
"Could not reacquire lock on interval[%s] version[%s] for task: %s from group %s.", "Could not reacquire lock on interval[%s] version[%s] for task[%s], groupId[%s].",
savedTaskLockWithPriority.getInterval(), savedTaskLockWithPriority.getInterval(),
savedTaskLockWithPriority.getVersion(), savedTaskLockWithPriority.getVersion(),
task.getId(), task.getId(),
@ -253,38 +249,28 @@ public class TaskLockbox
} }
/** /**
* This method is called only in {@link #syncFromStorage()} and verifies the given task and the taskLock have the same * Reacquire lock during {@link #syncFromStorage()}.
* groupId, dataSource, and priority. *
* @return null if the lock could not be reacquired.
*/ */
@VisibleForTesting @VisibleForTesting
@Nullable @Nullable
protected TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) protected TaskLockPosse reacquireLockOnStartup(Task task, TaskLock taskLock)
{ {
if (!taskMatchesLock(task, taskLock)) {
log.warn(
"Task[datasource: %s, groupId: %s, priority: %s] does not match"
+ " TaskLock[datasource: %s, groupId: %s, priority: %s].",
task.getDataSource(), task.getGroupId(), task.getPriority(),
taskLock.getDataSource(), taskLock.getGroupId(), taskLock.getNonNullPriority()
);
return null;
}
giant.lock(); giant.lock();
try { try {
Preconditions.checkArgument(
task.getGroupId().equals(taskLock.getGroupId()),
"lock groupId[%s] is different from task groupId[%s]",
taskLock.getGroupId(),
task.getGroupId()
);
Preconditions.checkArgument(
task.getDataSource().equals(taskLock.getDataSource()),
"lock dataSource[%s] is different from task dataSource[%s]",
taskLock.getDataSource(),
task.getDataSource()
);
final int taskPriority = task.getPriority(); final int taskPriority = task.getPriority();
final int lockPriority = taskLock.getNonNullPriority();
Preconditions.checkArgument(
lockPriority == taskPriority,
"lock priority[%s] is different from task priority[%s]",
lockPriority,
taskPriority
);
final LockRequest request; final LockRequest request;
switch (taskLock.getGranularity()) { switch (taskLock.getGranularity()) {
case SEGMENT: case SEGMENT:
@ -313,15 +299,13 @@ public class TaskLockbox
); );
break; break;
default: default:
throw new ISE("Unknown lockGranularity[%s]", taskLock.getGranularity()); throw DruidException.defensive("Unknown lockGranularity[%s]", taskLock.getGranularity());
} }
return createOrFindLockPosse(request, task, false); return createOrFindLockPosse(request, task, false);
} }
catch (Exception e) { catch (Exception e) {
log.error(e, log.error(e, "Could not reacquire lock for task[%s] from metadata store", task.getId());
"Could not reacquire lock for task: %s from metadata store", task.getId()
);
return null; return null;
} }
finally { finally {
@ -329,6 +313,17 @@ public class TaskLockbox
} }
} }
/**
* Returns true if the datasource, groupId and priority of the given Task
* match that of the TaskLock.
*/
private boolean taskMatchesLock(Task task, TaskLock taskLock)
{
return task.getGroupId().equals(taskLock.getGroupId())
&& task.getDataSource().equals(taskLock.getDataSource())
&& task.getPriority() == taskLock.getNonNullPriority();
}
/** /**
* Acquires a lock on behalf of a task. Blocks until the lock is acquired. * Acquires a lock on behalf of a task. Blocks until the lock is acquired.
* *
@ -751,13 +746,15 @@ public class TaskLockbox
{ {
return metadataStorageCoordinator.allocatePendingSegment( return metadataStorageCoordinator.allocatePendingSegment(
request.getDataSource(), request.getDataSource(),
request.getSequenceName(),
request.getPreviousSegmentId(),
request.getInterval(), request.getInterval(),
request.getPartialShardSpec(),
version,
request.isSkipSegmentLineageCheck(), request.isSkipSegmentLineageCheck(),
allocatorId new SegmentCreateRequest(
request.getSequenceName(),
request.getPreviousSegmentId(),
version,
request.getPartialShardSpec(),
allocatorId
)
); );
} }
@ -1818,7 +1815,6 @@ public class TaskLockbox
action.getPreviousSegmentId(), action.getPreviousSegmentId(),
acquiredLock == null ? lockRequest.getVersion() : acquiredLock.getVersion(), acquiredLock == null ? lockRequest.getVersion() : acquiredLock.getVersion(),
action.getPartialShardSpec(), action.getPartialShardSpec(),
null,
((PendingSegmentAllocatingTask) task).getTaskAllocatorId() ((PendingSegmentAllocatingTask) task).getTaskAllocatorId()
); );
} }

View File

@ -2270,10 +2270,10 @@ public class TaskLockboxTest
} }
@Override @Override
protected TaskLockPosse verifyAndCreateOrFindLockPosse(Task task, TaskLock taskLock) protected TaskLockPosse reacquireLockOnStartup(Task task, TaskLock taskLock)
{ {
return task.getGroupId() return task.getGroupId()
.contains("FailingLockAcquisition") ? null : super.verifyAndCreateOrFindLockPosse(task, taskLock); .contains("FailingLockAcquisition") ? null : super.reacquireLockOnStartup(task, taskLock);
} }
} }
} }

View File

@ -36,7 +36,6 @@ import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -242,20 +241,16 @@ public class TestIndexerMetadataStorageCoordinator implements IndexerMetadataSto
@Override @Override
public SegmentIdWithShardSpec allocatePendingSegment( public SegmentIdWithShardSpec allocatePendingSegment(
String dataSource, String dataSource,
String sequenceName,
String previousSegmentId,
Interval interval, Interval interval,
PartialShardSpec partialShardSpec,
String maxVersion,
boolean skipSegmentLineageCheck, boolean skipSegmentLineageCheck,
String taskAllocatorId SegmentCreateRequest createRequest
) )
{ {
return new SegmentIdWithShardSpec( return new SegmentIdWithShardSpec(
dataSource, dataSource,
interval, interval,
maxVersion, createRequest.getVersion(),
partialShardSpec.complete(objectMapper, 0, 0) createRequest.getPartialShardSpec().complete(objectMapper, 0, 0)
); );
} }

View File

@ -21,6 +21,7 @@ package org.apache.druid.timeline.partition;
/** /**
* ShardSpec for non-first-generation segments. * ShardSpec for non-first-generation segments.
* This shardSpec is created only by overwriting tasks using segment locks.
* This shardSpec is allocated a partitionId between {@link PartitionIds#NON_ROOT_GEN_START_PARTITION_ID} and * This shardSpec is allocated a partitionId between {@link PartitionIds#NON_ROOT_GEN_START_PARTITION_ID} and
* {@link PartitionIds#NON_ROOT_GEN_END_PARTITION_ID}. * {@link PartitionIds#NON_ROOT_GEN_END_PARTITION_ID}.
* *

View File

@ -31,6 +31,7 @@ public final class PartitionIds
public static final int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive public static final int ROOT_GEN_END_PARTITION_ID = 32768; // exclusive
/** /**
* Start partitionId available for non-root generation segments. * Start partitionId available for non-root generation segments.
* Used only with segment locks.
*/ */
public static final int NON_ROOT_GEN_START_PARTITION_ID = 32768; public static final int NON_ROOT_GEN_START_PARTITION_ID = 32768;
/** /**

View File

@ -26,7 +26,6 @@ import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec; import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline; import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.DateTime; import org.joda.time.DateTime;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -215,34 +214,23 @@ public interface IndexerMetadataStorageCoordinator
* Note that a segment sequence may include segments with a variety of different intervals and versions. * Note that a segment sequence may include segments with a variety of different intervals and versions.
* *
* @param dataSource dataSource for which to allocate a segment * @param dataSource dataSource for which to allocate a segment
* @param sequenceName name of the group of ingestion tasks producing a segment series
* @param previousSegmentId previous segment in the series; may be null or empty, meaning this is the first
* segment
* @param interval interval for which to allocate a segment * @param interval interval for which to allocate a segment
* @param partialShardSpec partialShardSpec containing all necessary information to create a shardSpec for the
* new segmentId
* @param maxVersion use this version if we have no better version to use. The returned segment
* identifier may have a version lower than this one, but will not have one higher.
* @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence. * @param skipSegmentLineageCheck if true, perform lineage validation using previousSegmentId for this sequence.
* Should be set to false if replica tasks would index events in same order * Should be set to false if replica tasks would index events in same order
* @param taskAllocatorId The task allocator id with which the pending segment is associated
* @return the pending segment identifier, or null if it was impossible to allocate a new segment * @return the pending segment identifier, or null if it was impossible to allocate a new segment
*/ */
@Nullable
SegmentIdWithShardSpec allocatePendingSegment( SegmentIdWithShardSpec allocatePendingSegment(
String dataSource, String dataSource,
String sequenceName,
@Nullable String previousSegmentId,
Interval interval, Interval interval,
PartialShardSpec partialShardSpec,
String maxVersion,
boolean skipSegmentLineageCheck, boolean skipSegmentLineageCheck,
String taskAllocatorId SegmentCreateRequest createRequest
); );
/** /**
* Delete pending segments created in the given interval belonging to the given data source from the pending segments * Delete pending segments created in the given interval belonging to the given data source from the pending segments
* table. The {@code created_date} field of the pending segments table is checked to find segments to be deleted. * table. The {@code created_date} field of the pending segments table is checked to find segments to be deleted.
* * <p>
* Note that the semantic of the interval (for `created_date`s) is different from the semantic of the interval * Note that the semantic of the interval (for `created_date`s) is different from the semantic of the interval
* parameters in some other methods in this class, such as {@link #retrieveUsedSegmentsForInterval} (where the * parameters in some other methods in this class, such as {@link #retrieveUsedSegmentsForInterval} (where the
* interval is about the time column value in rows belonging to the segment). * interval is about the time column value in rows belonging to the segment).
@ -269,7 +257,7 @@ public interface IndexerMetadataStorageCoordinator
* <p/> * <p/>
* If startMetadata and endMetadata are set, this insertion will be atomic with a compare-and-swap on dataSource * If startMetadata and endMetadata are set, this insertion will be atomic with a compare-and-swap on dataSource
* commit metadata. * commit metadata.
* * <p>
* If segmentsToDrop is not null and not empty, this insertion will be atomic with a insert-and-drop on inserting * If segmentsToDrop is not null and not empty, this insertion will be atomic with a insert-and-drop on inserting
* {@param segments} and dropping {@param segmentsToDrop}. * {@param segments} and dropping {@param segmentsToDrop}.
* *
@ -426,7 +414,7 @@ public interface IndexerMetadataStorageCoordinator
* Similar to {@link #commitSegments}, but meant for streaming ingestion tasks for handling * Similar to {@link #commitSegments}, but meant for streaming ingestion tasks for handling
* the case where the task ingested no records and created no segments, but still needs to update the metadata * the case where the task ingested no records and created no segments, but still needs to update the metadata
* with the progress that the task made. * with the progress that the task made.
* * <p>
* The metadata should undergo the same validation checks as performed by {@link #commitSegments}. * The metadata should undergo the same validation checks as performed by {@link #commitSegments}.
* *
* *

View File

@ -38,7 +38,6 @@ public class SegmentCreateRequest
private final String sequenceName; private final String sequenceName;
private final String previousSegmentId; private final String previousSegmentId;
private final PartialShardSpec partialShardSpec; private final PartialShardSpec partialShardSpec;
private final String upgradedFromSegmentId;
private final String taskAllocatorId; private final String taskAllocatorId;
public SegmentCreateRequest( public SegmentCreateRequest(
@ -46,7 +45,6 @@ public class SegmentCreateRequest
String previousSegmentId, String previousSegmentId,
String version, String version,
PartialShardSpec partialShardSpec, PartialShardSpec partialShardSpec,
String upgradedFromSegmentId,
String taskAllocatorId String taskAllocatorId
) )
{ {
@ -54,24 +52,31 @@ public class SegmentCreateRequest
this.previousSegmentId = previousSegmentId == null ? "" : previousSegmentId; this.previousSegmentId = previousSegmentId == null ? "" : previousSegmentId;
this.version = version; this.version = version;
this.partialShardSpec = partialShardSpec; this.partialShardSpec = partialShardSpec;
this.upgradedFromSegmentId = upgradedFromSegmentId;
this.taskAllocatorId = taskAllocatorId; this.taskAllocatorId = taskAllocatorId;
} }
/**
* Represents group of ingestion tasks that produce a segment series.
*/
public String getSequenceName() public String getSequenceName()
{ {
return sequenceName; return sequenceName;
} }
/** /**
* Non-null previous segment id. This can be used for persisting to the * Previous segment id allocated for this sequence.
* pending segments table in the metadata store. *
* @return Empty string if there is no previous segment in the series.
*/ */
public String getPreviousSegmentId() public String getPreviousSegmentId()
{ {
return previousSegmentId; return previousSegmentId;
} }
/**
* Version of the lock held by the task that has requested the segment allocation.
* The allocated segment must have a version less than or equal to this version.
*/
public String getVersion() public String getVersion()
{ {
return version; return version;
@ -82,11 +87,6 @@ public class SegmentCreateRequest
return partialShardSpec; return partialShardSpec;
} }
public String getUpgradedFromSegmentId()
{
return upgradedFromSegmentId;
}
public String getTaskAllocatorId() public String getTaskAllocatorId()
{ {
return taskAllocatorId; return taskAllocatorId;

View File

@ -80,7 +80,6 @@ import org.skife.jdbi.v2.TransactionCallback;
import org.skife.jdbi.v2.TransactionStatus; import org.skife.jdbi.v2.TransactionStatus;
import org.skife.jdbi.v2.Update; import org.skife.jdbi.v2.Update;
import org.skife.jdbi.v2.exceptions.CallbackFailedException; import org.skife.jdbi.v2.exceptions.CallbackFailedException;
import org.skife.jdbi.v2.tweak.HandleCallback;
import org.skife.jdbi.v2.util.ByteArrayMapper; import org.skife.jdbi.v2.util.ByteArrayMapper;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -350,8 +349,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
/** /**
* Fetches all the pending segments, whose interval overlaps with the given search interval, from the metadata store. * Fetches all the pending segments, whose interval overlaps with the given search interval, from the metadata store.
*/ */
@VisibleForTesting private List<PendingSegmentRecord> getPendingSegmentsForInterval(
List<PendingSegmentRecord> getPendingSegmentsForIntervalWithHandle(
final Handle handle, final Handle handle,
final String dataSource, final String dataSource,
final Interval interval final Interval interval
@ -390,7 +388,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
return pendingSegments.build(); return pendingSegments.build();
} }
List<PendingSegmentRecord> getPendingSegmentsForTaskAllocatorIdWithHandle( private List<PendingSegmentRecord> getPendingSegmentsForTaskAllocatorId(
final Handle handle, final Handle handle,
final String dataSource, final String dataSource,
final String taskAllocatorId final String taskAllocatorId
@ -580,7 +578,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
); );
} }
} }
SegmentPublishResult result = SegmentPublishResult.ok( return SegmentPublishResult.ok(
insertSegments( insertSegments(
handle, handle,
segmentsToInsert, segmentsToInsert,
@ -591,7 +589,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
), ),
upgradePendingSegmentsOverlappingWith(segmentsToInsert) upgradePendingSegmentsOverlappingWith(segmentsToInsert)
); );
return result;
}, },
3, 3,
getSqlMetadataMaxRetry() getSqlMetadataMaxRetry()
@ -740,21 +737,16 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
} }
@Override @Override
@Nullable
public SegmentIdWithShardSpec allocatePendingSegment( public SegmentIdWithShardSpec allocatePendingSegment(
final String dataSource, final String dataSource,
final String sequenceName,
@Nullable final String previousSegmentId,
final Interval interval, final Interval interval,
final PartialShardSpec partialShardSpec,
final String maxVersion,
final boolean skipSegmentLineageCheck, final boolean skipSegmentLineageCheck,
String taskAllocatorId final SegmentCreateRequest createRequest
) )
{ {
Preconditions.checkNotNull(dataSource, "dataSource"); Preconditions.checkNotNull(dataSource, "dataSource");
Preconditions.checkNotNull(sequenceName, "sequenceName");
Preconditions.checkNotNull(interval, "interval"); Preconditions.checkNotNull(interval, "interval");
Preconditions.checkNotNull(maxVersion, "version");
final Interval allocateInterval = interval.withChronology(ISOChronology.getInstanceUTC()); final Interval allocateInterval = interval.withChronology(ISOChronology.getInstanceUTC());
return connector.retryWithHandle( return connector.retryWithHandle(
@ -776,24 +768,17 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
return allocatePendingSegment( return allocatePendingSegment(
handle, handle,
dataSource, dataSource,
sequenceName,
allocateInterval, allocateInterval,
partialShardSpec, createRequest,
maxVersion, existingChunks
existingChunks,
taskAllocatorId
); );
} else { } else {
return allocatePendingSegmentWithSegmentLineageCheck( return allocatePendingSegmentWithSegmentLineageCheck(
handle, handle,
dataSource, dataSource,
sequenceName,
previousSegmentId,
allocateInterval, allocateInterval,
partialShardSpec, createRequest,
maxVersion, existingChunks
existingChunks,
taskAllocatorId
); );
} }
} }
@ -854,7 +839,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
int currentPartitionNumber = maxSegmentId.getShardSpec().getPartitionNum(); int currentPartitionNumber = maxSegmentId.getShardSpec().getPartitionNum();
final List<PendingSegmentRecord> overlappingPendingSegments final List<PendingSegmentRecord> overlappingPendingSegments
= getPendingSegmentsForIntervalWithHandle(handle, datasource, replaceInterval); = getPendingSegmentsForInterval(handle, datasource, replaceInterval);
for (PendingSegmentRecord overlappingPendingSegment : overlappingPendingSegments) { for (PendingSegmentRecord overlappingPendingSegment : overlappingPendingSegments) {
final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getId(); final SegmentIdWithShardSpec pendingSegmentId = overlappingPendingSegment.getId();
@ -929,17 +914,11 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck( private SegmentIdWithShardSpec allocatePendingSegmentWithSegmentLineageCheck(
final Handle handle, final Handle handle,
final String dataSource, final String dataSource,
final String sequenceName,
@Nullable final String previousSegmentId,
final Interval interval, final Interval interval,
final PartialShardSpec partialShardSpec, final SegmentCreateRequest createRequest,
final String maxVersion, final List<TimelineObjectHolder<String, DataSegment>> existingChunks
final List<TimelineObjectHolder<String, DataSegment>> existingChunks,
final String taskAllocatorId
) throws IOException ) throws IOException
{ {
final String previousSegmentIdNotNull = previousSegmentId == null ? "" : previousSegmentId;
final String sql = StringUtils.format( final String sql = StringUtils.format(
"SELECT payload FROM %s WHERE " "SELECT payload FROM %s WHERE "
+ "dataSource = :dataSource AND " + "dataSource = :dataSource AND "
@ -950,15 +929,15 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final Query<Map<String, Object>> query final Query<Map<String, Object>> query
= handle.createQuery(sql) = handle.createQuery(sql)
.bind("dataSource", dataSource) .bind("dataSource", dataSource)
.bind("sequence_name", sequenceName) .bind("sequence_name", createRequest.getSequenceName())
.bind("sequence_prev_id", previousSegmentIdNotNull); .bind("sequence_prev_id", createRequest.getPreviousSegmentId());
final String usedSegmentVersion = existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion(); final String usedSegmentVersion = existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion();
final CheckExistingSegmentIdResult result = findExistingPendingSegment( final CheckExistingSegmentIdResult result = findExistingPendingSegment(
query, query,
interval, interval,
sequenceName, createRequest.getSequenceName(),
previousSegmentIdNotNull, createRequest.getPreviousSegmentId(),
usedSegmentVersion usedSegmentVersion
); );
@ -967,12 +946,12 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
return result.segmentIdentifier; return result.segmentIdentifier;
} }
final SegmentIdWithShardSpec newIdentifier = createNewSegment( final SegmentIdWithShardSpec newIdentifier = createNewPendingSegment(
handle, handle,
dataSource, dataSource,
interval, interval,
partialShardSpec, createRequest.getPartialShardSpec(),
maxVersion, createRequest.getVersion(),
existingChunks existingChunks
); );
if (newIdentifier == null) { if (newIdentifier == null) {
@ -989,9 +968,9 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode( final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode(
Hashing.sha1() Hashing.sha1()
.newHasher() .newHasher()
.putBytes(StringUtils.toUtf8(sequenceName)) .putBytes(StringUtils.toUtf8(createRequest.getSequenceName()))
.putByte((byte) 0xff) .putByte((byte) 0xff)
.putBytes(StringUtils.toUtf8(previousSegmentIdNotNull)) .putBytes(StringUtils.toUtf8(createRequest.getPreviousSegmentId()))
.putByte((byte) 0xff) .putByte((byte) 0xff)
.putBytes(StringUtils.toUtf8(newIdentifier.getVersion())) .putBytes(StringUtils.toUtf8(newIdentifier.getVersion()))
.hash() .hash()
@ -1003,10 +982,10 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
newIdentifier, newIdentifier,
dataSource, dataSource,
interval, interval,
previousSegmentIdNotNull, createRequest.getPreviousSegmentId(),
sequenceName, createRequest.getSequenceName(),
sequenceNamePrevIdSha1, sequenceNamePrevIdSha1,
taskAllocatorId createRequest.getTaskAllocatorId()
); );
return newIdentifier; return newIdentifier;
} }
@ -1108,12 +1087,9 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
private SegmentIdWithShardSpec allocatePendingSegment( private SegmentIdWithShardSpec allocatePendingSegment(
final Handle handle, final Handle handle,
final String dataSource, final String dataSource,
final String sequenceName,
final Interval interval, final Interval interval,
final PartialShardSpec partialShardSpec, final SegmentCreateRequest createRequest,
final String maxVersion, final List<TimelineObjectHolder<String, DataSegment>> existingChunks
final List<TimelineObjectHolder<String, DataSegment>> existingChunks,
final String taskAllocatorId
) throws IOException ) throws IOException
{ {
final String sql = StringUtils.format( final String sql = StringUtils.format(
@ -1128,14 +1104,14 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final Query<Map<String, Object>> query final Query<Map<String, Object>> query
= handle.createQuery(sql) = handle.createQuery(sql)
.bind("dataSource", dataSource) .bind("dataSource", dataSource)
.bind("sequence_name", sequenceName) .bind("sequence_name", createRequest.getSequenceName())
.bind("start", interval.getStart().toString()) .bind("start", interval.getStart().toString())
.bind("end", interval.getEnd().toString()); .bind("end", interval.getEnd().toString());
final CheckExistingSegmentIdResult result = findExistingPendingSegment( final CheckExistingSegmentIdResult result = findExistingPendingSegment(
query, query,
interval, interval,
sequenceName, createRequest.getSequenceName(),
null, null,
existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion() existingChunks.isEmpty() ? null : existingChunks.get(0).getVersion()
); );
@ -1144,12 +1120,12 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
return result.segmentIdentifier; return result.segmentIdentifier;
} }
final SegmentIdWithShardSpec newIdentifier = createNewSegment( final SegmentIdWithShardSpec newIdentifier = createNewPendingSegment(
handle, handle,
dataSource, dataSource,
interval, interval,
partialShardSpec, createRequest.getPartialShardSpec(),
maxVersion, createRequest.getVersion(),
existingChunks existingChunks
); );
if (newIdentifier == null) { if (newIdentifier == null) {
@ -1166,7 +1142,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode( final String sequenceNamePrevIdSha1 = BaseEncoding.base16().encode(
Hashing.sha1() Hashing.sha1()
.newHasher() .newHasher()
.putBytes(StringUtils.toUtf8(sequenceName)) .putBytes(StringUtils.toUtf8(createRequest.getSequenceName()))
.putByte((byte) 0xff) .putByte((byte) 0xff)
.putLong(interval.getStartMillis()) .putLong(interval.getStartMillis())
.putLong(interval.getEndMillis()) .putLong(interval.getEndMillis())
@ -1183,14 +1159,14 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
dataSource, dataSource,
interval, interval,
"", "",
sequenceName, createRequest.getSequenceName(),
sequenceNamePrevIdSha1, sequenceNamePrevIdSha1,
taskAllocatorId createRequest.getTaskAllocatorId()
); );
log.info( log.info(
"Created new pending segment[%s] for datasource[%s], sequence[%s], interval[%s].", "Created new pending segment[%s] for datasource[%s], interval[%s].",
newIdentifier, dataSource, sequenceName, interval newIdentifier, dataSource, interval
); );
return newIdentifier; return newIdentifier;
@ -1334,7 +1310,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
private static class CheckExistingSegmentIdResult private static class CheckExistingSegmentIdResult
{ {
private final boolean found; private final boolean found;
@Nullable
private final SegmentIdWithShardSpec segmentIdentifier; private final SegmentIdWithShardSpec segmentIdentifier;
CheckExistingSegmentIdResult(boolean found, @Nullable SegmentIdWithShardSpec segmentIdentifier) CheckExistingSegmentIdResult(boolean found, @Nullable SegmentIdWithShardSpec segmentIdentifier)
@ -1391,21 +1366,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
} }
} }
private static void bindColumnValuesToQueryWithInCondition(
final String columnName,
final List<String> values,
final Update query
)
{
if (values == null) {
return;
}
for (int i = 0; i < values.size(); i++) {
query.bind(StringUtils.format("%s%d", columnName, i), values.get(i));
}
}
private int deletePendingSegmentsById(Handle handle, String datasource, List<String> pendingSegmentIds) private int deletePendingSegmentsById(Handle handle, String datasource, List<String> pendingSegmentIds)
{ {
if (pendingSegmentIds.isEmpty()) { if (pendingSegmentIds.isEmpty()) {
@ -1419,7 +1379,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", pendingSegmentIds) SqlSegmentsMetadataQuery.getParameterizedInConditionForColumn("id", pendingSegmentIds)
) )
).bind("dataSource", datasource); ).bind("dataSource", datasource);
bindColumnValuesToQueryWithInCondition("id", pendingSegmentIds, query); SqlSegmentsMetadataQuery.bindColumnValuesToQueryWithInCondition("id", pendingSegmentIds, query);
return query.execute(); return query.execute();
} }
@ -1442,7 +1402,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
final String dataSource = appendSegments.iterator().next().getDataSource(); final String dataSource = appendSegments.iterator().next().getDataSource();
final List<PendingSegmentRecord> segmentIdsForNewVersions = connector.retryTransaction( final List<PendingSegmentRecord> segmentIdsForNewVersions = connector.retryTransaction(
(handle, transactionStatus) (handle, transactionStatus)
-> getPendingSegmentsForTaskAllocatorIdWithHandle(handle, dataSource, taskAllocatorId), -> getPendingSegmentsForTaskAllocatorId(handle, dataSource, taskAllocatorId),
0, 0,
SQLMetadataConnector.DEFAULT_MAX_TRIES SQLMetadataConnector.DEFAULT_MAX_TRIES
); );
@ -1668,11 +1628,11 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
// across all shard specs (published + pending). // across all shard specs (published + pending).
// A pending segment having a higher partitionId must also be considered // A pending segment having a higher partitionId must also be considered
// to avoid clashes when inserting the pending segment created here. // to avoid clashes when inserting the pending segment created here.
final Set<SegmentIdWithShardSpec> pendingSegments = new HashSet<>( final Set<SegmentIdWithShardSpec> pendingSegments =
getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).stream() getPendingSegmentsForInterval(handle, dataSource, interval)
.map(PendingSegmentRecord::getId) .stream()
.collect(Collectors.toSet()) .map(PendingSegmentRecord::getId)
); .collect(Collectors.toSet());
final Map<SegmentCreateRequest, PendingSegmentRecord> createdSegments = new HashMap<>(); final Map<SegmentCreateRequest, PendingSegmentRecord> createdSegments = new HashMap<>();
final Map<UniqueAllocateRequest, PendingSegmentRecord> uniqueRequestToSegment = new HashMap<>(); final Map<UniqueAllocateRequest, PendingSegmentRecord> uniqueRequestToSegment = new HashMap<>();
@ -1686,7 +1646,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
if (uniqueRequestToSegment.containsKey(uniqueRequest)) { if (uniqueRequestToSegment.containsKey(uniqueRequest)) {
createdSegment = uniqueRequestToSegment.get(uniqueRequest); createdSegment = uniqueRequestToSegment.get(uniqueRequest);
} else { } else {
createdSegment = createNewSegment( createdSegment = createNewPendingSegment(
request, request,
dataSource, dataSource,
interval, interval,
@ -1712,7 +1672,8 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
return createdSegments; return createdSegments;
} }
private PendingSegmentRecord createNewSegment( @Nullable
private PendingSegmentRecord createNewPendingSegment(
SegmentCreateRequest request, SegmentCreateRequest request,
String dataSource, String dataSource,
Interval interval, Interval interval,
@ -1775,17 +1736,14 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
pendingSegmentId, pendingSegmentId,
request.getSequenceName(), request.getSequenceName(),
request.getPreviousSegmentId(), request.getPreviousSegmentId(),
request.getUpgradedFromSegmentId(), null,
request.getTaskAllocatorId() request.getTaskAllocatorId()
); );
} else if (!overallMaxId.getInterval().equals(interval)) { } else if (!overallMaxId.getInterval().equals(interval)) {
log.warn( log.warn(
"Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].", "Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].",
dataSource, dataSource, interval, existingVersion, overallMaxId
interval,
existingVersion,
overallMaxId
); );
return null; return null;
} else if (committedMaxId != null } else if (committedMaxId != null
@ -1793,8 +1751,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
== SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS) { == SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS) {
log.warn( log.warn(
"Cannot allocate new segment because of unknown core partition size of segment[%s], shardSpec[%s]", "Cannot allocate new segment because of unknown core partition size of segment[%s], shardSpec[%s]",
committedMaxId, committedMaxId, committedMaxId.getShardSpec()
committedMaxId.getShardSpec()
); );
return null; return null;
} else { } else {
@ -1815,28 +1772,20 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
getTrueAllocatedId(pendingSegmentId), getTrueAllocatedId(pendingSegmentId),
request.getSequenceName(), request.getSequenceName(),
request.getPreviousSegmentId(), request.getPreviousSegmentId(),
request.getUpgradedFromSegmentId(), null,
request.getTaskAllocatorId() request.getTaskAllocatorId()
); );
} }
} }
/** /**
* This function creates a new segment for the given datasource/interval/etc. A critical * Creates a new pending segment for the given datasource and interval.
* aspect of the creation is to make sure that the new version & new partition number will make
* sense given the existing segments & pending segments also very important is to avoid
* clashes with existing pending & used/unused segments.
* @param handle Database handle
* @param dataSource datasource for the new segment
* @param interval interval for the new segment
* @param partialShardSpec Shard spec info minus segment id stuff * @param partialShardSpec Shard spec info minus segment id stuff
* @param existingVersion Version of segments in interval, used to compute the version of the very first segment in * @param existingVersion Version of segments in interval, used to compute the version of the very first segment in
* interval * interval
* @return
* @throws IOException
*/ */
@Nullable @Nullable
private SegmentIdWithShardSpec createNewSegment( private SegmentIdWithShardSpec createNewPendingSegment(
final Handle handle, final Handle handle,
final String dataSource, final String dataSource,
final Interval interval, final Interval interval,
@ -1876,11 +1825,12 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
// across all shard specs (published + pending). // across all shard specs (published + pending).
// A pending segment having a higher partitionId must also be considered // A pending segment having a higher partitionId must also be considered
// to avoid clashes when inserting the pending segment created here. // to avoid clashes when inserting the pending segment created here.
final Set<SegmentIdWithShardSpec> pendings = new HashSet<>( final Set<SegmentIdWithShardSpec> pendings =
getPendingSegmentsForIntervalWithHandle(handle, dataSource, interval).stream() getPendingSegmentsForInterval(handle, dataSource, interval)
.map(PendingSegmentRecord::getId) .stream()
.collect(Collectors.toSet()) .map(PendingSegmentRecord::getId)
); .collect(Collectors.toSet());
if (committedMaxId != null) { if (committedMaxId != null) {
pendings.add(committedMaxId); pendings.add(committedMaxId);
} }
@ -1910,11 +1860,9 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
} }
if (overallMaxId == null) { if (overallMaxId == null) {
// When appending segments, null overallMaxId means that we are allocating the very initial // We are allocating the very first segment for this time chunk.
// segment for this time chunk. // Set numCorePartitions to 0 as core partitions are not determined for append segments
// This code is executed when the Overlord coordinates segment allocation, which is either you append segments // When you use segment lock, the core partitions set doesn't work with it. We simply set it 0 so that the
// or you use segment lock. Since the core partitions set is not determined for appended segments, we set
// it 0. When you use segment lock, the core partitions set doesn't work with it. We simply set it 0 so that the
// OvershadowableManager handles the atomic segment update. // OvershadowableManager handles the atomic segment update.
final int newPartitionId = partialShardSpec.useNonRootGenerationPartitionSpace() final int newPartitionId = partialShardSpec.useNonRootGenerationPartitionSpace()
? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID ? PartitionIds.NON_ROOT_GEN_START_PARTITION_ID
@ -1929,10 +1877,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
} else if (!overallMaxId.getInterval().equals(interval)) { } else if (!overallMaxId.getInterval().equals(interval)) {
log.warn( log.warn(
"Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].", "Cannot allocate new segment for dataSource[%s], interval[%s], existingVersion[%s]: conflicting segment[%s].",
dataSource, dataSource, interval, existingVersion, overallMaxId
interval,
existingVersion,
overallMaxId
); );
return null; return null;
} else if (committedMaxId != null } else if (committedMaxId != null
@ -1940,14 +1885,12 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
== SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS) { == SingleDimensionShardSpec.UNKNOWN_NUM_CORE_PARTITIONS) {
log.warn( log.warn(
"Cannot allocate new segment because of unknown core partition size of segment[%s], shardSpec[%s]", "Cannot allocate new segment because of unknown core partition size of segment[%s], shardSpec[%s]",
committedMaxId, committedMaxId, committedMaxId.getShardSpec()
committedMaxId.getShardSpec()
); );
return null; return null;
} else { } else {
// The number of core partitions must always be chosen from the set of used segments in the SegmentTimeline. // numCorePartitions must always be picked from the committedMaxId and not overallMaxId
// When the core partitions have been dropped, using pending segments may lead to an incorrect state // as overallMaxId may refer to a pending segment which might have stale info of numCorePartitions
// where the chunk is believed to have core partitions and queries results are incorrect.
final SegmentIdWithShardSpec allocatedId = new SegmentIdWithShardSpec( final SegmentIdWithShardSpec allocatedId = new SegmentIdWithShardSpec(
dataSource, dataSource,
interval, interval,
@ -1963,7 +1906,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
} }
/** /**
* Verifies that the allocated id doesn't already exist in the druid segments table. * Verifies that the allocated id doesn't already exist in the druid_segments table.
* If yes, try to get the max unallocated id considering the unused segments for the datasource, version and interval * If yes, try to get the max unallocated id considering the unused segments for the datasource, version and interval
* Otherwise, use the same id. * Otherwise, use the same id.
* @param allocatedId The segment allcoted on the basis of used and pending segments * @param allocatedId The segment allcoted on the basis of used and pending segments
@ -1977,7 +1920,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
} }
// If yes, try to compute allocated partition num using the max unused segment shard spec // If yes, try to compute allocated partition num using the max unused segment shard spec
SegmentId unusedMaxId = getUnusedMaxId( SegmentId unusedMaxId = getMaxIdOfUnusedSegment(
allocatedId.getDataSource(), allocatedId.getDataSource(),
allocatedId.getInterval(), allocatedId.getInterval(),
allocatedId.getVersion() allocatedId.getVersion()
@ -2002,7 +1945,14 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
); );
} }
private SegmentId getUnusedMaxId(String datasource, Interval interval, String version) /**
* Determines the highest ID amongst unused segments for the given datasource,
* interval and version.
*
* @return null if no unused segment exists for the given parameters.
*/
@Nullable
private SegmentId getMaxIdOfUnusedSegment(String datasource, Interval interval, String version)
{ {
List<String> unusedSegmentIds = retrieveUnusedSegmentIdsForExactIntervalAndVersion( List<String> unusedSegmentIds = retrieveUnusedSegmentIdsForExactIntervalAndVersion(
datasource, datasource,
@ -2134,7 +2084,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
.bind("created_date", now) .bind("created_date", now)
.bind("start", segment.getInterval().getStart().toString()) .bind("start", segment.getInterval().getStart().toString())
.bind("end", segment.getInterval().getEnd().toString()) .bind("end", segment.getInterval().getEnd().toString())
.bind("partitioned", (segment.getShardSpec() instanceof NoneShardSpec) ? false : true) .bind("partitioned", !(segment.getShardSpec() instanceof NoneShardSpec))
.bind("version", segment.getVersion()) .bind("version", segment.getVersion())
.bind("used", usedSegments.contains(segment)) .bind("used", usedSegments.contains(segment))
.bind("payload", jsonMapper.writeValueAsBytes(segment)) .bind("payload", jsonMapper.writeValueAsBytes(segment))
@ -2330,9 +2280,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
Map<String, String> upgradedFromSegmentIdMap Map<String, String> upgradedFromSegmentIdMap
) throws IOException ) throws IOException
{ {
boolean shouldPersistSchema = shouldPersistSchema(segmentSchemaMapping); if (shouldPersistSchema(segmentSchemaMapping)) {
if (shouldPersistSchema) {
persistSchema(handle, segments, segmentSchemaMapping); persistSchema(handle, segments, segmentSchemaMapping);
} }
@ -2407,6 +2355,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
return segmentsToInsert; return segmentsToInsert;
} }
@Nullable
private SegmentMetadata getSegmentMetadataFromSchemaMappingOrUpgradeMetadata( private SegmentMetadata getSegmentMetadataFromSchemaMappingOrUpgradeMetadata(
final SegmentId segmentId, final SegmentId segmentId,
final SegmentSchemaMapping segmentSchemaMapping, final SegmentSchemaMapping segmentSchemaMapping,
@ -2786,27 +2735,18 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
Hashing.sha1().hashBytes(newCommitMetadataBytes).asBytes() Hashing.sha1().hashBytes(newCommitMetadataBytes).asBytes()
); );
final String sql = "UPDATE %s SET "
+ "commit_metadata_payload = :new_commit_metadata_payload, "
+ "commit_metadata_sha1 = :new_commit_metadata_sha1 "
+ "WHERE dataSource = :dataSource";
return connector.retryWithHandle( return connector.retryWithHandle(
new HandleCallback<Boolean>() handle -> {
{ final int numRows = handle.createStatement(StringUtils.format(sql, dbTables.getDataSourceTable()))
@Override .bind("dataSource", dataSource)
public Boolean withHandle(Handle handle) .bind("new_commit_metadata_payload", newCommitMetadataBytes)
{ .bind("new_commit_metadata_sha1", newCommitMetadataSha1)
final int numRows = handle.createStatement( .execute();
StringUtils.format( return numRows == 1;
"UPDATE %s SET "
+ "commit_metadata_payload = :new_commit_metadata_payload, "
+ "commit_metadata_sha1 = :new_commit_metadata_sha1 "
+ "WHERE dataSource = :dataSource",
dbTables.getDataSourceTable()
)
)
.bind("dataSource", dataSource)
.bind("new_commit_metadata_payload", newCommitMetadataBytes)
.bind("new_commit_metadata_sha1", newCommitMetadataSha1)
.execute();
return numRows == 1;
}
} }
); );
} }
@ -3028,7 +2968,7 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
public List<PendingSegmentRecord> getPendingSegments(String datasource, Interval interval) public List<PendingSegmentRecord> getPendingSegments(String datasource, Interval interval)
{ {
return connector.retryWithHandle( return connector.retryWithHandle(
handle -> getPendingSegmentsForIntervalWithHandle(handle, datasource, interval) handle -> getPendingSegmentsForInterval(handle, datasource, interval)
); );
} }
@ -3178,7 +3118,6 @@ 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; private final String errorMsg;
public static final DataStoreMetadataUpdateResult SUCCESS = new DataStoreMetadataUpdateResult(false, false, null); public static final DataStoreMetadataUpdateResult SUCCESS = new DataStoreMetadataUpdateResult(false, false, null);
@ -3198,7 +3137,6 @@ public class IndexerSQLMetadataStorageCoordinator implements IndexerMetadataStor
this.failed = failed; this.failed = failed;
this.canRetry = canRetry; this.canRetry = canRetry;
this.errorMsg = null == errorMsg ? null : StringUtils.format(errorMsg, errorFormatArgs); this.errorMsg = null == errorMsg ? null : StringUtils.format(errorMsg, errorFormatArgs);
} }
public boolean isFailed() public boolean isFailed()

View File

@ -36,7 +36,6 @@ public class SegmentCreateRequestTest
null, null,
"version", "version",
partialShardSpec, partialShardSpec,
null,
null null
); );
Assert.assertEquals("sequence", request.getSequenceName()); Assert.assertEquals("sequence", request.getSequenceName());

View File

@ -2280,7 +2280,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
final String dataSource = "ds"; final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
null, null,
@ -2293,7 +2293,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version", identifier.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version", identifier.toString());
final SegmentIdWithShardSpec identifier1 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier1 = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
identifier.toString(), identifier.toString(),
@ -2306,7 +2306,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_1", identifier1.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_1", identifier1.toString());
final SegmentIdWithShardSpec identifier2 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier2 = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
identifier1.toString(), identifier1.toString(),
@ -2319,7 +2319,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_2", identifier2.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_2", identifier2.toString());
final SegmentIdWithShardSpec identifier3 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier3 = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
identifier1.toString(), identifier1.toString(),
@ -2333,7 +2333,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_2", identifier3.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_2", identifier3.toString());
Assert.assertEquals(identifier2, identifier3); Assert.assertEquals(identifier2, identifier3);
final SegmentIdWithShardSpec identifier4 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier4 = allocatePendingSegment(
dataSource, dataSource,
"seq1", "seq1",
null, null,
@ -2370,7 +2370,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
final String dataSource = "ds"; final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
null, null,
@ -2385,7 +2385,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals(0, identifier.getShardSpec().getNumCorePartitions()); Assert.assertEquals(0, identifier.getShardSpec().getNumCorePartitions());
// simulate one more load using kafka streaming (as if previous segment was published, note different sequence name) // simulate one more load using kafka streaming (as if previous segment was published, note different sequence name)
final SegmentIdWithShardSpec identifier1 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier1 = allocatePendingSegment(
dataSource, dataSource,
"seq2", "seq2",
identifier.toString(), identifier.toString(),
@ -2400,7 +2400,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals(0, identifier1.getShardSpec().getNumCorePartitions()); Assert.assertEquals(0, identifier1.getShardSpec().getNumCorePartitions());
// simulate one more load using kafka streaming (as if previous segment was published, note different sequence name) // simulate one more load using kafka streaming (as if previous segment was published, note different sequence name)
final SegmentIdWithShardSpec identifier2 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier2 = allocatePendingSegment(
dataSource, dataSource,
"seq3", "seq3",
identifier1.toString(), identifier1.toString(),
@ -2431,7 +2431,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_new", ids.get(0)); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_version_new", ids.get(0));
// one more load on same interval: // one more load on same interval:
final SegmentIdWithShardSpec identifier3 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier3 = allocatePendingSegment(
dataSource, dataSource,
"seq4", "seq4",
identifier1.toString(), identifier1.toString(),
@ -2450,7 +2450,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
// and final load, this reproduces an issue that could happen with multiple streaming appends, // and final load, this reproduces an issue that could happen with multiple streaming appends,
// followed by a reindex, followed by a drop, and more streaming data coming in for same interval // followed by a reindex, followed by a drop, and more streaming data coming in for same interval
final SegmentIdWithShardSpec identifier4 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier4 = allocatePendingSegment(
dataSource, dataSource,
"seq5", "seq5",
identifier1.toString(), identifier1.toString(),
@ -2484,7 +2484,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
final String dataSource = "ds"; final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
null, null,
@ -2513,7 +2513,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
// 1.1) simulate one more append load (as if previous segment was published, note different sequence name) // 1.1) simulate one more append load (as if previous segment was published, note different sequence name)
final SegmentIdWithShardSpec identifier1 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier1 = allocatePendingSegment(
dataSource, dataSource,
"seq2", "seq2",
identifier.toString(), identifier.toString(),
@ -2542,7 +2542,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
// 1.2) simulate one more append load (as if previous segment was published, note different sequence name) // 1.2) simulate one more append load (as if previous segment was published, note different sequence name)
final SegmentIdWithShardSpec identifier2 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier2 = allocatePendingSegment(
dataSource, dataSource,
"seq3", "seq3",
identifier1.toString(), identifier1.toString(),
@ -2597,7 +2597,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
// unused segment: // unused segment:
// 4) pending segment of version = B, id = 1 <= appending new data, aborted // 4) pending segment of version = B, id = 1 <= appending new data, aborted
final SegmentIdWithShardSpec identifier3 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier3 = allocatePendingSegment(
dataSource, dataSource,
"seq4", "seq4",
identifier2.toString(), identifier2.toString(),
@ -2632,7 +2632,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals(1, unused.size()); Assert.assertEquals(1, unused.size());
// Simulate one more append load // Simulate one more append load
final SegmentIdWithShardSpec identifier4 = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier4 = allocatePendingSegment(
dataSource, dataSource,
"seq5", "seq5",
identifier1.toString(), identifier1.toString(),
@ -2678,7 +2678,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final String sequenceName = "seq"; final String sequenceName = "seq";
final SegmentCreateRequest request = new SegmentCreateRequest(sequenceName, null, "v1", partialShardSpec, null, null); final SegmentCreateRequest request = new SegmentCreateRequest(sequenceName, null, "v1", partialShardSpec, null);
final SegmentIdWithShardSpec segmentId0 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId0 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2690,7 +2690,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1", segmentId0.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1", segmentId0.toString());
final SegmentCreateRequest request1 = final SegmentCreateRequest request1 =
new SegmentCreateRequest(sequenceName, segmentId0.toString(), segmentId0.getVersion(), partialShardSpec, null, null); new SegmentCreateRequest(sequenceName, segmentId0.toString(), segmentId0.getVersion(), partialShardSpec, null);
final SegmentIdWithShardSpec segmentId1 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId1 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2702,7 +2702,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_1", segmentId1.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_1", segmentId1.toString());
final SegmentCreateRequest request2 = final SegmentCreateRequest request2 =
new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null, null); new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null);
final SegmentIdWithShardSpec segmentId2 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId2 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2714,7 +2714,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId2.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId2.toString());
final SegmentCreateRequest request3 = final SegmentCreateRequest request3 =
new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null, null); new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null);
final SegmentIdWithShardSpec segmentId3 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId3 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2727,7 +2727,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals(segmentId2, segmentId3); Assert.assertEquals(segmentId2, segmentId3);
final SegmentCreateRequest request4 = final SegmentCreateRequest request4 =
new SegmentCreateRequest("seq1", null, "v1", partialShardSpec, null, null); new SegmentCreateRequest("seq1", null, "v1", partialShardSpec, null);
final SegmentIdWithShardSpec segmentId4 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId4 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2747,7 +2747,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final String sequenceName = "seq"; final String sequenceName = "seq";
final SegmentCreateRequest request = new SegmentCreateRequest(sequenceName, null, "v1", partialShardSpec, null, null); final SegmentCreateRequest request = new SegmentCreateRequest(sequenceName, null, "v1", partialShardSpec, null);
final SegmentIdWithShardSpec segmentId0 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId0 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2759,7 +2759,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1", segmentId0.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1", segmentId0.toString());
final SegmentCreateRequest request1 = final SegmentCreateRequest request1 =
new SegmentCreateRequest(sequenceName, segmentId0.toString(), segmentId0.getVersion(), partialShardSpec, null, null); new SegmentCreateRequest(sequenceName, segmentId0.toString(), segmentId0.getVersion(), partialShardSpec, null);
final SegmentIdWithShardSpec segmentId1 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId1 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2771,7 +2771,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_1", segmentId1.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_1", segmentId1.toString());
final SegmentCreateRequest request2 = final SegmentCreateRequest request2 =
new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null, null); new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null);
final SegmentIdWithShardSpec segmentId2 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId2 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2783,7 +2783,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId2.toString()); Assert.assertEquals("ds_2017-01-01T00:00:00.000Z_2017-02-01T00:00:00.000Z_v1_2", segmentId2.toString());
final SegmentCreateRequest request3 = final SegmentCreateRequest request3 =
new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null, null); new SegmentCreateRequest(sequenceName, segmentId1.toString(), segmentId1.getVersion(), partialShardSpec, null);
final SegmentIdWithShardSpec segmentId3 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId3 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2796,7 +2796,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertEquals(segmentId2, segmentId3); Assert.assertEquals(segmentId2, segmentId3);
final SegmentCreateRequest request4 = final SegmentCreateRequest request4 =
new SegmentCreateRequest("seq1", null, "v1", partialShardSpec, null, null); new SegmentCreateRequest("seq1", null, "v1", partialShardSpec, null);
final SegmentIdWithShardSpec segmentId4 = coordinator.allocatePendingSegments( final SegmentIdWithShardSpec segmentId4 = coordinator.allocatePendingSegments(
dataSource, dataSource,
interval, interval,
@ -2833,7 +2833,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance(); final PartialShardSpec partialShardSpec = NumberedPartialShardSpec.instance();
final String dataSource = "ds"; final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
null, null,
@ -2857,7 +2857,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final DateTime begin = DateTimes.nowUtc(); final DateTime begin = DateTimes.nowUtc();
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
prevSegmentId, prevSegmentId,
@ -2873,7 +2873,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final DateTime secondBegin = DateTimes.nowUtc(); final DateTime secondBegin = DateTimes.nowUtc();
for (int i = 0; i < 5; i++) { for (int i = 0; i < 5; i++) {
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
prevSegmentId, prevSegmentId,
@ -2901,7 +2901,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
String prevSegmentId = null; String prevSegmentId = null;
for (int i = 0; i < 10; i++) { for (int i = 0; i < 10; i++) {
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
prevSegmentId, prevSegmentId,
@ -2970,7 +2970,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
final String dataSource = "ds"; final String dataSource = "ds";
final Interval interval = Intervals.of("2017-01-01/2017-02-01"); final Interval interval = Intervals.of("2017-01-01/2017-02-01");
SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( SegmentIdWithShardSpec id = allocatePendingSegment(
dataSource, dataSource,
"seq", "seq",
null, null,
@ -3003,7 +3003,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)
); );
id = coordinator.allocatePendingSegment( id = allocatePendingSegment(
dataSource, dataSource,
"seq2", "seq2",
null, null,
@ -3036,7 +3036,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION) new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)
); );
id = coordinator.allocatePendingSegment( id = allocatePendingSegment(
dataSource, dataSource,
"seq3", "seq3",
null, null,
@ -3084,7 +3084,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
); );
} }
coordinator.commitSegments(originalSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); coordinator.commitSegments(originalSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION));
final SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec id = allocatePendingSegment(
datasource, datasource,
"seq", "seq",
null, null,
@ -3130,7 +3130,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
); );
} }
coordinator.commitSegments(originalSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)); coordinator.commitSegments(originalSegments, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION));
final SegmentIdWithShardSpec id = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec id = allocatePendingSegment(
datasource, datasource,
"seq", "seq",
null, null,
@ -3377,7 +3377,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertTrue(coordinator.commitSegments(tombstones, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(tombstones)); Assert.assertTrue(coordinator.commitSegments(tombstones, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(tombstones));
// Allocate and commit a data segment by appending to the same interval // Allocate and commit a data segment by appending to the same interval
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
TestDataSource.WIKI, TestDataSource.WIKI,
"seq", "seq",
tombstoneSegment.getVersion(), tombstoneSegment.getVersion(),
@ -3432,7 +3432,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
Assert.assertTrue(coordinator.commitSegments(tombstones, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(tombstones)); Assert.assertTrue(coordinator.commitSegments(tombstones, new SegmentSchemaMapping(CentralizedDatasourceSchemaConfig.SCHEMA_VERSION)).containsAll(tombstones));
// Allocate and commit a data segment by appending to the same interval // Allocate and commit a data segment by appending to the same interval
final SegmentIdWithShardSpec identifier = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec identifier = allocatePendingSegment(
TestDataSource.WIKI, TestDataSource.WIKI,
"seq", "seq",
tombstoneSegment.getVersion(), tombstoneSegment.getVersion(),
@ -3471,7 +3471,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
@Test @Test
public void testSegmentIdShouldNotBeReallocated() public void testSegmentIdShouldNotBeReallocated()
{ {
final SegmentIdWithShardSpec idWithNullTaskAllocator = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec idWithNullTaskAllocator = allocatePendingSegment(
TestDataSource.WIKI, TestDataSource.WIKI,
"seq", "seq",
"0", "0",
@ -3487,7 +3487,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
idWithNullTaskAllocator.getShardSpec() idWithNullTaskAllocator.getShardSpec()
); );
final SegmentIdWithShardSpec idWithValidTaskAllocator = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec idWithValidTaskAllocator = allocatePendingSegment(
TestDataSource.WIKI, TestDataSource.WIKI,
"seq", "seq",
"1", "1",
@ -3510,7 +3510,7 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
// Mark all segments as unused // Mark all segments as unused
coordinator.markSegmentsAsUnusedWithinInterval(TestDataSource.WIKI, Intervals.ETERNITY); coordinator.markSegmentsAsUnusedWithinInterval(TestDataSource.WIKI, Intervals.ETERNITY);
final SegmentIdWithShardSpec theId = coordinator.allocatePendingSegment( final SegmentIdWithShardSpec theId = allocatePendingSegment(
TestDataSource.WIKI, TestDataSource.WIKI,
"seq", "seq",
"2", "2",
@ -3791,6 +3791,30 @@ public class IndexerSQLMetadataStorageCoordinatorTest extends IndexerSqlMetadata
); );
} }
private SegmentIdWithShardSpec allocatePendingSegment(
String datasource,
String sequenceName,
String previousSegmentId,
Interval interval,
PartialShardSpec partialShardSpec,
String maxVersion,
boolean skipSegmentLineageCheck,
String taskAllocatorId
)
{
return coordinator.allocatePendingSegment(
datasource,
interval,
skipSegmentLineageCheck,
new SegmentCreateRequest(
sequenceName,
previousSegmentId,
maxVersion,
partialShardSpec,
taskAllocatorId
)
);
}
private void insertUsedSegments(Set<DataSegment> segments, Map<String, String> upgradedFromSegmentIdMap) private void insertUsedSegments(Set<DataSegment> segments, Map<String, String> upgradedFromSegmentIdMap)
{ {