mirror of https://github.com/apache/druid.git
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:
parent
b86ea4d5c4
commit
24e5d8a9e8
|
@ -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()
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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}.
|
||||||
*
|
*
|
||||||
|
|
|
@ -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;
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -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}.
|
||||||
*
|
*
|
||||||
*
|
*
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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()
|
||||||
|
|
|
@ -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());
|
||||||
|
|
|
@ -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)
|
||||||
{
|
{
|
||||||
|
|
Loading…
Reference in New Issue