Skip to content

Commit

Permalink
Add query context parameter for segment load wait (apache#15076)
Browse files Browse the repository at this point in the history
Add segmentLoadWait as a query context parameter. If this is true, the controller queries the broker and waits till the segments created (if any) have been loaded by the load rules. The controller also provides this information in the live reports and task reports. If this is false, the controller exits immediately after finishing the query.
  • Loading branch information
adarshsanjeev authored and ycp2 committed Nov 17, 2023
1 parent 85d2dad commit 8802e62
Show file tree
Hide file tree
Showing 4 changed files with 75 additions and 68 deletions.
1 change: 1 addition & 0 deletions docs/multi-stage-query/reference.md
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,7 @@ The following table lists the context parameters for the MSQ task engine:
| `durableShuffleStorage` | SELECT, INSERT, REPLACE <br /><br />Whether to use durable storage for shuffle mesh. To use this feature, configure the durable storage at the server level using `druid.msq.intermediate.storage.enable=true`). If these properties are not configured, any query with the context variable `durableShuffleStorage=true` fails with a configuration error. <br /><br /> | `false` |
| `faultTolerance` | SELECT, INSERT, REPLACE<br /><br /> Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` |
| `selectDestination` | SELECT<br /><br /> Controls where the final result of the select query is written. <br />Use `taskReport`(the default) to write select results to the task report. <b> This is not scalable since task reports size explodes for large results </b> <br/>Use `durableStorage` to write results to durable storage location. <b>For large results sets, its recommended to use `durableStorage` </b>. To configure durable storage see [`this`](#durable-storage) section. | `taskReport` |
| `waitTillSegmentsLoad` | INSERT, REPLACE<br /><br /> If set, the ingest query waits for the generated segment to be loaded before exiting, else the ingest query exits without waiting. The task and live reports contain the information about the status of loading segments if this flag is set. This will ensure that any future queries made after the ingestion exits will include results from the ingestion. The drawback is that the controller task will stall till the segments are loaded. | `false` |

## Joins

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -463,14 +463,18 @@ public TaskStatus runTask(final Closer closer)
}
}

boolean shouldWaitForSegmentLoad = MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context());

try {
releaseTaskLocks();
cleanUpDurableStorageIfNeeded();

if (queryKernel != null && queryKernel.isSuccess()) {
if (segmentLoadWaiter != null) {
// If successful and there are segments created, segmentLoadWaiter should wait for them to become available.
if (shouldWaitForSegmentLoad && segmentLoadWaiter != null) {
// If successful, there are segments created and segment load is enabled, segmentLoadWaiter should wait
// for them to become available.
log.info("Controller will now wait for segments to be loaded. The query has already finished executing,"
+ " and results will be included once the segments are loaded, even if this query is cancelled now.");
segmentLoadWaiter.waitForSegmentsToLoad();
}
}
Expand Down Expand Up @@ -1363,31 +1367,35 @@ private void publishAllSegments(final Set<DataSegment> segments) throws IOExcept
}
} else {
Set<String> versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
if (MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context())) {
segmentLoadWaiter = new SegmentLoadStatusFetcher(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getId(),
task.getDataSource(),
versionsToAwait,
segmentsWithTombstones.size(),
true
);
}
performSegmentPublish(
context.taskActionClient(),
SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones)
);
}
} else if (!segments.isEmpty()) {
Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
if (MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context())) {
segmentLoadWaiter = new SegmentLoadStatusFetcher(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getId(),
task.getDataSource(),
versionsToAwait,
segmentsWithTombstones.size(),
segments.size(),
true
);
performSegmentPublish(
context.taskActionClient(),
SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones)
);
}
} else if (!segments.isEmpty()) {
Set<String> versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet());
segmentLoadWaiter = new SegmentLoadStatusFetcher(
context.injector().getInstance(BrokerClient.class),
context.jsonMapper(),
task.getId(),
task.getDataSource(),
versionsToAwait,
segments.size(),
true
);
// Append mode.
performSegmentPublish(
context.taskActionClient(),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -41,13 +41,10 @@

import javax.annotation.Nullable;
import javax.ws.rs.core.MediaType;
import java.util.HashMap;
import java.util.Iterator;
import java.util.Map;
import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
import java.util.stream.Collectors;

/**
* Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table
Expand Down Expand Up @@ -84,14 +81,14 @@ public class SegmentLoadStatusFetcher implements AutoCloseable
+ "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n"
+ "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n"
+ "FROM sys.segments\n"
+ "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'";
+ "WHERE datasource = '%s' AND is_overshadowed = 0 AND version in (%s)";

private final BrokerClient brokerClient;
private final ObjectMapper objectMapper;
// Map of version vs latest load status.
private final Map<String, VersionLoadStatus> versionToLoadStatusMap;
private final AtomicReference<VersionLoadStatus> versionLoadStatusReference;
private final String datasource;
private final Set<String> versionsToAwait;
private final String versionsInClauseString;
private final int totalSegmentsGenerated;
private final boolean doWait;
// since live reports fetch the value in another thread, we need to use AtomicReference
Expand All @@ -112,8 +109,11 @@ public SegmentLoadStatusFetcher(
this.brokerClient = brokerClient;
this.objectMapper = objectMapper;
this.datasource = datasource;
this.versionsToAwait = new TreeSet<>(versionsToAwait);
this.versionToLoadStatusMap = new HashMap<>();
this.versionsInClauseString = String.join(
",",
versionsToAwait.stream().map(s -> StringUtils.format("'%s'", s)).collect(Collectors.toSet())
);
this.versionLoadStatusReference = new AtomicReference<>(new VersionLoadStatus(0, 0, 0, 0, totalSegmentsGenerated));
this.totalSegmentsGenerated = totalSegmentsGenerated;
this.status = new AtomicReference<>(new SegmentLoadWaiterStatus(
State.INIT,
Expand Down Expand Up @@ -145,8 +145,9 @@ public void waitForSegmentsToLoad()
final AtomicReference<Boolean> hasAnySegmentBeenLoaded = new AtomicReference<>(false);
try {
FutureUtils.getUnchecked(executorService.submit(() -> {
long lastLogMillis = -TimeUnit.MINUTES.toMillis(1);
try {
while (!versionsToAwait.isEmpty()) {
while (!(hasAnySegmentBeenLoaded.get() && versionLoadStatusReference.get().isLoadingComplete())) {
// Check the timeout and exit if exceeded.
long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis();
if (runningMillis > TIMEOUT_DURATION_MILLIS) {
Expand All @@ -159,29 +160,21 @@ public void waitForSegmentsToLoad()
return;
}

Iterator<String> iterator = versionsToAwait.iterator();
log.info(
"Fetching segment load status for datasource[%s] from broker for segment versions[%s]",
datasource,
versionsToAwait
);

// Query the broker for all pending versions
while (iterator.hasNext()) {
String version = iterator.next();

// Fetch the load status for this version from the broker
VersionLoadStatus loadStatus = fetchLoadStatusForVersion(version);
versionToLoadStatusMap.put(version, loadStatus);
hasAnySegmentBeenLoaded.set(hasAnySegmentBeenLoaded.get() || loadStatus.getUsedSegments() > 0);

// If loading is done for this stage, remove it from future loops.
if (hasAnySegmentBeenLoaded.get() && loadStatus.isLoadingComplete()) {
iterator.remove();
}
if (runningMillis - lastLogMillis >= TimeUnit.MINUTES.toMillis(1)) {
lastLogMillis = runningMillis;
log.info(
"Fetching segment load status for datasource[%s] from broker for segment versions[%s]",
datasource,
versionsInClauseString
);
}

if (!versionsToAwait.isEmpty()) {
// Fetch the load status from the broker
VersionLoadStatus loadStatus = fetchLoadStatusFromBroker();
versionLoadStatusReference.set(loadStatus);
hasAnySegmentBeenLoaded.set(hasAnySegmentBeenLoaded.get() || loadStatus.getUsedSegments() > 0);

if (!(hasAnySegmentBeenLoaded.get() && versionLoadStatusReference.get().isLoadingComplete())) {
// Update the status.
updateStatus(State.WAITING, startTime);
// Sleep for a bit before checking again.
Expand Down Expand Up @@ -216,50 +209,45 @@ private void waitIfNeeded(long waitTimeMillis) throws Exception
}

/**
* Updates the {@link #status} with the latest details based on {@link #versionToLoadStatusMap}
* Updates the {@link #status} with the latest details based on {@link #versionLoadStatusReference}
*/
private void updateStatus(State state, DateTime startTime)
{
int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, onDemandSegmentCount = 0, unknownSegmentCount = 0;
for (Map.Entry<String, VersionLoadStatus> entry : versionToLoadStatusMap.entrySet()) {
usedSegmentsCount += entry.getValue().getUsedSegments();
precachedSegmentCount += entry.getValue().getPrecachedSegments();
onDemandSegmentCount += entry.getValue().getOnDemandSegments();
unknownSegmentCount += entry.getValue().getUnknownSegments();
pendingSegmentCount += entry.getValue().getPendingSegments();
}

long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis();
VersionLoadStatus versionLoadStatus = versionLoadStatusReference.get();
status.set(
new SegmentLoadWaiterStatus(
state,
startTime,
runningMillis,
totalSegmentsGenerated,
usedSegmentsCount,
precachedSegmentCount,
onDemandSegmentCount,
pendingSegmentCount,
unknownSegmentCount
versionLoadStatus.getUsedSegments(),
versionLoadStatus.getPrecachedSegments(),
versionLoadStatus.getOnDemandSegments(),
versionLoadStatus.getPendingSegments(),
versionLoadStatus.getUnknownSegments()
)
);
}

/**
* Uses {@link #brokerClient} to fetch latest load status for a given version. Converts the response into a
* Uses {@link #brokerClient} to fetch latest load status for a given set of versions. Converts the response into a
* {@link VersionLoadStatus} and returns it.
*/
private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Exception
private VersionLoadStatus fetchLoadStatusFromBroker() throws Exception
{
Request request = brokerClient.makeRequest(HttpMethod.POST, "/druid/v2/sql/");
SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, version),
SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, versionsInClauseString),
ResultFormat.OBJECTLINES,
false, false, false, null, null
);
request.setContent(MediaType.APPLICATION_JSON, objectMapper.writeValueAsBytes(sqlQuery));
String response = brokerClient.sendQuery(request);

if (response.trim().isEmpty()) {
if (response == null) {
// Unable to query broker
return new VersionLoadStatus(0, 0, 0, 0, totalSegmentsGenerated);
} else if (response.trim().isEmpty()) {
// If no segments are returned for a version, all segments have been dropped by a drop rule.
return new VersionLoadStatus(0, 0, 0, 0, 0);
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,8 @@ public class MultiStageQueryContext

public static final String CTX_FAULT_TOLERANCE = "faultTolerance";
public static final boolean DEFAULT_FAULT_TOLERANCE = false;
public static final String CTX_SEGMENT_LOAD_WAIT = "waitTillSegmentsLoad";
public static final boolean DEFAULT_SEGMENT_LOAD_WAIT = false;
public static final String CTX_MAX_INPUT_BYTES_PER_WORKER = "maxInputBytesPerWorker";

public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode";
Expand Down Expand Up @@ -148,6 +150,14 @@ public static boolean isFaultToleranceEnabled(final QueryContext queryContext)
);
}

public static boolean shouldWaitForSegmentLoad(final QueryContext queryContext)
{
return queryContext.getBoolean(
CTX_SEGMENT_LOAD_WAIT,
DEFAULT_SEGMENT_LOAD_WAIT
);
}

public static boolean isReindex(final QueryContext queryContext)
{
return queryContext.getBoolean(
Expand Down

0 comments on commit 8802e62

Please sign in to comment.