Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Reduce metadata IO during segment allocation #17496

Open
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,6 @@
import org.apache.druid.indexing.common.task.IndexTaskUtils;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.config.TaskLockConfig;
import org.apache.druid.java.util.common.ISE;
Expand All @@ -41,6 +40,7 @@
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions;
import org.joda.time.Interval;

import java.util.ArrayList;
Expand Down Expand Up @@ -87,6 +87,8 @@ public class SegmentAllocationQueue
private final ConcurrentHashMap<AllocateRequestKey, AllocateRequestBatch> keyToBatch = new ConcurrentHashMap<>();
private final BlockingDeque<AllocateRequestBatch> processingQueue = new LinkedBlockingDeque<>(MAX_QUEUE_SIZE);

private final boolean reduceMetadataIO;

@Inject
public SegmentAllocationQueue(
TaskLockbox taskLockbox,
Expand All @@ -100,6 +102,7 @@ public SegmentAllocationQueue(
this.taskLockbox = taskLockbox;
this.metadataStorage = metadataStorage;
this.maxWaitTimeMillis = taskLockConfig.getBatchAllocationWaitTime();
this.reduceMetadataIO = taskLockConfig.isBatchAllocationReduceMetadataIO();

this.executor = taskLockConfig.isBatchSegmentAllocation()
? executorFactory.create(1, "SegmentAllocQueue-%s") : null;
Expand Down Expand Up @@ -380,13 +383,11 @@ private boolean processBatch(AllocateRequestBatch requestBatch)

private Set<DataSegment> retrieveUsedSegments(AllocateRequestKey key)
{
return new HashSet<>(
metadataStorage.retrieveUsedSegmentsForInterval(
key.dataSource,
key.preferredAllocationInterval,
Segments.ONLY_VISIBLE
)
);
return metadataStorage.getSegmentTimelineForAllocation(
key.dataSource,
key.preferredAllocationInterval,
(key.lockGranularity == LockGranularity.TIME_CHUNK) && reduceMetadataIO
).findNonOvershadowedObjectsInInterval(Intervals.ETERNITY, Partitions.ONLY_COMPLETE);
}

private int allocateSegmentsForBatch(AllocateRequestBatch requestBatch, Set<DataSegment> usedSegments)
Expand Down Expand Up @@ -493,7 +494,8 @@ private int allocateSegmentsForInterval(
requestKey.dataSource,
tryInterval,
requestKey.skipSegmentLineageCheck,
requestKey.lockGranularity
requestKey.lockGranularity,
reduceMetadataIO
);

int successfulRequests = 0;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -466,14 +466,17 @@ public LockResult tryLock(final Task task, final LockRequest request)
* @param skipSegmentLineageCheck Whether lineage check is to be skipped
* (this is true for streaming ingestion)
* @param lockGranularity Granularity of task lock
* @param reduceMetadataIO Whether to skip fetching payloads for all used
* segments and rely on their IDs instead.
* @return List of allocation results in the same order as the requests.
*/
public List<SegmentAllocateResult> allocateSegments(
List<SegmentAllocateRequest> requests,
String dataSource,
Interval interval,
boolean skipSegmentLineageCheck,
LockGranularity lockGranularity
LockGranularity lockGranularity,
boolean reduceMetadataIO
)
{
log.info("Allocating [%d] segments for datasource [%s], interval [%s]", requests.size(), dataSource, interval);
Expand All @@ -487,9 +490,15 @@ public List<SegmentAllocateResult> allocateSegments(
if (isTimeChunkLock) {
// For time-chunk locking, segment must be allocated only after acquiring the lock
holderList.getPending().forEach(holder -> acquireTaskLock(holder, true));
allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending());
allocateSegmentIds(
dataSource,
interval,
skipSegmentLineageCheck,
holderList.getPending(),
reduceMetadataIO
);
} else {
allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending());
allocateSegmentIds(dataSource, interval, skipSegmentLineageCheck, holderList.getPending(), false);
holderList.getPending().forEach(holder -> acquireTaskLock(holder, false));
}
holderList.getPending().forEach(SegmentAllocationHolder::markSucceeded);
Expand Down Expand Up @@ -702,12 +711,12 @@ private TaskLockPosse createNewTaskLockPosse(LockRequest request)
* for the given requests. Updates the holder with the allocated segment if
* the allocation succeeds, otherwise marks it as failed.
*/
@VisibleForTesting
void allocateSegmentIds(
private void allocateSegmentIds(
String dataSource,
Interval interval,
boolean skipSegmentLineageCheck,
Collection<SegmentAllocationHolder> holders
Collection<SegmentAllocationHolder> holders,
boolean reduceMetadataIO
)
{
if (holders.isEmpty()) {
Expand All @@ -724,7 +733,8 @@ void allocateSegmentIds(
dataSource,
interval,
skipSegmentLineageCheck,
createRequests
createRequests,
reduceMetadataIO
);

for (SegmentAllocationHolder holder : holders) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,9 @@ public class TaskLockConfig
@JsonProperty
private long batchAllocationWaitTime = 0L;

@JsonProperty
private boolean batchAllocationReduceMetadataIO = false;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should this be always on and then we remove this config in next release ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I guess the changes are safe enough. We can always keep it on. Let me try to run ITs with it turned on.


public boolean isForceTimeChunkLock()
{
return forceTimeChunkLock;
Expand All @@ -50,4 +53,10 @@ public long getBatchAllocationWaitTime()
{
return batchAllocationWaitTime;
}

public boolean isBatchAllocationReduceMetadataIO()
{
return batchAllocationReduceMetadataIO;
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -93,21 +93,28 @@ public class SegmentAllocateActionTest

private SegmentAllocationQueue allocationQueue;

@Parameterized.Parameters(name = "granularity = {0}, useBatch = {1}")
@Parameterized.Parameters(name = "granularity = {0}, useBatch = {1}, skipSegmentPayloadFetchForAllocation = {2}")
public static Iterable<Object[]> constructorFeeder()
{
return ImmutableList.of(
new Object[]{LockGranularity.SEGMENT, true},
new Object[]{LockGranularity.SEGMENT, false},
new Object[]{LockGranularity.TIME_CHUNK, true},
new Object[]{LockGranularity.TIME_CHUNK, false}
new Object[]{LockGranularity.SEGMENT, true, true},
new Object[]{LockGranularity.SEGMENT, true, false},
new Object[]{LockGranularity.SEGMENT, false, false},
new Object[]{LockGranularity.TIME_CHUNK, true, true},
new Object[]{LockGranularity.TIME_CHUNK, true, false},
new Object[]{LockGranularity.TIME_CHUNK, false, false}
);
}

public SegmentAllocateActionTest(LockGranularity lockGranularity, boolean useBatch)
public SegmentAllocateActionTest(
LockGranularity lockGranularity,
boolean useBatch,
boolean skipSegmentPayloadFetchForAllocation
)
{
this.lockGranularity = lockGranularity;
this.useBatch = useBatch;
this.taskActionTestKit.setSkipSegmentPayloadFetchForAllocation(skipSegmentPayloadFetchForAllocation);
}

@Before
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;

import java.util.ArrayList;
import java.util.List;
Expand All @@ -44,6 +46,7 @@
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;

@RunWith(Parameterized.class)
public class SegmentAllocationQueueTest
{
@Rule
Expand All @@ -54,6 +57,19 @@ public class SegmentAllocationQueueTest
private StubServiceEmitter emitter;
private BlockingExecutorService executor;

private final boolean reduceMetadataIO;

@Parameterized.Parameters(name = "reduceMetadataIO = {0}")
public static Object[][] getTestParameters()
{
return new Object[][]{{true}, {false}};
}

public SegmentAllocationQueueTest(boolean reduceMetadataIO)
{
this.reduceMetadataIO = reduceMetadataIO;
}

@Before
public void setUp()
{
Expand All @@ -73,6 +89,12 @@ public long getBatchAllocationWaitTime()
{
return 0;
}

@Override
public boolean isBatchAllocationReduceMetadataIO()
{
return reduceMetadataIO;
}
};

allocationQueue = new SegmentAllocationQueue(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,8 @@ public class TaskActionTestKit extends ExternalResource
private SegmentSchemaManager segmentSchemaManager;
private SegmentSchemaCache segmentSchemaCache;

private boolean skipSegmentPayloadFetchForAllocation = new TaskLockConfig().isBatchAllocationReduceMetadataIO();

public TaskLockbox getTaskLockbox()
{
return taskLockbox;
Expand All @@ -78,6 +80,11 @@ public TaskActionToolbox getTaskActionToolbox()
return taskActionToolbox;
}

public void setSkipSegmentPayloadFetchForAllocation(boolean skipSegmentPayloadFetchForAllocation)
{
this.skipSegmentPayloadFetchForAllocation = skipSegmentPayloadFetchForAllocation;
}

@Override
public void before()
{
Expand Down Expand Up @@ -126,6 +133,12 @@ public long getBatchAllocationWaitTime()
{
return 10L;
}

@Override
public boolean isBatchAllocationReduceMetadataIO()
{
return skipSegmentPayloadFetchForAllocation;
}
};

taskActionToolbox = new TaskActionToolbox(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@
import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
Expand Down Expand Up @@ -168,7 +169,8 @@ public Map<SegmentCreateRequest, SegmentIdWithShardSpec> allocatePendingSegments
String dataSource,
Interval interval,
boolean skipSegmentLineageCheck,
List<SegmentCreateRequest> requests
List<SegmentCreateRequest> requests,
boolean isTimeChunk
)
{
return Collections.emptyMap();
Expand Down Expand Up @@ -332,6 +334,20 @@ public Map<String, Set<String>> retrieveUpgradedToSegmentIds(
return Collections.emptyMap();
}

@Override
public SegmentTimeline getSegmentTimelineForAllocation(
String dataSource,
Interval interval,
boolean skipSegmentPayloadFetchForAllocation
)
{
return SegmentTimeline.forSegments(retrieveUsedSegmentsForIntervals(
dataSource,
Collections.singletonList(interval),
Segments.INCLUDING_OVERSHADOWED
));
}

public Set<DataSegment> getPublished()
{
return ImmutableSet.copyOf(published);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.realtime.appenderator.SegmentIdWithShardSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
import org.apache.druid.timeline.partition.PartialShardSpec;
import org.joda.time.DateTime;
import org.joda.time.Interval;
Expand Down Expand Up @@ -179,14 +180,29 @@ List<DataSegment> retrieveUnusedSegmentsForInterval(
* Should be set to false if replica tasks would index events in same order
* @param requests Requests for which to allocate segments. All
* the requests must share the same partition space.
* @param reduceMetadataIO If true, try to use the segment ids instead of fetching every segment
* payload from the metadata store
* @return Map from request to allocated segment id. The map does not contain
* entries for failed requests.
*/
Map<SegmentCreateRequest, SegmentIdWithShardSpec> allocatePendingSegments(
String dataSource,
Interval interval,
boolean skipSegmentLineageCheck,
List<SegmentCreateRequest> requests
List<SegmentCreateRequest> requests,
boolean reduceMetadataIO
);

/**
* Return a segment timeline of all used segments including overshadowed ones for a given datasource and interval
* if skipSegmentPayloadFetchForAllocation is set to true, do not fetch all the segment payloads for allocation
* Instead fetch all the ids and numCorePartitions using exactly one segment per version per interval
* return a dummy DataSegment for each id that holds only the SegmentId and a NumberedShardSpec with numCorePartitions
*/
SegmentTimeline getSegmentTimelineForAllocation(
String dataSource,
Interval interval,
boolean skipSegmentPayloadFetchForAllocation
);

/**
Expand Down
Loading
Loading